Merge release 0.9.7 into master
Change-Id:If97ac8dad8df435c9a0c15ef4d7a7ef12bb3acde
diff --git a/README.md b/README.md
index c996c75..3b52c76 100644
--- a/README.md
+++ b/README.md
@@ -27,9 +27,7 @@
extending JSON with object database ideas
- __Query languages__<br/>
-Two expressive and declarative query languages ([SQL++](http://asterixdb.apache.org/docs/0.9.1/sqlpp/manual.html)
-and [AQL](http://asterixdb.apache.org/docs/0.9.1/aql/manual.html)) that support a broad range of queries and analysis
-over semistructured data
+An expressive and declarative query language ([SQL++](http://asterixdb.apache.org/docs/0.9.7/sqlpp/manual.html) that supports a broad range of queries and analysis over semistructured data
- __Scalability__<br/>
A parallel runtime query execution engine, Apache Hyracks, that has been scale-tested on up to 1000+ cores and 500+ disks
@@ -95,6 +93,7 @@
are used in the documentation that are generated directly from the grammar.
* [master](https://ci.apache.org/projects/asterixdb/index.html) |
+ [0.9.7](http://asterixdb.apache.org/docs/0.9.7/index.html) |
[0.9.6](http://asterixdb.apache.org/docs/0.9.6/index.html) |
[0.9.5](http://asterixdb.apache.org/docs/0.9.5/index.html) |
[0.9.4.1](http://asterixdb.apache.org/docs/0.9.4.1/index.html) |
diff --git a/asterixdb/LICENSE b/asterixdb/LICENSE
index 4b2db69..cc61b93 100644
--- a/asterixdb/LICENSE
+++ b/asterixdb/LICENSE
@@ -301,15 +301,6 @@
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.
---
- Portions of the AsterixDB runtime
- located at:
- asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/*
-
- are available under The Apache License, Version 2.0:
----
- Source files in asterix-hivecompat are derived from portions of Apache Hive
- Query Language v0.13.0 (org.apache.hive:hive-exec).
----
Portions of the AsterixDB Dashboard fonts
located at:
asterix-dashboard/src/main/resources/assets/fonts/material-icons/MaterialIcons-Regular.eot,
diff --git a/asterixdb/NOTICE b/asterixdb/NOTICE
index 4aabe27..17c97bb 100644
--- a/asterixdb/NOTICE
+++ b/asterixdb/NOTICE
@@ -1,5 +1,5 @@
Apache AsterixDB
-Copyright 2015-2021 The Apache Software Foundation
+Copyright 2015-2022 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java
deleted file mode 100644
index 7c047c3..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java
+++ /dev/null
@@ -1,82 +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.
- */
-
-package org.apache.asterix.active;
-
-import java.util.Map;
-
-import org.apache.asterix.common.metadata.DataverseName;
-
-public class Activity implements Comparable<Activity> {
-
- protected int activityId;
- protected final EntityId activeEntityId;
- protected final Map<String, String> activityDetails;
-
- public Activity(EntityId activeEntityId, Map<String, String> activityDetails) {
- this.activeEntityId = activeEntityId;
- this.activityDetails = activityDetails;
- }
-
- public DataverseName getDataverseName() {
- return activeEntityId.getDataverseName();
- }
-
- public String getActiveEntityName() {
- return activeEntityId.getEntityName();
- }
-
- @Override
- public boolean equals(Object other) {
- if (this == other) {
- return true;
- }
- if (!(other instanceof Activity)) {
- return false;
- }
- return ((Activity) other).activeEntityId.equals(activeEntityId)
- && ((Activity) other).getActivityId() != (activityId);
- }
-
- @Override
- public int hashCode() {
- return toString().hashCode();
- }
-
- @Override
- public String toString() {
- return activeEntityId + "." + activityId;
- }
-
- public int getActivityId() {
- return activityId;
- }
-
- public void setActivityId(int activityId) {
- this.activityId = activityId;
- }
-
- public Map<String, String> getActivityDetails() {
- return activityDetails;
- }
-
- @Override
- public int compareTo(Activity o) {
- return o.getActivityId() - this.activityId;
- }
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
deleted file mode 100644
index 477bb5a..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/CountRetryPolicyFactory.java
+++ /dev/null
@@ -1,37 +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.
- */
-package org.apache.asterix.active;
-
-import org.apache.hyracks.util.CountRetryPolicy;
-import org.apache.hyracks.util.IRetryPolicy;
-
-public class CountRetryPolicyFactory implements IRetryPolicyFactory {
-
- private final int count;
-
- public CountRetryPolicyFactory(int count) {
- this.count = count;
- }
-
- @Override
- public IRetryPolicy create(IActiveEntityEventsListener listener) {
- return new CountRetryPolicy(count);
- }
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveMessage.java
deleted file mode 100644
index e4c7171..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveMessage.java
+++ /dev/null
@@ -1,44 +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.
- */
-package org.apache.asterix.active;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.dataflow.value.JSONSerializable;
-
-/**
- * @deprecated
- * This interface is expected to go away. instead, one should use the IMessageBroker interfaces to exchange
- * messages
- */
-@Deprecated
-public interface IActiveMessage extends Serializable, JSONSerializable {
-
- public enum MessageType {
- END
- }
-
- /**
- * Gets the type associated with this message
- *
- * @return MessageType type associated with this message
- */
- public MessageType getMessageType();
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
index 1d29828..60bfa69 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveNotificationHandler.java
@@ -33,19 +33,6 @@
void recover();
/**
- * Set whether handler initialization has completed or not
- *
- * @param initialized
- * @throws HyracksDataException
- */
- void setInitialized(boolean initialized) throws HyracksDataException;
-
- /**
- * @return true if initialization has completed, false otherwise
- */
- boolean isInitialized();
-
- /**
* Register a listener for events of an active entity
*
* @param listener
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
deleted file mode 100644
index 6f43c64..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicy.java
+++ /dev/null
@@ -1,44 +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.
- */
-package org.apache.asterix.active;
-
-import org.apache.hyracks.util.IRetryPolicy;
-
-public class InfiniteRetryPolicy implements IRetryPolicy {
-
- private final IActiveEntityEventsListener listener;
-
- public InfiniteRetryPolicy(IActiveEntityEventsListener listener) {
- this.listener = listener;
- }
-
- @Override
- public boolean retry(Throwable failure) {
- synchronized (listener) {
- try {
- listener.wait(5000); //NOSONAR this method is being called in a while loop
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return false;
- }
- }
- return true;
- }
-
-}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
deleted file mode 100644
index d33e1da..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/InfiniteRetryPolicyFactory.java
+++ /dev/null
@@ -1,30 +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.
- */
-package org.apache.asterix.active;
-
-import org.apache.hyracks.util.IRetryPolicy;
-
-public class InfiniteRetryPolicyFactory implements IRetryPolicyFactory {
-
- @Override
- public IRetryPolicy create(IActiveEntityEventsListener listener) {
- return new InfiniteRetryPolicy(listener);
- }
-
-}
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 680fb1e..7a32f42 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -129,12 +129,6 @@
<groupId>org.apache.asterix</groupId>
<artifactId>asterix-external-data</artifactId>
<version>${project.version}</version>
- <exclusions>
- <exclusion>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-exec</artifactId>
- </exclusion>
- </exclusions>
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
@@ -231,10 +225,5 @@
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
</dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-fuzzyjoin</artifactId>
- <version>${project.version}</version>
- </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index ed8e9bf..06a6687 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -21,6 +21,8 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.DataSourceIndex;
import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -37,7 +39,6 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
@@ -46,6 +47,7 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
@@ -61,6 +63,7 @@
import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
/**
@@ -116,31 +119,45 @@
int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
+ boolean propagateFilter = unnestMap.propagateIndexFilter();
+ IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateFilter, context);
int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
- boolean propagateFilter = unnestMap.propagateIndexFilter();
MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
ITupleFilterFactory tupleFilterFactory = null;
long outputLimit = -1;
- if (unnestMap.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
- UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
- outputLimit = unnestMapOp.getOutputLimit();
- if (unnestMapOp.getSelectCondition() != null) {
- tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
- typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
- }
+ boolean retainMissing = false;
+ IMissingWriterFactory nonMatchWriterFactory = null;
+ switch (unnestMap.getOperatorTag()) {
+ case UNNEST_MAP:
+ UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
+ outputLimit = unnestMapOp.getOutputLimit();
+ if (unnestMapOp.getSelectCondition() != null) {
+ tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
+ typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
+ }
+ break;
+ case LEFT_OUTER_UNNEST_MAP:
+ // By nature, LEFT_OUTER_UNNEST_MAP should generate missing (or null) values for non-matching tuples.
+ retainMissing = true;
+ nonMatchWriterFactory =
+ getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMap).getMissingValue(), context,
+ unnestMap.getSourceLocation());
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, unnestMap.getSourceLocation(),
+ String.valueOf(unnestMap.getOperatorTag()));
}
- // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
- boolean retainMissing = op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP;
+
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
- builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing, dataset,
- jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
- jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes,
- tupleFilterFactory, outputLimit, unnestMap.getGenerateCallBackProceedResultVar(),
- isPrimaryIndexPointSearch(op));
+ builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing,
+ nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
+ jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
+ nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
+ unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
IOperatorDescriptor opDesc = btreeSearch.first;
opDesc.setSourceLocation(unnestMap.getSourceLocation());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
index 73fcf92..4de30ce 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
@@ -20,10 +20,13 @@
import java.util.List;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.declared.DataSourceId;
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.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
@@ -36,6 +39,9 @@
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
/**
* Class that embodies the commonalities between access method physical operators.
@@ -102,4 +108,21 @@
}
return ss;
}
+
+ protected static IMissingWriterFactory getNonMatchWriterFactory(IAlgebricksConstantValue missingValue,
+ JobGenContext context, SourceLocation sourceLoc) throws CompilationException {
+ IMissingWriterFactory nonMatchWriterFactory;
+ if (missingValue.isMissing()) {
+ nonMatchWriterFactory = context.getMissingWriterFactory();
+ } else if (missingValue.isNull()) {
+ nonMatchWriterFactory = context.getNullWriterFactory();
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, missingValue.toString());
+ }
+ return nonMatchWriterFactory;
+ }
+
+ protected static IMissingWriterFactory getNonFilterWriterFactory(boolean propagateFilter, JobGenContext context) {
+ return propagateFilter ? context.getMissingWriterFactory() : null;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 532fb43..00eef69 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -49,11 +49,13 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.algebra.properties.LocalMemoryRequirements;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -117,13 +119,18 @@
Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
+ boolean propagateIndexFilter = unnestMapOp.propagateIndexFilter();
+ IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateIndexFilter, context);
int[] minFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMinFilterVars(), inputSchemas);
int[] maxFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMaxFilterVars(), inputSchemas);
- boolean retainNull = false;
+ boolean retainMissing = false;
+ IMissingWriterFactory nonMatchWriterFactory = null;
if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
- // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching
- // tuples.
- retainNull = true;
+ // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples
+ retainMissing = true;
+ nonMatchWriterFactory =
+ getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMapOp).getMissingValue(), context,
+ unnestMapOp.getSourceLocation());
}
// In-memory budget (frame limit) for inverted-index search operations
int frameLimit = localMemoryRequirements.getMemoryBudgetInFrames();
@@ -131,9 +138,10 @@
// Build runtime.
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch =
buildInvertedIndexRuntime(metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema,
- jobGenParams.getRetainInput(), retainNull, jobGenParams.getDatasetName(), dataset,
- jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(), keyIndexes,
- jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold(),
+ jobGenParams.getRetainInput(), retainMissing, nonMatchWriterFactory,
+ jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(),
+ jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
+ jobGenParams.getSimilarityThreshold(), propagateIndexFilter, nonFilterWriterFactory,
minFilterFieldIndexes, maxFilterFieldIndexes, jobGenParams.getIsFullTextSearch(), frameLimit);
IOperatorDescriptor opDesc = invIndexSearch.first;
opDesc.setSourceLocation(unnestMapOp.getSourceLocation());
@@ -148,11 +156,11 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
MetadataProvider metadataProvider, JobGenContext context, JobSpecification jobSpec,
AbstractUnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, boolean retainMissing,
- String datasetName, Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
- SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
- int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int frameLimit)
- throws AlgebricksException {
- boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
+ IMissingWriterFactory nonMatchWriterFactory, String datasetName, Dataset dataset, String indexName,
+ ATypeTag searchKeyType, int[] keyFields, SearchModifierType searchModifierType,
+ IAlgebricksConstantValue similarityThreshold, boolean propagateIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+ boolean isFullTextSearchQuery, int frameLimit) throws AlgebricksException {
IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
int numPrimaryKeys = dataset.getPrimaryKeys().size();
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
@@ -178,14 +186,14 @@
IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), secondarySplitsAndConstraint.first);
- LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp =
- new LSMInvertedIndexSearchOperatorDescriptor(jobSpec, outputRecDesc, queryField, dataflowHelperFactory,
- queryTokenizerFactory, fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput,
- retainMissing, context.getMissingWriterFactory(),
- dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
- IndexOperation.SEARCH, null),
- minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
- propagateIndexFilter, frameLimit);
+ LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+ jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory,
+ fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput, retainMissing,
+ nonMatchWriterFactory,
+ dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
+ IndexOperation.SEARCH, null),
+ minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
+ propagateIndexFilter, nonFilterWriterFactory, frameLimit);
return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index a80e5b3..6534ebe 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -42,10 +42,12 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
/**
* Contributes the runtime operator for an unnest-map representing a RTree
@@ -82,6 +84,7 @@
int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
+ IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateIndexFilter, context);
int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
@@ -93,15 +96,20 @@
outputVars = new ArrayList<LogicalVariable>();
VariableUtilities.getLiveVariables(unnestMap, outputVars);
}
- boolean retainNull = false;
+ boolean retainMissing = false;
+ IMissingWriterFactory nonMatchWriterFactory = null;
if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
// By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
- retainNull = true;
+ retainMissing = true;
+ nonMatchWriterFactory = getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMap).getMissingValue(),
+ context, unnestMap.getSourceLocation());
}
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = mp.buildRtreeRuntime(
- builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
- dataset, jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, minFilterFieldIndexes,
- maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch =
+ mp.buildRtreeRuntime(builder.getJobSpec(), outputVars, opSchema, typeEnv, context,
+ jobGenParams.getRetainInput(), retainMissing, nonMatchWriterFactory, dataset,
+ jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, nonFilterWriterFactory,
+ minFilterFieldIndexes, maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
IOperatorDescriptor opDesc = rtreeSearch.first;
opDesc.setSourceLocation(unnestMap.getSourceLocation());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/SpatialJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/SpatialJoinPOperator.java
new file mode 100644
index 0000000..c0d9f4b
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/SpatialJoinPOperator.java
@@ -0,0 +1,180 @@
+/*
+ * 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.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.asterix.runtime.operators.joins.spatial.PlaneSweepJoinOperatorDescriptor;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtilFactory;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.ListSet;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+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.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+
+/**
+ * The right input is broadcast and the left input can be partitioned in any way.
+ */
+public class SpatialJoinPOperator extends AbstractJoinPOperator {
+
+ private final List<LogicalVariable> keysLeftBranch;
+ private final List<LogicalVariable> keysRightBranch;
+
+ protected final ISpatialJoinUtilFactory mjcf;
+ private final int memSizeInFrames;
+
+ public SpatialJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+ List<LogicalVariable> keysLeftBranch, List<LogicalVariable> keysRightBranch, int memSizeInFrames,
+ ISpatialJoinUtilFactory mjcf) {
+ super(kind, partitioningType);
+ this.keysLeftBranch = keysLeftBranch;
+ this.keysRightBranch = keysRightBranch;
+ this.mjcf = mjcf;
+ this.memSizeInFrames = memSizeInFrames;
+ }
+
+ public List<LogicalVariable> getKeysLeftBranch() {
+ return keysLeftBranch;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SPATIAL_JOIN;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "SPATIAL_JOIN" + " " + keysLeftBranch + " " + keysRightBranch;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context)
+ throws AlgebricksException {
+ IPartitioningProperty pp;
+ AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.LOCAL) {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ IPhysicalPropertiesVector pv0 = op0.getPhysicalOperator().getDeliveredProperties();
+ AbstractLogicalOperator op1 = (AbstractLogicalOperator) op.getInputs().get(1).getValue();
+ IPhysicalPropertiesVector pv1 = op1.getPhysicalOperator().getDeliveredProperties();
+
+ if (pv0 == null || pv1 == null) {
+ pp = null;
+ } else {
+ pp = pv0.getPartitioningProperty();
+ }
+ } else {
+ pp = IPartitioningProperty.UNPARTITIONED;
+ }
+ this.deliveredProperties = new StructuralPropertiesVector(pp, deliveredLocalProperties(iop, context));
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+ List<LogicalVariable> keysLeftBranchTileId = new ArrayList<>();
+ keysLeftBranchTileId.add(keysLeftBranch.get(0));
+ List<LogicalVariable> keysRightBranchTileId = new ArrayList<>();
+ keysRightBranchTileId.add(keysRightBranch.get(0));
+ IPartitioningProperty pp1 = new UnorderedPartitionedProperty(new ListSet<>(keysLeftBranchTileId),
+ context.getComputationNodeDomain());
+ IPartitioningProperty pp2 = new UnorderedPartitionedProperty(new ListSet<>(keysRightBranchTileId),
+ context.getComputationNodeDomain());
+
+ List<ILocalStructuralProperty> localProperties1 = new ArrayList<>();
+ List<OrderColumn> orderColumns1 = new ArrayList<OrderColumn>();
+ orderColumns1.add(new OrderColumn(keysLeftBranch.get(0), OrderOperator.IOrder.OrderKind.ASC));
+ orderColumns1.add(new OrderColumn(keysLeftBranch.get(1), OrderOperator.IOrder.OrderKind.ASC));
+ localProperties1.add(new LocalOrderProperty(orderColumns1));
+
+ List<ILocalStructuralProperty> localProperties2 = new ArrayList<>();
+ List<OrderColumn> orderColumns2 = new ArrayList<OrderColumn>();
+ orderColumns2.add(new OrderColumn(keysRightBranch.get(0), OrderOperator.IOrder.OrderKind.ASC));
+ orderColumns2.add(new OrderColumn(keysRightBranch.get(1), OrderOperator.IOrder.OrderKind.ASC));
+ localProperties2.add(new LocalOrderProperty(orderColumns2));
+
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+ pv[0] = new StructuralPropertiesVector(pp1, localProperties1);
+ pv[1] = new StructuralPropertiesVector(pp2, localProperties2);
+
+ return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ int[] keysBuild = JobGenHelper.variablesToFieldIndexes(keysLeftBranch, inputSchemas[0]);
+ int[] keysProbe = JobGenHelper.variablesToFieldIndexes(keysRightBranch, inputSchemas[1]);
+
+ IOperatorDescriptorRegistry spec = builder.getJobSpec();
+ RecordDescriptor recordDescriptor =
+ JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+
+ IOperatorDescriptor opDesc = new PlaneSweepJoinOperatorDescriptor(spec, memSizeInFrames, keysBuild, keysProbe,
+ recordDescriptor, mjcf);
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src1 = op.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(src1, 0, op, 0);
+ ILogicalOperator src2 = op.getInputs().get(1).getValue();
+ builder.contributeGraphEdge(src2, 0, op, 1);
+ }
+
+ protected List<ILocalStructuralProperty> deliveredLocalProperties(ILogicalOperator op,
+ IOptimizationContext context) {
+ AbstractLogicalOperator op0 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ IPhysicalPropertiesVector pv0 = op0.getPhysicalOperator().getDeliveredProperties();
+ List<ILocalStructuralProperty> lp0 = pv0.getLocalProperties();
+ if (lp0 != null) {
+ // maintains the local properties on the probe side
+ return new LinkedList<>(lp0);
+ }
+ return new LinkedList<>();
+ }
+}
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 b6e287b..5493dd1 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
@@ -45,6 +45,8 @@
import org.apache.asterix.optimizer.rules.ExtractOrderExpressionsRule;
import org.apache.asterix.optimizer.rules.ExtractWindowExpressionsRule;
import org.apache.asterix.optimizer.rules.FeedScanCollectionToUnnest;
+import org.apache.asterix.optimizer.rules.FilterRefineSpatialJoinRuleForSTDistanceFunction;
+import org.apache.asterix.optimizer.rules.FilterRefineSpatialJoinRuleForSTFunctions;
import org.apache.asterix.optimizer.rules.FindDataSourcesRule;
import org.apache.asterix.optimizer.rules.FixReplicateOperatorOutputsRule;
import org.apache.asterix.optimizer.rules.FullTextContainsParameterCheckAndSetRule;
@@ -66,15 +68,16 @@
import org.apache.asterix.optimizer.rules.LoadRecordFieldsRule;
import org.apache.asterix.optimizer.rules.MetaFunctionToMetaVariableRule;
import org.apache.asterix.optimizer.rules.NestGroupByRule;
+import org.apache.asterix.optimizer.rules.PullSelectOutOfSpatialJoin;
import org.apache.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
import org.apache.asterix.optimizer.rules.PushAggregateIntoNestedSubplanRule;
import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
-import org.apache.asterix.optimizer.rules.PushFieldAccessToExternalDataScanRule;
import org.apache.asterix.optimizer.rules.PushGroupByThroughProduct;
import org.apache.asterix.optimizer.rules.PushLimitIntoOrderByRule;
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.PushValueAccessToExternalDataScanRule;
import org.apache.asterix.optimizer.rules.RemoveDuplicateFieldsRule;
import org.apache.asterix.optimizer.rules.RemoveLeftOuterUnnestForLeftOuterJoinRule;
import org.apache.asterix.optimizer.rules.RemoveRedundantListifyRule;
@@ -121,6 +124,7 @@
import org.apache.hyracks.algebricks.rewriter.rules.IntroduceAggregateCombinerRule;
import org.apache.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
import org.apache.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
+import org.apache.hyracks.algebricks.rewriter.rules.PopulateResultMetadataRule;
import org.apache.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
import org.apache.hyracks.algebricks.rewriter.rules.PushGroupByIntoSortRule;
import org.apache.hyracks.algebricks.rewriter.rules.PushMapOperatorDownThroughProductRule;
@@ -147,7 +151,6 @@
import org.apache.hyracks.algebricks.rewriter.rules.subplan.EliminateSubplanWithInputCardinalityOneRule;
import org.apache.hyracks.algebricks.rewriter.rules.subplan.NestedSubplanToJoinRule;
import org.apache.hyracks.algebricks.rewriter.rules.subplan.PushSubplanIntoGroupByRule;
-import org.apache.hyracks.algebricks.rewriter.rules.subplan.SubplanOutOfGroupRule;
public final class RuleCollections {
@@ -237,7 +240,6 @@
condPushDownAndJoinInference.add(new PushSubplanWithAggregateDownThroughProductRule());
condPushDownAndJoinInference
.add(new AsterixPushMapOperatorThroughUnionRule(LogicalOperatorTag.ASSIGN, LogicalOperatorTag.SELECT));
- condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
condPushDownAndJoinInference.add(new AsterixExtractFunctionsFromJoinConditionRule());
@@ -278,6 +280,8 @@
fieldLoads.add(new NestedSubplanToJoinRule());
fieldLoads.add(new InlineSubplanInputForNestedTupleSourceRule());
fieldLoads.add(new RemoveLeftOuterUnnestForLeftOuterJoinRule());
+ fieldLoads.add(new FilterRefineSpatialJoinRuleForSTFunctions());
+ fieldLoads.add(new FilterRefineSpatialJoinRuleForSTDistanceFunction());
return fieldLoads;
}
@@ -373,6 +377,8 @@
physicalRewritesAllLevels.add(new ConsolidateAssignsRule(true));
// After adding projects, we may need need to set physical operators again.
physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
+ // Optimized spatial join's query plan produces more join conditions, so we need to pull out these conditions
+ physicalRewritesAllLevels.add(new PullSelectOutOfSpatialJoin());
return physicalRewritesAllLevels;
}
@@ -385,11 +391,12 @@
// We are going to apply a constant folding rule again for this case.
physicalRewritesTopLevel.add(new ConstantFoldingRule(appCtx));
physicalRewritesTopLevel.add(new PushLimitIntoOrderByRule());
+ //Must run before PushLimitIntoPrimarySearchRule to ensure the select condition is inspected
+ physicalRewritesTopLevel.add(new PushValueAccessToExternalDataScanRule());
physicalRewritesTopLevel.add(new PushLimitIntoPrimarySearchRule());
// remove assigns that could become unused after PushLimitIntoPrimarySearchRule
physicalRewritesTopLevel.add(new RemoveUnusedAssignAndAggregateRule());
physicalRewritesTopLevel.add(new IntroduceProjectsRule());
- physicalRewritesTopLevel.add(new PushFieldAccessToExternalDataScanRule());
physicalRewritesTopLevel.add(new SetAsterixPhysicalOperatorsRule());
physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
physicalRewritesTopLevel.add(new SetExecutionModeRule());
@@ -412,6 +419,7 @@
prepareForJobGenRewrites.add(new SetAsterixMemoryRequirementsRule());
prepareForJobGenRewrites.add(new SweepIllegalNonfunctionalFunctions());
prepareForJobGenRewrites.add(new FixReplicateOperatorOutputsRule());
+ prepareForJobGenRewrites.add(new PopulateResultMetadataRule());
return prepareForJobGenRewrites;
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
index 0124d25..9cbed97 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
@@ -31,7 +31,7 @@
@Override
protected boolean isComparisonFunction(FunctionIdentifier fi) {
- return BuiltinFunctions.isSimilarityFunction(fi);
+ return BuiltinFunctions.isSimilarityFunction(fi) || BuiltinFunctions.isSpatialFilterFunction(fi);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
index 83e62b8..df2e1ff 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
@@ -58,7 +58,7 @@
ScalarFunctionCallExpression nonSystemNullTest =
new ScalarFunctionCallExpression(finfoNot, new MutableObject<>(isSystemNullTest));
nonSystemNullTest.setSourceLocation(sourceLoc);
- selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest), false, null);
+ selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest));
selectNonSystemNull.setSourceLocation(sourceLoc);
} else {
List<Mutable<ILogicalExpression>> isSystemNullTestList = new ArrayList<>();
@@ -76,7 +76,7 @@
}
IFunctionInfo finfoAnd = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
selectNonSystemNull = new SelectOperator(
- new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)), false, null);
+ new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)));
selectNonSystemNull.setSourceLocation(sourceLoc);
}
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 0c2eeba..03d112d 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
@@ -33,6 +33,7 @@
import org.apache.asterix.common.exceptions.WarningCollector;
import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
+import org.apache.asterix.dataflow.data.nontagged.NullWriterFactory;
import org.apache.asterix.formats.nontagged.ADMPrinterFactoryProvider;
import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
@@ -125,7 +126,8 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) {
throw new IllegalStateException();
}
@@ -149,7 +151,7 @@
BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE,
- UnnestingPositionWriterFactory.INSTANCE, null,
+ NullWriterFactory.INSTANCE, UnnestingPositionWriterFactory.INSTANCE, null,
new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
@@ -233,12 +235,15 @@
try {
if (expr.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
- ARecordType rt = (ARecordType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
- String str = ConstantExpressionUtil.getStringConstant(expr.getArguments().get(1).getValue());
- int k = rt.getFieldIndex(str);
- if (k >= 0) {
- // wait for the ByNameToByIndex rule to apply
- return new Pair<>(changed, expr);
+ IAType argType = (IAType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
+ if (argType.getTypeTag() == ATypeTag.OBJECT) {
+ ARecordType rt = (ARecordType) argType;
+ String str = ConstantExpressionUtil.getStringConstant(expr.getArguments().get(1).getValue());
+ int k = rt.getFieldIndex(str);
+ if (k >= 0) {
+ // wait for the ByNameToByIndex rule to apply
+ return new Pair<>(changed, expr);
+ }
}
}
IAObject c = FUNC_ID_TO_CONSTANT.get(expr.getFunctionIdentifier());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTDistanceFunction.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTDistanceFunction.java
new file mode 100644
index 0000000..254e254
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTDistanceFunction.java
@@ -0,0 +1,163 @@
+/*
+ * 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.List;
+
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * If the join condition is st_distance(), this rule applies the spatial join into the query
+ * by adding the spatial-intersect function and sends the extended mbr of the geometries to it.
+ *
+ * For example:<br/>
+ *
+ * join (lt(st-distance($$54, $$55), 1)) -- |UNPARTITIONED|
+ * assign [$$54] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$48, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ * assign [$$55] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$49, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ *
+ * Becomes,
+ *
+ * join (and(spatial-intersect(st-mbr-enlarge($$54, 1), st-mbr($$55)), lt(st-distance($$54, $$55), 1))) -- |UNPARTITIONED|
+ * assign [$$54] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$48, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ * assign [$$55] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$49, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ *
+ * st-mbr() computes the mbr of a Geometry, and st-mbr-enlarge() computes the mbr of a Geometry and extending
+ * by the second parameter.
+ *
+ * The /*+ spatial-partitioning(x1, y1, x2, y2, row, col) */ annotation allows users to define the MBR and
+ * grid size (row,col) and these are used for partitioning. If the query does not have an annotation, the MBR is
+ * computed dynamically and grid size set to 100 100.
+ */
+public class FilterRefineSpatialJoinRuleForSTDistanceFunction 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.INNERJOIN) {
+ return false;
+ }
+
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+ Mutable<ILogicalExpression> joinConditionRef = joinOp.getCondition();
+ ILogicalExpression joinCondition = joinConditionRef.getValue();
+
+ if (joinCondition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) joinCondition;
+ if (!(funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.LT)
+ || funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.LE)
+ || funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.EQ))) {
+ return false;
+ }
+
+ List<Mutable<ILogicalExpression>> inputExprs = funcExpr.getArguments();
+
+ ILogicalExpression leftOperatingExpr = inputExprs.get(0).getValue();
+ ILogicalExpression rightOperatingExpr = inputExprs.get(1).getValue();
+
+ if (leftOperatingExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL
+ || rightOperatingExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression distanceFuncCallExpr = (AbstractFunctionCallExpression) leftOperatingExpr;
+ ConstantExpression distanceValExpr = (ConstantExpression) rightOperatingExpr;
+
+ if (!distanceFuncCallExpr.getFunctionIdentifier().equals(BuiltinFunctions.ST_DISTANCE)) {
+ return false;
+ }
+
+ // Left and right arguments of the st_distance function should be either variable or function call.
+ List<Mutable<ILogicalExpression>> distanceFuncCallArgs = distanceFuncCallExpr.getArguments();
+ Mutable<ILogicalExpression> distanceFuncCallLeftArg = distanceFuncCallArgs.get(0);
+ Mutable<ILogicalExpression> distanceFuncCallRightArg = distanceFuncCallArgs.get(1);
+ if (distanceFuncCallLeftArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT
+ || distanceFuncCallRightArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ return false;
+ }
+
+ // Enlarge the MBR of the left argument of the refine function (st_distance)
+ IAlgebricksConstantValue distanceVal = distanceValExpr.getValue();
+ ScalarFunctionCallExpression enlargedLeft = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR_ENLARGE), distanceFuncCallLeftArg,
+ new MutableObject<>(new ConstantExpression(distanceVal)));
+ enlargedLeft.setSourceLocation(distanceFuncCallLeftArg.getValue().getSourceLocation());
+ // Compute the MBR of the right argument of the refine function (st_distance)
+ ScalarFunctionCallExpression rightMBR = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR), distanceFuncCallRightArg);
+ rightMBR.setSourceLocation(distanceFuncCallRightArg.getValue().getSourceLocation());
+
+ // Create filter function (spatial_intersect)
+ ScalarFunctionCallExpression spatialIntersect = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SPATIAL_INTERSECT),
+ new MutableObject<>(enlargedLeft.cloneExpression()), new MutableObject<>(rightMBR.cloneExpression()));
+ spatialIntersect.setSourceLocation(op.getSourceLocation());
+
+ // Attach the annotation to the spatial_intersect function if it is available
+ if (distanceFuncCallExpr.getAnnotation(SpatialJoinAnnotation.class) != null) {
+ spatialIntersect.putAnnotation(distanceFuncCallExpr.getAnnotation(SpatialJoinAnnotation.class));
+ }
+
+ // Update join condition with filter and refine function
+ ScalarFunctionCallExpression updatedJoinCondition =
+ new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND),
+ new MutableObject<>(spatialIntersect), new MutableObject<>(funcExpr));
+ updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+
+ joinConditionRef.setValue(updatedJoinCondition);
+
+ return true;
+ }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTFunctions.java
new file mode 100644
index 0000000..fecd91c
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTFunctions.java
@@ -0,0 +1,147 @@
+/*
+ * 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.List;
+
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * If the join condition is one of the below Geometry functions,
+ * this rule applies the spatial join by adding the spatial-intersect condition into the join predicate.
+ *
+ * <ul>
+ * <li>st_intersects</li>
+ * <li>st_overlaps</li>
+ * <li>st_touches</li>
+ * <li>st_contains</li>
+ * <li>st_crosses</li>
+ * <li>st_within</li>
+ * </ul>
+ *
+ *
+ * For example:<br/>
+ *
+ * join (st-intersects($$50, $$51)) -- |UNPARTITIONED|
+ * assign [$$50] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$47, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ * assign [$$51] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$48, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ *
+ * Becomes,
+ *
+ * join (and(spatial-intersect(st-mbr($$50), st-mbr($$51)), st-intersects($$50, $$51))) -- |UNPARTITIONED|
+ * assign [$$50] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$47, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ * assign [$$51] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ * data-scan []<-[$$48, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ * empty-tuple-source -- |UNPARTITIONED|
+ *
+ * st-mbr($x, $y) computes the mbr of the geometry and returns rectangles to pass it spatial_intersect($x, $y)
+ *
+ * The /*+ spatial-partitioning(x1, y1, x2, y2, row, col) */ annotation allows users to define the MBR and
+ * grid size (row,col) which are used for partitioning. If the query does not have an annotation, the MBR is computed
+ * dynamically and grid size set to 100 100.
+ */
+public class FilterRefineSpatialJoinRuleForSTFunctions 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.INNERJOIN) {
+ return false;
+ }
+
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+ Mutable<ILogicalExpression> joinConditionRef = joinOp.getCondition();
+ ILogicalExpression joinCondition = joinConditionRef.getValue();
+
+ if (joinCondition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+
+ AbstractFunctionCallExpression stFuncExpr = (AbstractFunctionCallExpression) joinCondition;
+ if (!BuiltinFunctions.isSTFilterRefineFunction(stFuncExpr.getFunctionIdentifier())) {
+ return false;
+ }
+
+ // Left and right arguments of the refine function should be either variable or function call.
+ List<Mutable<ILogicalExpression>> stFuncArgs = stFuncExpr.getArguments();
+ Mutable<ILogicalExpression> stFuncLeftArg = stFuncArgs.get(0);
+ Mutable<ILogicalExpression> stFuncRightArg = stFuncArgs.get(1);
+ if (stFuncLeftArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT
+ || stFuncRightArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ return false;
+ }
+
+ // Compute MBRs of the left and right arguments of the refine function
+ ScalarFunctionCallExpression left = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR), stFuncLeftArg);
+ left.setSourceLocation(stFuncLeftArg.getValue().getSourceLocation());
+ ScalarFunctionCallExpression right = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR), stFuncRightArg);
+ right.setSourceLocation(stFuncRightArg.getValue().getSourceLocation());
+
+ // Create filter function (spatial_intersect)
+ ScalarFunctionCallExpression spatialIntersect = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SPATIAL_INTERSECT),
+ new MutableObject<>(left.cloneExpression()), new MutableObject<>(right.cloneExpression()));
+ spatialIntersect.setSourceLocation(op.getSourceLocation());
+
+ // Attach the annotation to the spatial_intersect function if it is available
+ if (stFuncExpr.getAnnotation(SpatialJoinAnnotation.class) != null) {
+ spatialIntersect.putAnnotation(stFuncExpr.getAnnotation(SpatialJoinAnnotation.class));
+ }
+
+ // Update join condition with filter and refine function
+ ScalarFunctionCallExpression updatedJoinCondition =
+ new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND),
+ new MutableObject<>(spatialIntersect), new MutableObject<>(stFuncExpr));
+ updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+ joinConditionRef.setValue(updatedJoinCondition);
+
+ return true;
+ }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index c3859be..3fc178a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -18,14 +18,13 @@
*/
package org.apache.asterix.optimizer.rules;
-import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.Deque;
import java.util.HashMap;
-import java.util.HashSet;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
@@ -43,11 +42,13 @@
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctionInfo;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
import org.apache.asterix.om.types.AOrderedListType;
@@ -313,34 +314,46 @@
// Set our key variables and expressions for non-array indexes. Our secondary keys for array indexes will
// always be an empty list.
List<LogicalVariable> secondaryKeyVars = new ArrayList<>();
+ List<LogicalVariable> beforeOpSecondaryKeyVars = new ArrayList<>();
List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<>();
List<Mutable<ILogicalExpression>> beforeOpSecondaryExpressions = new ArrayList<>();
ILogicalOperator replicateOutput;
if (!index.getIndexType().equals(IndexType.ARRAY)) {
for (int i = 0; i < secondaryKeyFields.size(); i++) {
- IndexFieldId indexFieldId = new IndexFieldId(secondaryKeySources.get(i), secondaryKeyFields.get(i),
- secondaryKeyTypes.get(i).getTypeTag());
+ IAType skType = secondaryKeyTypes.get(i);
+ Integer skSrc = secondaryKeySources.get(i);
+ List<String> skName = secondaryKeyFields.get(i);
+ ARecordType sourceType = dataset.hasMetaPart()
+ ? skSrc.intValue() == Index.RECORD_INDICATOR ? recType : metaType : recType;
+ IndexFieldId indexFieldId = createIndexFieldId(index, skName, skType, skSrc, sourceType, sourceLoc);
LogicalVariable skVar = fieldVarsForNewRecord.get(indexFieldId);
secondaryKeyVars.add(skVar);
VariableReferenceExpression skVarRef = new VariableReferenceExpression(skVar);
skVarRef.setSourceLocation(sourceLoc);
- secondaryExpressions.add(new MutableObject<ILogicalExpression>(skVarRef));
+ secondaryExpressions.add(new MutableObject<>(skVarRef));
if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
- VariableReferenceExpression varRef =
- new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId));
+ LogicalVariable beforeKeyVar = fieldVarsForBeforeOperation.get(indexFieldId);
+ beforeOpSecondaryKeyVars.add(beforeKeyVar);
+ VariableReferenceExpression varRef = new VariableReferenceExpression(beforeKeyVar);
varRef.setSourceLocation(sourceLoc);
- beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(varRef));
+ beforeOpSecondaryExpressions.add(new MutableObject<>(varRef));
}
}
}
IndexInsertDeleteUpsertOperator indexUpdate;
if (index.getIndexType() != IndexType.RTREE) {
+ // B-Tree, inverted index, array index
// Create an expression per key
Mutable<ILogicalExpression> filterExpression =
- (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
- : createFilterExpression(secondaryKeyVars, context.getOutputTypeEnvironment(currentTop),
- index.getIndexDetails().isOverridingKeyFieldTypes());
+ createFilterExpression(index, secondaryKeyVars, context.getOutputTypeEnvironment(currentTop),
+ index.getIndexDetails().isOverridingKeyFieldTypes());
+ Mutable<ILogicalExpression> beforeOpFilterExpression = null;
+ if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
+ beforeOpFilterExpression = createFilterExpression(index, beforeOpSecondaryKeyVars,
+ context.getOutputTypeEnvironment(currentTop),
+ index.getIndexDetails().isOverridingKeyFieldTypes());
+ }
DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
// Introduce the TokenizeOperator only when doing bulk-load,
@@ -370,8 +383,8 @@
// Check the field type of the secondary key.
IAType secondaryKeyType;
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
- secondaryKeyFields.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index,
+ secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
secondaryKeyType = keyPairType.first;
List<Object> varTypes = new ArrayList<>();
@@ -406,8 +419,8 @@
indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
OperatorManipulationUtil
.cloneExpressions(primaryIndexModificationOp.getPrimaryKeyExpressions()),
- tokenizeKeyExprs, filterExpression, primaryIndexModificationOp.getOperation(),
- primaryIndexModificationOp.isBulkload(),
+ tokenizeKeyExprs, filterExpression, beforeOpFilterExpression,
+ primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
: primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
indexUpdate.setSourceLocation(sourceLoc);
@@ -419,8 +432,8 @@
indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
OperatorManipulationUtil
.cloneExpressions(primaryIndexModificationOp.getPrimaryKeyExpressions()),
- secondaryExpressions, filterExpression, primaryIndexModificationOp.getOperation(),
- primaryIndexModificationOp.isBulkload(),
+ secondaryExpressions, filterExpression, beforeOpFilterExpression,
+ primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
: primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
indexUpdate.setSourceLocation(sourceLoc);
@@ -449,11 +462,11 @@
context.computeAndSetTypeEnvironmentForOperator(unnestSourceOp);
UnnestBranchCreator unnestSIDXBranch = buildUnnestBranch(unnestSourceOp, index, newRecordVar,
newMetaVar, recType, metaType, dataset.hasMetaPart());
- unnestSIDXBranch.applyProjectDistinct();
+ unnestSIDXBranch.applyProjectOnly();
// If there exists a filter expression, add it to the top of our nested plan.
filterExpression = (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
- : createFilterExpression(unnestSIDXBranch.lastFieldVars,
+ : createAnyUnknownFilterExpression(unnestSIDXBranch.lastFieldVars,
context.getOutputTypeEnvironment(unnestSIDXBranch.currentTop),
index.getIndexDetails().isOverridingKeyFieldTypes());
if (filterExpression != null) {
@@ -477,7 +490,7 @@
UnnestBranchCreator unnestBeforeSIDXBranch = buildUnnestBranch(unnestBeforeSourceOp, index,
primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, recType,
metaType, dataset.hasMetaPart());
- unnestBeforeSIDXBranch.applyProjectDistinct();
+ unnestBeforeSIDXBranch.applyProjectOnly();
indexUpdate.getNestedPlans().add(unnestBeforeSIDXBranch.buildBranch());
}
} else if (index.getIndexType() == IndexType.ARRAY && isBulkload) {
@@ -498,7 +511,7 @@
indexUpdate.setSecondaryKeyExprs(secondaryExpressions);
// Update the filter expression to include these new keys.
- filterExpression = createFilterExpression(unnestSIDXBranch.lastFieldVars,
+ filterExpression = createAnyUnknownFilterExpression(unnestSIDXBranch.lastFieldVars,
context.getOutputTypeEnvironment(unnestSIDXBranch.currentTop),
index.getIndexDetails().isOverridingKeyFieldTypes());
indexUpdate.setFilterExpression(filterExpression);
@@ -513,8 +526,8 @@
}
} else {
// Get type, dimensions and number of keys
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index, secondaryKeyTypes.get(0),
+ secondaryKeyFields.get(0), recType);
IAType spatialType = keyPairType.first;
boolean isPointMBR =
spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
@@ -560,8 +573,11 @@
assignCoordinates.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
context.computeAndSetTypeEnvironmentForOperator(assignCoordinates);
replicateOutput = assignCoordinates;
- Mutable<ILogicalExpression> filterExpression = null;
+ boolean forceFilter = keyPairType.second;
+ Mutable<ILogicalExpression> filterExpression = createAnyUnknownFilterExpression(keyVarList,
+ context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
AssignOperator originalAssignCoordinates = null;
+ Mutable<ILogicalExpression> beforeOpFilterExpression = null;
// We do something similar for beforeOp key if the operation is an upsert
if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
List<LogicalVariable> originalKeyVarList = new ArrayList<>();
@@ -592,19 +608,15 @@
originalAssignCoordinates.setSourceLocation(sourceLoc);
originalAssignCoordinates.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
context.computeAndSetTypeEnvironmentForOperator(originalAssignCoordinates);
- } else {
- // We must enforce the filter if the originating spatial type is
- // nullable.
- boolean forceFilter = keyPairType.second;
- filterExpression = createFilterExpression(keyVarList,
- context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
+ beforeOpFilterExpression = createAnyUnknownFilterExpression(originalKeyVarList,
+ context.getOutputTypeEnvironment(originalAssignCoordinates), forceFilter);
}
DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
OperatorManipulationUtil
.cloneExpressions(primaryIndexModificationOp.getPrimaryKeyExpressions()),
- secondaryExpressions, filterExpression, primaryIndexModificationOp.getOperation(),
- primaryIndexModificationOp.isBulkload(),
+ secondaryExpressions, filterExpression, beforeOpFilterExpression,
+ primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
: primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
indexUpdate.setSourceLocation(sourceLoc);
@@ -629,8 +641,8 @@
}
if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
- indexUpdate.setUpsertIndicatorExpr(new MutableObject<>(
- new VariableReferenceExpression(primaryIndexModificationOp.getUpsertIndicatorVar())));
+ indexUpdate.setOperationExpr(new MutableObject<>(
+ new VariableReferenceExpression(primaryIndexModificationOp.getOperationVar())));
}
context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
@@ -670,31 +682,14 @@
boolean hasMetaPart) throws AlgebricksException {
Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
- // First, locate a field having the required UNNEST path. Queue this first, and all other keys will follow.
- Deque<Integer> keyPositionQueue = new ArrayDeque<>();
- for (int i = 0; i < arrayIndexDetails.getElementList().size(); i++) {
- Index.ArrayIndexElement e = arrayIndexDetails.getElementList().get(i);
- if (e.getUnnestList().isEmpty()) {
- keyPositionQueue.addLast(i);
- } else {
- keyPositionQueue.addFirst(i);
- }
- }
-
// Get the record variable associated with our record path.
- Index.ArrayIndexElement workingElement = arrayIndexDetails.getElementList().get(keyPositionQueue.getFirst());
- int sourceIndicatorForBaseRecord = workingElement.getSourceIndicator();
+ int sourceIndicatorForBaseRecord = arrayIndexDetails.getElementList().get(0).getSourceIndicator();
LogicalVariable sourceVarForBaseRecord = hasMetaPart
? ((sourceIndicatorForBaseRecord == Index.RECORD_INDICATOR) ? recordVar : metaVar) : recordVar;
UnnestBranchCreator branchCreator = new UnnestBranchCreator(sourceVarForBaseRecord, unnestSourceOp);
- int initialKeyPositionQueueSize = keyPositionQueue.size();
- Set<LogicalVariable> secondaryKeyVars = new HashSet<>();
- for (int i = 0; i < initialKeyPositionQueueSize; i++) {
-
- // Poll from our queue, and get a key position.
- int workingKeyPos = keyPositionQueue.pollFirst();
- workingElement = arrayIndexDetails.getElementList().get(workingKeyPos);
+ Set<LogicalVariable> secondaryKeyVars = new LinkedHashSet<>();
+ for (Index.ArrayIndexElement workingElement : arrayIndexDetails.getElementList()) {
int sourceIndicator = workingElement.getSourceIndicator();
ARecordType recordType =
hasMetaPart ? ((sourceIndicator == Index.RECORD_INDICATOR) ? recType : metaType) : recType;
@@ -706,7 +701,6 @@
isOpenOrNestedField =
(atomicFieldName.size() != 1) || !recordType.isClosedField(atomicFieldName.get(0));
- // The UNNEST path has already been created (we queued this first), so we look at the current top.
LogicalVariable newVar = context.newVar();
VariableReferenceExpression varRef = new VariableReferenceExpression(sourceVarForBaseRecord);
varRef.setSourceLocation(sourceLoc);
@@ -715,23 +709,23 @@
recordType.getFieldIndex(atomicFieldName.get(0)), atomicFieldName)
: getFieldAccessFunction(new MutableObject<>(varRef), -1, atomicFieldName);
+ // Add an assign on top to extract the atomic element.
AssignOperator newAssignOp = new AssignOperator(newVar, new MutableObject<>(newVarRef));
newAssignOp.setSourceLocation(sourceLoc);
branchCreator.currentTop = introduceNewOp(branchCreator.currentTop, newAssignOp, true);
secondaryKeyVars.add(newVar);
+ if (branchCreator.currentBottom == null) {
+ branchCreator.currentBottom = branchCreator.currentTop;
+ }
} else {
- // We have an array element. The "open / nestedness" is determined by the first UNNEST field.
- isOpenOrNestedField = workingElement.getUnnestList().get(0).size() > 1
- || !recordType.isClosedField(workingElement.getUnnestList().get(0).get(0));
-
- // Walk the array path.
+ // We have an array element. Walk the array path.
List<String> flatFirstFieldName = ArrayIndexUtil.getFlattenedKeyFieldNames(
workingElement.getUnnestList(), workingElement.getProjectList().get(0));
- List<Integer> firstArrayIndicator = ArrayIndexUtil
- .getArrayDepthIndicator(workingElement.getUnnestList(), workingElement.getProjectList().get(0));
- ArrayIndexUtil.walkArrayPath((isOpenOrNestedField) ? null : recordType, flatFirstFieldName,
- firstArrayIndicator, branchCreator);
+ List<Boolean> firstUnnestFlags = ArrayIndexUtil.getUnnestFlags(workingElement.getUnnestList(),
+ workingElement.getProjectList().get(0));
+ ArrayIndexUtil.walkArrayPath(index, recordType, flatFirstFieldName, firstUnnestFlags, branchCreator);
+ secondaryKeyVars.add(branchCreator.lastFieldVars.get(0));
// For all other elements in the PROJECT list, add an assign.
for (int j = 1; j < workingElement.getProjectList().size(); j++) {
@@ -746,15 +740,11 @@
secondaryKeyVars.add(newVar);
}
}
-
- branchCreator.lowerIsFirstWalkFlag();
- secondaryKeyVars.addAll(branchCreator.lastFieldVars);
}
// Update the variables we are to use for the head operators.
branchCreator.lastFieldVars.clear();
branchCreator.lastFieldVars.addAll(secondaryKeyVars);
-
return branchCreator;
}
@@ -832,22 +822,27 @@
String.valueOf(index.getIndexType()));
}
for (int i = 0; i < skNames.size(); i++) {
- IndexFieldId indexFieldId =
- new IndexFieldId(indicators.get(i), skNames.get(i), skTypes.get(i).getTypeTag());
+ List<String> skName = skNames.get(i);
+ Integer skSrc = indicators.get(i);
+ IAType skType = skTypes.get(i);
+
+ ARecordType sourceType = dataset.hasMetaPart()
+ ? skSrc.intValue() == Index.RECORD_INDICATOR ? recType : metaType : recType;
+ LogicalVariable sourceVar = dataset.hasMetaPart()
+ ? skSrc.intValue() == Index.RECORD_INDICATOR ? recordVar : metaVar : recordVar;
+
+ IAType fieldType = sourceType.getSubFieldType(skName);
+ IndexFieldId indexFieldId = createIndexFieldId(index, skName, skType, skSrc, sourceType, sourceLoc);
if (fieldAccessVars.containsKey(indexFieldId)) {
// already handled in a different index
continue;
}
- ARecordType sourceType = dataset.hasMetaPart()
- ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recType : metaType : recType;
- LogicalVariable sourceVar = dataset.hasMetaPart()
- ? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordVar : metaVar : recordVar;
- LogicalVariable fieldVar = context.newVar();
// create record variable ref
VariableReferenceExpression varRef = new VariableReferenceExpression(sourceVar);
varRef.setSourceLocation(sourceLoc);
- IAType fieldType = sourceType.getSubFieldType(indexFieldId.fieldName);
+
AbstractFunctionCallExpression theFieldAccessFunc;
+ LogicalVariable fieldVar = context.newVar();
if (fieldType == null) {
// Open field. must prevent inlining to maintain the cast before the primaryOp and
// make handling of records with incorrect value type for this field easier and cleaner
@@ -856,12 +851,8 @@
AbstractFunctionCallExpression fieldAccessFunc =
getFieldAccessFunction(new MutableObject<>(varRef), -1, indexFieldId.fieldName);
// create cast
- theFieldAccessFunc = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(
- index.isEnforced() ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX));
- theFieldAccessFunc.setSourceLocation(sourceLoc);
- // The first argument is the field
- theFieldAccessFunc.getArguments().add(new MutableObject<ILogicalExpression>(fieldAccessFunc));
- TypeCastUtils.setRequiredAndInputTypes(theFieldAccessFunc, skTypes.get(i), BuiltinType.ANY);
+ theFieldAccessFunc = createCastExpression(index, skType, fieldAccessFunc, sourceLoc,
+ indexFieldId.funId, indexFieldId.extraArg);
} else {
// Get the desired field position
int pos = indexFieldId.fieldName.size() > 1 ? -1
@@ -869,9 +860,17 @@
// Field not found --> This is either an open field or a nested field. it can't be accessed by index
theFieldAccessFunc =
getFieldAccessFunction(new MutableObject<>(varRef), pos, indexFieldId.fieldName);
+ // check IndexUtil.castDefaultNull(index), too, because we always want to cast even if
+ // the overriding type is the same as the overridden type (this is for the case where overriding
+ // the type of closed field is allowed)
+ // e.g. field "a" is a string in the dataset ds; CREATE INDEX .. ON ds(a:string) CAST (DEFAULT NULL)
+ if (IndexUtil.castDefaultNull(index)) {
+ theFieldAccessFunc = castConstructorFunction(indexFieldId.funId, indexFieldId.extraArg,
+ theFieldAccessFunc, sourceLoc);
+ }
}
vars.add(fieldVar);
- exprs.add(new MutableObject<ILogicalExpression>(theFieldAccessFunc));
+ exprs.add(new MutableObject<>(theFieldAccessFunc));
fieldAccessVars.put(indexFieldId, fieldVar);
}
}
@@ -884,6 +883,76 @@
return currentTop;
}
+ private static IndexFieldId createIndexFieldId(Index index, List<String> skName, IAType skType, Integer skSrc,
+ ARecordType sourceType, SourceLocation srcLoc) throws AlgebricksException {
+ IAType fieldType = sourceType.getSubFieldType(skName);
+ FunctionIdentifier skFun = null;
+ IAObject fmtArg = null;
+ Pair<FunctionIdentifier, IAObject> castExpr;
+ if (fieldType == null) {
+ // open field
+ castExpr = getCastExpression(index, skType, srcLoc);
+ skFun = castExpr.first;
+ fmtArg = castExpr.second;
+ } else {
+ // closed field
+ if (IndexUtil.castDefaultNull(index)) {
+ castExpr = IndexUtil.getTypeConstructorDefaultNull(index, skType, srcLoc);
+ skFun = castExpr.first;
+ fmtArg = castExpr.second;
+ }
+ }
+ return new IndexFieldId(skSrc, skName, skType.getTypeTag(), skFun, fmtArg);
+ }
+
+ private static Pair<FunctionIdentifier, IAObject> getCastExpression(Index index, IAType skType,
+ SourceLocation srcLoc) throws AlgebricksException {
+ if (IndexUtil.castDefaultNull(index)) {
+ return IndexUtil.getTypeConstructorDefaultNull(index, skType, srcLoc);
+ } else if (index.isEnforced()) {
+ return new Pair<>(BuiltinFunctions.CAST_TYPE, null);
+ } else {
+ return new Pair<>(BuiltinFunctions.CAST_TYPE_LAX, null);
+ }
+ }
+
+ private AbstractFunctionCallExpression createCastExpression(Index index, IAType targetType,
+ AbstractFunctionCallExpression inputExpr, SourceLocation sourceLoc, FunctionIdentifier castFun,
+ IAObject fmtArg) throws CompilationException {
+ ScalarFunctionCallExpression castExpr;
+ if (IndexUtil.castDefaultNull(index)) {
+ castExpr = castConstructorFunction(castFun, fmtArg, inputExpr, sourceLoc);
+ } else {
+ castExpr = castFunction(castFun, targetType, inputExpr, sourceLoc);
+ }
+ return castExpr;
+ }
+
+ private ScalarFunctionCallExpression castFunction(FunctionIdentifier castFun, IAType requiredType,
+ AbstractFunctionCallExpression inputExpr, SourceLocation sourceLoc) throws CompilationException {
+ BuiltinFunctionInfo castInfo = BuiltinFunctions.getBuiltinFunctionInfo(castFun);
+ ScalarFunctionCallExpression castExpr = new ScalarFunctionCallExpression(castInfo);
+ castExpr.setSourceLocation(sourceLoc);
+ castExpr.getArguments().add(new MutableObject<>(inputExpr));
+ TypeCastUtils.setRequiredAndInputTypes(castExpr, requiredType, BuiltinType.ANY);
+ return castExpr;
+ }
+
+ private ScalarFunctionCallExpression castConstructorFunction(FunctionIdentifier typeConstructorFun, IAObject fmt,
+ AbstractFunctionCallExpression inputExpr, SourceLocation srcLoc) {
+ BuiltinFunctionInfo typeConstructorInfo = BuiltinFunctions.getBuiltinFunctionInfo(typeConstructorFun);
+ ScalarFunctionCallExpression constructorExpr = new ScalarFunctionCallExpression(typeConstructorInfo);
+ constructorExpr.getArguments().add(new MutableObject<>(inputExpr));
+ // add the format argument if specified
+ if (fmt != null) {
+ ConstantExpression fmtExpr = new ConstantExpression(new AsterixConstantValue(fmt));
+ fmtExpr.setSourceLocation(srcLoc);
+ constructorExpr.getArguments().add(new MutableObject<>(fmtExpr));
+ }
+ constructorExpr.setSourceLocation(srcLoc);
+ return constructorExpr;
+ }
+
private ILogicalOperator introduceNewOp(ILogicalOperator currentTopOp, ILogicalOperator newOp, boolean afterOp)
throws AlgebricksException {
if (afterOp) {
@@ -942,11 +1011,42 @@
return new MutableObject<>(new ConstantExpression(new AsterixConstantValue(constantObject)));
}
- private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
+ private Mutable<ILogicalExpression> createFilterExpression(Index index, List<LogicalVariable> secondaryKeyVars,
IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
+ IndexType indexType = index.getIndexType();
+ if (indexType == IndexType.BTREE) {
+ if (index.isPrimaryKeyIndex()) {
+ return createAnyUnknownFilterExpression(secondaryKeyVars, typeEnv, forceFilter);
+ } else {
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ boolean excludeUnknown = indexDetails.getExcludeUnknownKey().getOrElse(false);
+ return createAllUnknownFilterExpression(secondaryKeyVars, typeEnv, forceFilter, excludeUnknown);
+ }
+ } else {
+ // inverted index && array index
+ return createAnyUnknownFilterExpression(secondaryKeyVars, typeEnv, forceFilter);
+ }
+ }
+
+ private Mutable<ILogicalExpression> createAnyUnknownFilterExpression(List<LogicalVariable> secondaryKeyVars,
+ IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
+ return createFilterExpression(secondaryKeyVars, typeEnv, forceFilter, true, BuiltinFunctions.AND);
+ }
+
+ private Mutable<ILogicalExpression> createAllUnknownFilterExpression(List<LogicalVariable> secondaryKeyVars,
+ IVariableTypeEnvironment typeEnv, boolean forceFilter, boolean excludeUnknownKey)
+ throws AlgebricksException {
+ return createFilterExpression(secondaryKeyVars, typeEnv, forceFilter, excludeUnknownKey, BuiltinFunctions.OR);
+ }
+
+ private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
+ IVariableTypeEnvironment typeEnv, boolean forceFilter, boolean excludeUnknownKey,
+ FunctionIdentifier combiner) throws AlgebricksException {
+ if (!excludeUnknownKey) {
+ return null;
+ }
List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<>();
- // Add 'is not null' to all nullable secondary index keys as a filtering
- // condition.
+ // Add 'is not null' to all nullable secondary index keys as a filtering condition
for (LogicalVariable secondaryKeyVar : secondaryKeyVars) {
IAType secondaryKeyType = (IAType) typeEnv.getVarType(secondaryKeyVar);
if (!NonTaggedFormatUtil.isOptional(secondaryKeyType) && !forceFilter) {
@@ -970,11 +1070,11 @@
}
Mutable<ILogicalExpression> filterExpression;
if (filterExpressions.size() > 1) {
- // Create a conjunctive condition.
- ScalarFunctionCallExpression andExpr = new ScalarFunctionCallExpression(
- FunctionUtil.getFunctionInfo(BuiltinFunctions.AND), filterExpressions);
- andExpr.setSourceLocation(sourceLoc);
- filterExpression = new MutableObject<>(andExpr);
+ // Combine the conditions
+ ScalarFunctionCallExpression combinerExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(combiner), filterExpressions);
+ combinerExpr.setSourceLocation(sourceLoc);
+ filterExpression = new MutableObject<>(combinerExpr);
} else {
filterExpression = filterExpressions.get(0);
}
@@ -987,8 +1087,7 @@
private class UnnestBranchCreator implements ArrayIndexUtil.TypeTrackerCommandExecutor {
private final List<LogicalVariable> lastFieldVars;
private LogicalVariable lastRecordVar;
- private ILogicalOperator currentTop, currentBottom;
- private boolean isFirstWalk = true;
+ private ILogicalOperator currentTop, currentBottom = null;
public UnnestBranchCreator(LogicalVariable recordVar, ILogicalOperator sourceOperator) {
this.lastRecordVar = recordVar;
@@ -1000,16 +1099,19 @@
return new ALogicalPlanImpl(new MutableObject<>(currentTop));
}
- public void lowerIsFirstWalkFlag() {
- isFirstWalk = false;
- }
-
public VariableReferenceExpression createLastRecordVarRef() {
VariableReferenceExpression varRef = new VariableReferenceExpression(lastRecordVar);
varRef.setSourceLocation(sourceLoc);
return varRef;
}
+ public final void applyProjectOnly() throws AlgebricksException {
+ List<LogicalVariable> projectVars = new ArrayList<>(this.lastFieldVars);
+ ProjectOperator projectOperator = new ProjectOperator(projectVars);
+ projectOperator.setSourceLocation(sourceLoc);
+ this.currentTop = introduceNewOp(currentTop, projectOperator, true);
+ }
+
@SafeVarargs
public final void applyProjectDistinct(List<Mutable<ILogicalExpression>>... auxiliaryExpressions)
throws AlgebricksException {
@@ -1047,32 +1149,21 @@
}
public void applyFilteringExpression(Mutable<ILogicalExpression> filterExpression) throws AlgebricksException {
- SelectOperator selectOperator = new SelectOperator(filterExpression, false, null);
+ SelectOperator selectOperator = new SelectOperator(filterExpression);
selectOperator.setSourceLocation(sourceLoc);
this.currentTop = introduceNewOp(currentTop, selectOperator, true);
}
@Override
public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
- List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
- boolean isLastUnnestInIntermediateStep) throws AlgebricksException {
- if (!isFirstWalk) {
- // We have already built the UNNEST path, do not build again.
- return;
- }
-
+ List<String> fieldName, boolean isFirstArrayStep, boolean isLastUnnestInIntermediateStep)
+ throws AlgebricksException {
+ // Get the field we want to UNNEST from our record.
ILogicalExpression accessToUnnestVar;
- if (isFirstUnnestInStep) {
- // This is the first UNNEST step. Get the field we want to UNNEST from our record.
- accessToUnnestVar = (startingStepRecordType != null)
- ? getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()),
- startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
- : getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()), -1, fieldName);
- } else {
- // This is the second+ UNNEST step. Refer back to the previously unnested variable.
- accessToUnnestVar = new VariableReferenceExpression(this.lastFieldVars.get(0));
- this.lastFieldVars.clear();
- }
+ accessToUnnestVar = (startingStepRecordType != null)
+ ? getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()),
+ startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
+ : getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()), -1, fieldName);
UnnestingFunctionCallExpression scanCollection = new UnnestingFunctionCallExpression(
BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
Collections.singletonList(new MutableObject<>(accessToUnnestVar)));
@@ -1084,7 +1175,7 @@
UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollection));
unnestOp.setSourceLocation(sourceLoc);
this.currentTop = introduceNewOp(currentTop, unnestOp, true);
- if (isFirstArrayStep) {
+ if (isFirstArrayStep && this.currentBottom == null) {
this.currentBottom = unnestOp;
}
@@ -1117,15 +1208,20 @@
}
}
- private final class IndexFieldId {
+ private static class IndexFieldId {
private final int indicator;
private final List<String> fieldName;
private final ATypeTag fieldType;
+ private final FunctionIdentifier funId;
+ private final IAObject extraArg; // currently, only for datetime constructor functions with the format arg
- private IndexFieldId(int indicator, List<String> fieldName, ATypeTag fieldType) {
+ private IndexFieldId(int indicator, List<String> fieldName, ATypeTag fieldType, FunctionIdentifier funId,
+ IAObject extraArg) {
this.indicator = indicator;
this.fieldName = fieldName;
this.fieldType = fieldType;
+ this.funId = funId;
+ this.extraArg = extraArg;
}
@Override
@@ -1133,6 +1229,8 @@
int result = indicator;
result = 31 * result + fieldName.hashCode();
result = 31 * result + fieldType.hashCode();
+ result = 31 * result + Objects.hashCode(funId);
+ result = 31 * result + Objects.hashCode(extraArg);
return result;
}
@@ -1145,13 +1243,9 @@
return false;
}
IndexFieldId that = (IndexFieldId) o;
- if (indicator != that.indicator) {
- return false;
- }
- if (!fieldName.equals(that.fieldName)) {
- return false;
- }
- return fieldType == that.fieldType;
+ return indicator == that.indicator && Objects.equals(fieldName, that.fieldName)
+ && fieldType == that.fieldType && Objects.equals(funId, that.funId)
+ && Objects.equals(extraArg, that.extraArg);
}
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
new file mode 100644
index 0000000..507906b
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
@@ -0,0 +1,162 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * If the join predicate contains multiple conditions, in which one of them is spatial_intersect($x, $y), all other
+ * condition will be pull out to a SELECT operator after the SPATIAL_JOIN operator.
+ *
+ * For example:<br/>
+ * join (and(spatial-intersect($$52, $$53), lt(st-distance($$56, $$57), 1.0))
+ * -- SPATIAL_JOIN [$$62, $$52] [$$63, $$53] |PARTITIONED|
+ *
+ * Becomes,
+ *
+ * select (lt(st-distance($$56, $$57), 1.0))
+ * -- STREAM_SELECT |PARTITIONED|
+ * exchange
+ * -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ * join (spatial-intersect($$52, $$53))
+ * -- SPATIAL_JOIN [$$62, $$52] [$$63, $$53] |PARTITIONED|
+ */
+
+public class PullSelectOutOfSpatialJoin 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.INNERJOIN) {
+ return false;
+ }
+ AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
+
+ if (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.SPATIAL_JOIN) {
+ return false;
+ }
+
+ ILogicalExpression expr = join.getCondition().getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier fi = fexp.getFunctionIdentifier();
+ if (!fi.equals(AlgebricksBuiltinFunctions.AND)) {
+ return false;
+ }
+ List<Mutable<ILogicalExpression>> spatialVarVarComps = new ArrayList<Mutable<ILogicalExpression>>();
+ List<Mutable<ILogicalExpression>> otherPredicates = new ArrayList<Mutable<ILogicalExpression>>();
+ for (Mutable<ILogicalExpression> arg : fexp.getArguments()) {
+ if (isSpatialVarVar(arg.getValue(), join, context)) {
+ spatialVarVarComps.add(arg);
+ } else {
+ otherPredicates.add(arg);
+ }
+ }
+ if (spatialVarVarComps.isEmpty() || otherPredicates.isEmpty()) {
+ return false;
+ }
+ // pull up
+ ILogicalExpression pulledCond = makeCondition(otherPredicates, context, op);
+ SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
+ select.setSourceLocation(op.getSourceLocation());
+ ILogicalExpression newJoinCond = makeCondition(spatialVarVarComps, context, op);
+ join.getCondition().setValue(newJoinCond);
+ select.getInputs().add(new MutableObject<ILogicalOperator>(join));
+ context.computeAndSetTypeEnvironmentForOperator(select);
+ select.recomputeSchema();
+ opRef.setValue(select);
+
+ return true;
+ }
+
+ private ILogicalExpression makeCondition(List<Mutable<ILogicalExpression>> predList, IOptimizationContext context,
+ AbstractLogicalOperator op) {
+ if (predList.size() > 1) {
+ IFunctionInfo finfo = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
+ ScalarFunctionCallExpression conditionExpr = new ScalarFunctionCallExpression(finfo, predList);
+ conditionExpr.setSourceLocation(op.getSourceLocation());
+ return conditionExpr;
+ } else {
+ return predList.get(0).getValue();
+ }
+ }
+
+ private boolean isSpatialVarVar(ILogicalExpression expr, AbstractBinaryJoinOperator join,
+ IOptimizationContext context) throws AlgebricksException {
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+ if (!f.getFunctionIdentifier().equals(BuiltinFunctions.SPATIAL_INTERSECT)) {
+ return false;
+ }
+
+ // We only apply this rule if the arguments of spatial_intersect are ARectangle
+ IVariableTypeEnvironment typeEnvironment = join.computeInputTypeEnvironment(context);
+ IAType leftType = (IAType) context.getExpressionTypeComputer().getType(f.getArguments().get(0).getValue(),
+ context.getMetadataProvider(), typeEnvironment);
+ IAType rightType = (IAType) context.getExpressionTypeComputer().getType(f.getArguments().get(1).getValue(),
+ context.getMetadataProvider(), typeEnvironment);
+ if ((leftType.getTypeTag() != BuiltinType.ARECTANGLE.getTypeTag())
+ || (rightType.getTypeTag() != BuiltinType.ARECTANGLE.getTypeTag())) {
+ return false;
+ }
+
+ ILogicalExpression e1 = f.getArguments().get(0).getValue();
+ if (e1.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return false;
+ } else {
+ ILogicalExpression e2 = f.getArguments().get(1).getValue();
+ return e2.getExpressionTag() == LogicalExpressionTag.VARIABLE;
+ }
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessToExternalDataScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessToExternalDataScanRule.java
deleted file mode 100644
index f25e058..0000000
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessToExternalDataScanRule.java
+++ /dev/null
@@ -1,248 +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.
- */
-package org.apache.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.metadata.DataverseName;
-import org.apache.asterix.external.util.ExternalDataUtils;
-import org.apache.asterix.metadata.declared.DataSource;
-import org.apache.asterix.metadata.declared.DatasetDataSource;
-import org.apache.asterix.metadata.declared.ExternalDataProjectionInfo;
-import org.apache.asterix.metadata.declared.MetadataProvider;
-import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
-import org.apache.asterix.om.functions.BuiltinFunctions;
-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.base.LogicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
-import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-
-/**
- * Pushes field-access expression to the external dataset scan to minimize the size of the record.
- * This rule currently does not remove the field access expression in ASSIGN and SCAN operators. Instead,
- * it adds the requested field names to external dataset details to produce records that only contain the requested
- * fields. Thus, no changes would occur in the plan's structure after firing this rule.
- * Example:
- * Before plan:
- * ...
- * select (and(gt($$00, 20), gt($$r.getField("salary"), 70000)))
- * ...
- * assign [$$00] <- [$$r.getField("personalInfo").getField("age")]
- * ...
- * data-scan []<-[$$r] <- ParquetDataverse.ParquetDataset
- * <p>
- * After plan:
- * ...
- * select (and(gt($$00, 20), gt($$r.getField("salary"), 70000)))
- * ...
- * assign [$$00] <- [$$r.getField("personalInfo").getField("age")]
- * ...
- * data-scan []<-[$$r] <- ParquetDataverse.ParquetDataset project (personalInfo.age, salary)
- * <p>
- * The resulting record $$r will be {"personalInfo":{"age": *AGE*}, "salary": *SALARY*}
- * and other fields will not be included in $$r.
- */
-public class PushFieldAccessToExternalDataScanRule implements IAlgebraicRewriteRule {
- //Datasets payload variables
- private final List<LogicalVariable> recordVariables = new ArrayList<>();
- //Dataset scan operators' projection info
- private final List<ExternalDataProjectionInfo> projectionInfos = new ArrayList<>();
- //Final result live variables
- private final Set<LogicalVariable> projectedVariables = new HashSet<>();
-
- @Override
- public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
- final ILogicalOperator currentOp = opRef.getValue();
- final LogicalOperatorTag currentOpTag = currentOp.getOperatorTag();
- if (!context.getPhysicalOptimizationConfig().isExternalFieldPushdown()) {
- return false;
- }
- if (currentOpTag == LogicalOperatorTag.PROJECT) {
- ProjectOperator projectOp = (ProjectOperator) currentOp;
- projectedVariables.addAll(projectOp.getVariables());
- return false;
- }
-
- if (currentOpTag != LogicalOperatorTag.DATASOURCESCAN) {
- return false;
- }
-
- return setDatasetProperties(currentOp, (MetadataProvider) context.getMetadataProvider());
- }
-
- @Override
- public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
- final ILogicalOperator op = opRef.getValue();
- if (!context.getPhysicalOptimizationConfig().isExternalFieldPushdown()
- || context.checkIfInDontApplySet(this, op) || projectionInfos.isEmpty()) {
- return false;
- }
-
- if (op.getOperatorTag() != LogicalOperatorTag.SELECT && op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
- return false;
- }
-
- if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
- final SelectOperator selectOp = (SelectOperator) op;
- pushFieldAccessExpression(selectOp.getCondition(), context);
- } else {
- final AssignOperator assignOp = (AssignOperator) op;
- pushFieldAccessExpression(assignOp.getExpressions(), context);
- }
-
- //Add to do not apply to avoid pushing the same expression twice when the plan contains REPLICATE
- context.addToDontApplySet(this, op);
-
- return false;
- }
-
- private void pushFieldAccessExpression(List<Mutable<ILogicalExpression>> exprList, IOptimizationContext context)
- throws AlgebricksException {
-
- for (Mutable<ILogicalExpression> exprRef : exprList) {
- pushFieldAccessExpression(exprRef, context);
- }
- }
-
- private void pushFieldAccessExpression(Mutable<ILogicalExpression> exprRef, IOptimizationContext context)
- throws AlgebricksException {
- final ILogicalExpression expr = exprRef.getValue();
- if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return;
- }
-
- final AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-
- //Only field access expressions are allowed
- if (!isFieldAccessByName(funcExpr)) {
- pushFieldAccessExpression(funcExpr.getArguments(), context);
- return;
- }
-
- //Get root expression input variable in case it is nested field access
- final LogicalVariable funcRootInputVar = getRootExpressionInputVariable(funcExpr);
- if (funcRootInputVar != null) {
- final int recordVarIndex = recordVariables.indexOf(funcRootInputVar);
- //Is funcRootInputVar originated from a data-scan operator?
- if (recordVarIndex >= 0) {
- final List<List<String>> projectedFieldNames = projectionInfos.get(recordVarIndex).getProjectionInfo();
- final List<String> fieldNames = new ArrayList<>();
- //Add fieldAccessExpr to field names list
- buildFieldNames(funcExpr, fieldNames);
- if (!fieldNames.isEmpty()) {
- projectedFieldNames.add(fieldNames);
- }
- }
- } else {
- //Descend to the arguments expressions to see if any can be pushed
- pushFieldAccessExpression(funcExpr.getArguments(), context);
- }
- }
-
- private boolean setDatasetProperties(ILogicalOperator op, MetadataProvider mp) throws AlgebricksException {
- final DataSourceScanOperator scan = (DataSourceScanOperator) op;
- final DataSource dataSource = (DataSource) scan.getDataSource();
-
- if (dataSource == null) {
- return false;
- }
- final DataverseName dataverse = dataSource.getId().getDataverseName();
- final String datasetName = dataSource.getId().getDatasourceName();
- final Dataset dataset = mp.findDataset(dataverse, datasetName);
-
- //Only external dataset can have pushed down expressions
- if (dataset == null || dataset.getDatasetType() == DatasetType.INTERNAL
- || dataset.getDatasetType() == DatasetType.EXTERNAL && !ExternalDataUtils
- .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties())) {
- return false;
- }
-
- boolean changed = false;
- final DatasetDataSource datasetDataSource = (DatasetDataSource) dataSource;
- final LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(scan.getVariables());
- if (!projectedVariables.contains(recordVar) && scan.getProjectionInfo() == null) {
- //Do not push expressions to data scan if the whole record is needed
- recordVariables.add(recordVar);
- ExternalDataProjectionInfo projectionInfo = new ExternalDataProjectionInfo();
- scan.setProjectionInfo(projectionInfo);
- projectionInfos.add(projectionInfo);
- changed = true;
- }
- return changed;
- }
-
- private static LogicalVariable getRootExpressionInputVariable(AbstractFunctionCallExpression funcExpr) {
- ILogicalExpression currentExpr = funcExpr.getArguments().get(0).getValue();
- while (isFieldAccessByName(currentExpr)) {
- currentExpr = ((AbstractFunctionCallExpression) currentExpr).getArguments().get(0).getValue();
- }
-
- if (currentExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
- return ((VariableReferenceExpression) currentExpr).getVariableReference();
- }
- return null;
- }
-
- private static boolean isFieldAccessByName(ILogicalExpression expression) {
- return expression.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
- && BuiltinFunctions.FIELD_ACCESS_BY_NAME
- .equals(((AbstractFunctionCallExpression) expression).getFunctionIdentifier());
- }
-
- private static void buildFieldNames(ILogicalExpression expr, List<String> fieldNames) throws CompilationException {
- if (!isFieldAccessByName(expr)) {
- /*
- * We only push nested field-access expressions.
- * This is a sanity check if the previous checks have missed.
- */
- return;
- }
-
- AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
- ILogicalExpression objectExpr = funcExpr.getArguments().get(0).getValue();
- if (!isPayload(objectExpr)) {
- buildFieldNames(objectExpr, fieldNames);
- }
- fieldNames.add(ConstantExpressionUtil.getStringArgument(funcExpr, 1));
- }
-
- private static boolean isPayload(ILogicalExpression expr) {
- return expr.getExpressionTag() == LogicalExpressionTag.VARIABLE;
- }
-}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
index 48c4324..6d92b51 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
@@ -84,24 +84,27 @@
Mutable<ILogicalOperator> opLeftRef = join.getInputs().get(0);
ILogicalOperator opLeft = opLeftRef.getValue();
+ Mutable<ILogicalOperator> opRightRef = join.getInputs().get(1);
+ ILogicalOperator opRight = opRightRef.getValue();
switch (canPushThrough(gby, opLeft, decorToPush, decorNotToPush)) {
case REPEATED_DECORS: {
return false;
}
case TRUE: {
- push(opRef, opRef2, 0, decorToPush, decorNotToPush, context);
- return true;
+ if (OperatorPropertiesUtil.isCardinalityZeroOrOne(opRight)) {
+ push(opRef, opRef2, 0, decorToPush, decorNotToPush, context);
+ return true;
+ }
+ return false;
}
case FALSE: {
decorToPush.clear();
- Mutable<ILogicalOperator> opRightRef = join.getInputs().get(1);
- ILogicalOperator opRight = opRightRef.getValue();
- if (canPushThrough(gby, opRight, decorToPush, decorNotToPush) == PushTestResult.TRUE) {
+ if (canPushThrough(gby, opRight, decorToPush, decorNotToPush) == PushTestResult.TRUE
+ && OperatorPropertiesUtil.isCardinalityZeroOrOne(opLeft)) {
push(opRef, opRef2, 1, decorToPush, decorNotToPush, context);
return true;
- } else {
- return false;
}
+ return false;
}
default: {
throw new IllegalStateException();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToExternalDataScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToExternalDataScanRule.java
new file mode 100644
index 0000000..405e2bd
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushValueAccessToExternalDataScanRule.java
@@ -0,0 +1,123 @@
+/*
+ * 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.Set;
+
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
+import org.apache.asterix.optimizer.base.AsterixOptimizationContext;
+import org.apache.asterix.optimizer.rules.pushdown.OperatorValueAccessPushdownVisitor;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.objects.ObjectSet;
+
+/**
+ * Pushes value access expressions to the external dataset scan to minimize the size of the record.
+ * This rule currently does not remove the value access expression. Instead, it adds the requested field names to
+ * external dataset details to produce records that only contain the requested values. Thus, no changes would occur
+ * to the plan's structure after firing this rule.
+ * Example:
+ * Before plan:
+ * ...
+ * select (and(gt($$00, 20), gt($$r.getField("salary"), 70000)))
+ * ...
+ * assign [$$00] <- [$$r.getField("personalInfo").getField("age")]
+ * ...
+ * data-scan []<-[$$r] <- ParquetDataverse.ParquetDataset
+ * <p>
+ * After plan:
+ * ...
+ * select (and(gt($$00, 20), gt($$r.getField("salary"), 70000)))
+ * ...
+ * assign [$$00] <- [$$r.getField("personalInfo").getField("age")]
+ * ...
+ * data-scan []<-[$$r] <- ParquetDataverse.ParquetDataset project ({personalInfo:{age: VALUE},salary:VALUE})
+ * <p>
+ * The resulting record $$r will be {"personalInfo":{"age": *AGE*}, "salary": *SALARY*}
+ * and other fields will not be included in $$r.
+ */
+public class PushValueAccessToExternalDataScanRule implements IAlgebraicRewriteRule {
+ //Initially, assume we need to run the rule
+ private boolean run = true;
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ if (!context.getPhysicalOptimizationConfig().isExternalFieldPushdown() || !run) {
+ //The rule was fired, or value access pushdown is disabled
+ return false;
+ }
+
+ /*
+ * Only run the rewrite rule once and only if the plan contains a data-scan on an external dataset that
+ * support value access pushdown.
+ */
+ run = shouldRun(context);
+ if (run) {
+ run = false;
+ OperatorValueAccessPushdownVisitor visitor = new OperatorValueAccessPushdownVisitor(context);
+ opRef.getValue().accept(visitor, null);
+ visitor.finish();
+ }
+
+ //This rule does not do any actual structural changes to the plan
+ return false;
+ }
+
+ /**
+ * Check whether the plan contains an external dataset that supports pushdown
+ *
+ * @param context optimization context
+ * @return true if the plan contains such dataset, false otherwise
+ */
+ private boolean shouldRun(IOptimizationContext context) throws AlgebricksException {
+ ObjectSet<Int2ObjectMap.Entry<Set<DataSource>>> entrySet =
+ ((AsterixOptimizationContext) context).getDataSourceMap().int2ObjectEntrySet();
+ MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+ for (Int2ObjectMap.Entry<Set<DataSource>> dataSources : entrySet) {
+ for (DataSource dataSource : dataSources.getValue()) {
+ if (supportPushdown(metadataProvider, dataSource)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ private boolean supportPushdown(MetadataProvider metadataProvider, DataSource dataSource)
+ throws AlgebricksException {
+ DataverseName dataverse = dataSource.getId().getDataverseName();
+ String datasetName = dataSource.getId().getDatasourceName();
+ Dataset dataset = metadataProvider.findDataset(dataverse, datasetName);
+
+ return dataset != null && dataset.getDatasetType() == DatasetConfig.DatasetType.EXTERNAL && ExternalDataUtils
+ .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties());
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
index 775a1df..f9f9a4c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
@@ -22,6 +22,7 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -38,7 +39,9 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
@@ -46,6 +49,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
/**
@@ -55,7 +59,7 @@
* left-outer-unnest $x <- scan-collection($y)
* group-by($k){
* aggregate $y <- listify($z1)
- * select not(is-missing($z2))
+ * select not(is-missing/null($z2))
* NTS
* }
* left outer join ($a=$b)
@@ -89,7 +93,7 @@
// Checks whether the left outer unnest and the group-by operator are qualified for rewriting.
Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult =
- checkUnnestAndGby(outerUnnest, gbyOperator);
+ checkUnnestAndGby(outerUnnest, gbyOperator, lojOperator);
// The argument for listify and not(is-missing(...)) check should be variables.
if (!checkGbyResult.first || checkGbyResult.second == null || !isVariableReference(checkGbyResult.second)
|| checkGbyResult.third == null || !isVariableReference(checkGbyResult.third)) {
@@ -127,7 +131,7 @@
// Checks left outer unnest and gby.
private Triple<Boolean, ILogicalExpression, ILogicalExpression> checkUnnestAndGby(
- LeftOuterUnnestOperator outerUnnest, GroupByOperator gbyOperator) {
+ LeftOuterUnnestOperator outerUnnest, GroupByOperator gbyOperator, LeftOuterJoinOperator lojOperator) {
// Checks left outer unnest.
Pair<Boolean, LogicalVariable> checkUnnestResult = checkUnnest(outerUnnest);
if (!checkUnnestResult.first) {
@@ -136,7 +140,8 @@
// Checks group-by.
LogicalVariable varToUnnest = checkUnnestResult.second;
- Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult = checkGroupBy(gbyOperator, varToUnnest);
+ Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult =
+ checkGroupBy(gbyOperator, varToUnnest, lojOperator.getMissingValue());
if (!checkGbyResult.first) {
return new Triple<>(false, null, null);
}
@@ -159,7 +164,7 @@
// Checks the group-by operator on top of the left outer join operator.
private Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGroupBy(GroupByOperator gbyOperator,
- LogicalVariable varToUnnest) {
+ LogicalVariable varToUnnest, IAlgebricksConstantValue leftOuterMissingValue) {
Pair<Boolean, ILogicalOperator> checkNestedPlanResult = checkNestedPlan(gbyOperator);
if (!checkNestedPlanResult.first) {
return new Triple<>(false, null, null);
@@ -182,7 +187,7 @@
return new Triple<>(false, null, null);
}
SelectOperator select = (SelectOperator) rootInputOp;
- Pair<Boolean, ILogicalExpression> conditionArgPair = checkSelect(select);
+ Pair<Boolean, ILogicalExpression> conditionArgPair = checkSelect(select, leftOuterMissingValue);
return new Triple<>(true, listifyArgPair.second, conditionArgPair.second);
}
@@ -222,7 +227,8 @@
}
// Checks the expression for the nested select operator inside the group-by operator.
- private Pair<Boolean, ILogicalExpression> checkSelect(SelectOperator select) {
+ private Pair<Boolean, ILogicalExpression> checkSelect(SelectOperator select,
+ IAlgebricksConstantValue leftOuterMissingValue) {
ILogicalExpression condition = select.getCondition().getValue();
if (condition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return new Pair<>(false, null);
@@ -236,7 +242,9 @@
return new Pair<>(false, null);
}
conditionFunc = (AbstractFunctionCallExpression) condition;
- if (!conditionFunc.getFunctionIdentifier().equals(BuiltinFunctions.IS_MISSING)) {
+ FunctionIdentifier isMissingNullFuncId =
+ Objects.requireNonNull(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue));
+ if (!conditionFunc.getFunctionIdentifier().equals(isMissingNullFuncId)) {
return new Pair<>(false, null);
}
ILogicalExpression conditionArg = conditionFunc.getArguments().get(0).getValue();
@@ -265,7 +273,7 @@
lhs.add(outerUnnest.getVariable());
VariableReferenceExpression listifyVarRef = new VariableReferenceExpression(listifyVar);
listifyVarRef.setSourceLocation(gbyOperator.getSourceLocation());
- rhs.add(new MutableObject<ILogicalExpression>(listifyVarRef));
+ rhs.add(new MutableObject<>(listifyVarRef));
List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gbyOperator.getGroupByList();
for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : gbyList) {
lhs.add(gbyPair.first);
@@ -273,7 +281,9 @@
}
AssignOperator assignOp = new AssignOperator(lhs, rhs);
assignOp.setSourceLocation(outerUnnest.getSourceLocation());
- assignOp.getInputs().add(new MutableObject<ILogicalOperator>(lojOperator));
+ assignOp.getInputs().add(new MutableObject<>(lojOperator));
+ lojOperator.setMissingValue(outerUnnest.getMissingValue());
+ context.computeAndSetTypeEnvironmentForOperator(lojOperator);
context.computeAndSetTypeEnvironmentForOperator(assignOp);
opRef.setValue(assignOp);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 76b57ef..7f1ff4a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -61,7 +61,6 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -81,7 +80,6 @@
* methods.
*/
public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
-
protected MetadataProvider metadataProvider;
public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
@@ -110,9 +108,9 @@
}
protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
- Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context)
- throws AlgebricksException {
- fillSubTreeIndexExprs(subTree, analyzedAMs, context, false);
+ Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context,
+ boolean isJoinLeftBranch) throws AlgebricksException {
+ fillSubTreeIndexExprs(subTree, analyzedAMs, context, isJoinLeftBranch, false);
}
/**
@@ -121,6 +119,7 @@
* @param subTree
* @param analyzedAMs
* @param context
+ * @param isJoinLeftBranch
* @param isArbitraryFormOfSubtree
* if the given subtree is in an arbitrary form that OptimizableSubTree class can't initialize, we try
* to fill the field type of each variable that is used in the optimizable function expressions.
@@ -130,7 +129,7 @@
*/
protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs, IOptimizationContext context,
- boolean isArbitraryFormOfSubtree) throws AlgebricksException {
+ boolean isJoinLeftBranch, boolean isArbitraryFormOfSubtree) throws AlgebricksException {
Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
// Check applicability of indexes by access method type.
while (amIt.hasNext()) {
@@ -138,10 +137,13 @@
AccessMethodAnalysisContext amCtx = entry.getValue();
// For the current access method type, map variables to applicable
// indexes.
- if (!isArbitraryFormOfSubtree) {
- fillAllIndexExprs(subTree, amCtx, context);
- } else {
+ if (isArbitraryFormOfSubtree) {
fillVarFieldTypeForOptFuncExprs(subTree, amCtx, context);
+ } else {
+ if (isJoinLeftBranch) {
+ fillVarFieldTypeForOptFuncExprs(subTree, amCtx, context);
+ }
+ fillAllIndexExprs(subTree, amCtx, context, entry.getKey());
}
}
}
@@ -395,7 +397,10 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var,
+ List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists)
throws AlgebricksException {
if (var.equals(optFuncExpr.getSourceVar(exprAndVarIdx.second))) {
@@ -484,6 +489,25 @@
if (lastFieldMatched < 0) {
indexExprAndVarIt.remove();
continue;
+
+ } else if (Index.IndexCategory.of(indexType).equals(Index.IndexCategory.ARRAY)) {
+ // For array indexes, we cannot make the decision to apply the prefix until we see a conjunct
+ // conditioning on an array. We should improve using array indexes for queries that don't involve
+ // the array component in the future.
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ int indexOfFirstArrayField = 0;
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ if (!e.getUnnestList().isEmpty()) {
+ break;
+ }
+ for (List<String> ignored : e.getProjectList()) {
+ indexOfFirstArrayField++;
+ }
+ }
+ if (lastFieldMatched < indexOfFirstArrayField) {
+ indexExprAndVarIt.remove();
+ continue;
+ }
}
}
analysisCtx.putNumberOfMatchedKeys(index, numMatchedKeys);
@@ -643,15 +667,19 @@
*/
protected boolean fillIndexExprs(List<Index> datasetIndexes, List<String> fieldName, IAType fieldType,
IOptimizableFuncExpr optFuncExpr, int matchedFuncExprIndex, int varIdx,
- OptimizableOperatorSubTree matchedSubTree, AccessMethodAnalysisContext analysisCtx, int fieldSource)
- throws AlgebricksException {
+ OptimizableOperatorSubTree matchedSubTree, AccessMethodAnalysisContext analysisCtx, int fieldSource,
+ IAccessMethod accessMethod) throws AlgebricksException {
List<Index> indexCandidates = new ArrayList<>();
// Add an index to the candidates if one of the indexed fields is fieldName
for (Index index : datasetIndexes) {
+ if (!accessMethod.matchIndexType(index.getIndexType())) {
+ continue;
+ }
List<List<String>> keyFieldNames;
List<IAType> keyFieldTypes;
List<Integer> keySources;
boolean isOverridingKeyFieldTypes;
+ boolean hasCastDefaultNull = false;
switch (Index.IndexCategory.of(index.getIndexType())) {
case ARRAY:
Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
@@ -674,6 +702,7 @@
keyFieldTypes = valueIndexDetails.getKeyFieldTypes();
keySources = valueIndexDetails.getKeyFieldSourceIndicators();
isOverridingKeyFieldTypes = valueIndexDetails.isOverridingKeyFieldTypes();
+ hasCastDefaultNull = valueIndexDetails.getCastDefaultNull().getOrElse(false);
break;
case TEXT:
Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
@@ -690,13 +719,16 @@
int keyIdx = keyFieldNames.indexOf(fieldName);
if (keyIdx >= 0 && keySourceMatches(keySources, keyIdx, fieldSource)
&& index.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
- indexCandidates.add(index);
- boolean isFieldTypeUnknown = fieldType == BuiltinType.AMISSING || fieldType == BuiltinType.ANY;
- if (isFieldTypeUnknown && (!isOverridingKeyFieldTypes || index.isEnforced())) {
- IAType indexedType = keyFieldTypes.get(keyIdx);
- optFuncExpr.setFieldType(varIdx, indexedType);
+ IAType indexedType = keyFieldTypes.get(keyIdx);
+ List<AbstractFunctionCallExpression> stepExprs = optFuncExpr.getStepsExprs(varIdx);
+ if (acceptSteps(index, accessMethod, stepExprs, indexedType, hasCastDefaultNull)) {
+ indexCandidates.add(index);
+ boolean isFieldTypeUnknown = fieldType == BuiltinType.AMISSING || fieldType == BuiltinType.ANY;
+ if (isFieldTypeUnknown && (!isOverridingKeyFieldTypes || index.isEnforced())) {
+ optFuncExpr.setFieldType(varIdx, indexedType);
+ }
+ analysisCtx.addIndexExpr(matchedSubTree.getDataset(), index, matchedFuncExprIndex, varIdx);
}
- analysisCtx.addIndexExpr(matchedSubTree.getDataset(), index, matchedFuncExprIndex, varIdx);
}
}
// No index candidates for fieldName.
@@ -706,13 +738,23 @@
return true;
}
+ private boolean acceptSteps(Index index, IAccessMethod accessMethod, List<AbstractFunctionCallExpression> stepExprs,
+ IAType indexedType, boolean indexHasCastDefaultNull) throws AlgebricksException {
+ for (int i = stepExprs.size() - 1; i >= 0; i--) {
+ if (!accessMethod.acceptsFunction(stepExprs.get(i), index, indexedType, indexHasCastDefaultNull, i == 0)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
private static boolean keySourceMatches(List<Integer> keySources, int keyIdx, int fieldSource) {
// TODO(ali): keySources from Index should not be null. should investigate if it can happen (ie on external ds)
return keySources == null ? fieldSource == 0 : keySources.get(keyIdx) == fieldSource;
}
protected void fillAllIndexExprs(OptimizableOperatorSubTree subTree, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context) throws AlgebricksException {
+ IOptimizationContext context, IAccessMethod accessMethod) throws AlgebricksException {
int optFuncExprIndex = 0;
List<Index> datasetIndexes = new ArrayList<>();
LogicalVariable datasetMetaVar = null;
@@ -732,10 +774,10 @@
AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
analyzeAssignOp((AssignOperator) op, optFuncExpr, subTree, assignOrUnnestIndex, datasetMetaVar,
- context, datasetIndexes, optFuncExprIndex, analysisCtx);
+ context, datasetIndexes, optFuncExprIndex, analysisCtx, accessMethod);
} else {
analyzeUnnestOp((UnnestOperator) op, optFuncExpr, subTree, assignOrUnnestIndex, datasetMetaVar,
- context, datasetIndexes, optFuncExprIndex, analysisCtx);
+ context, datasetIndexes, optFuncExprIndex, analysisCtx, accessMethod);
}
}
@@ -744,7 +786,7 @@
List<LogicalVariable> dsVarList = subTree.getDataSourceVariables();
matchVarsFromOptFuncExprToDataSourceScan(optFuncExpr, optFuncExprIndex, datasetIndexes, dsVarList, subTree,
- analysisCtx, context, false);
+ analysisCtx, context, false, accessMethod);
// If there is one more datasource in the subtree, we need to scan that datasource, too.
List<LogicalVariable> additionalDsVarList = null;
@@ -756,7 +798,7 @@
}
matchVarsFromOptFuncExprToDataSourceScan(optFuncExpr, optFuncExprIndex, datasetIndexes,
- additionalDsVarList, subTree, analysisCtx, context, true);
+ additionalDsVarList, subTree, analysisCtx, context, true, accessMethod);
}
@@ -767,7 +809,7 @@
private void analyzeUnnestOp(UnnestOperator unnestOp, IOptimizableFuncExpr optFuncExpr,
OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetMetaVar,
IOptimizationContext context, List<Index> datasetIndexes, int optFuncExprIndex,
- AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+ AccessMethodAnalysisContext analysisCtx, IAccessMethod accessMethod) throws AlgebricksException {
LogicalVariable var = unnestOp.getVariable();
int funcVarIndex = optFuncExpr.findLogicalVar(var);
// No matching var in optFuncExpr.
@@ -781,17 +823,18 @@
List<String> fieldName = null;
MutableInt fieldSource = new MutableInt(0);
if (subTree.getDataSourceType() == DataSourceType.COLLECTION_SCAN) {
- VariableReferenceExpression varRef = new VariableReferenceExpression(var);
- varRef.setSourceLocation(unnestOp.getSourceLocation());
- optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
+ ILogicalExpression expr = optFuncExpr.getArgument(funcVarIndex).getValue();
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ optFuncExpr.addStepExpr(funcVarIndex, ((AbstractFunctionCallExpression) expr));
+ }
+ optFuncExpr.setLogicalExpr(funcVarIndex, expr);
} else {
if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
subTree.setLastMatchedDataSourceVars(0, funcVarIndex);
}
- fieldName = AccessMethodUtils.getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
- subTree.getRecordType(), funcVarIndex,
- optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(), subTree.getMetaRecordType(),
- datasetMetaVar, fieldSource, false);
+ fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
+ subTree.getRecordType(), funcVarIndex, optFuncExpr.getArgument(funcVarIndex).getValue(),
+ subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
if (fieldName.isEmpty()) {
return;
}
@@ -806,17 +849,16 @@
setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
if (subTree.hasDataSource()) {
fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex, subTree,
- analysisCtx, fieldSource.intValue());
+ analysisCtx, fieldSource.intValue(), accessMethod);
}
}
private void analyzeAssignOp(AssignOperator assignOp, IOptimizableFuncExpr optFuncExpr,
OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetMetaVar,
IOptimizationContext context, List<Index> datasetIndexes, int optFuncExprIndex,
- AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+ AccessMethodAnalysisContext analysisCtx, IAccessMethod accessMethod) throws AlgebricksException {
boolean doesArrayIndexQualify = context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
- && datasetIndexes.stream().anyMatch(i -> i.getIndexType() == IndexType.ARRAY)
- && assignOrUnnestIndex == subTree.getAssignsAndUnnests().size() - 1;
+ && datasetIndexes.stream().anyMatch(i -> i.getIndexType() == IndexType.ARRAY);
List<LogicalVariable> varList = assignOp.getVariables();
MutableInt fieldSource = new MutableInt(0);
for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
@@ -825,12 +867,12 @@
if (optVarIndex == -1) {
if (doesArrayIndexQualify && subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
// We may be able to apply an array index to this variable.
- Triple<Integer, List<String>, IAType> fieldTriplet =
- AccessMethodUtils.analyzeVarForArrayIndexes(assignOp, optFuncExpr, subTree, datasetMetaVar,
- context, datasetIndexes, analysisCtx.getMatchedFuncExprs(), varIndex);
+ Triple<Integer, List<String>, IAType> fieldTriplet = AccessMethodUtils
+ .analyzeVarForArrayIndexes(datasetIndexes, optFuncExpr, subTree, context, var, analysisCtx);
if (fieldTriplet != null && subTree.hasDataSource()) {
fillIndexExprs(datasetIndexes, fieldTriplet.second, fieldTriplet.third, optFuncExpr,
- optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue());
+ optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue(),
+ accessMethod);
}
}
continue;
@@ -844,10 +886,10 @@
}
fieldSource.setValue(0);
- List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(optFuncExpr, subTree,
+ List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(optFuncExpr, subTree,
assignOrUnnestIndex, varIndex, subTree.getRecordType(), optVarIndex,
- optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(), subTree.getMetaRecordType(),
- datasetMetaVar, fieldSource, false);
+ optFuncExpr.getArgument(optVarIndex).getValue(), subTree.getMetaRecordType(), datasetMetaVar,
+ fieldSource, false);
IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
// Set the fieldName in the corresponding matched
@@ -858,15 +900,15 @@
setTypeTag(context, subTree, optFuncExpr, optVarIndex);
if (subTree.hasDataSource()) {
fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, optVarIndex,
- subTree, analysisCtx, fieldSource.intValue());
+ subTree, analysisCtx, fieldSource.intValue(), accessMethod);
}
}
}
private void matchVarsFromOptFuncExprToDataSourceScan(IOptimizableFuncExpr optFuncExpr, int optFuncExprIndex,
List<Index> datasetIndexes, List<LogicalVariable> dsVarList, OptimizableOperatorSubTree subTree,
- AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean fromAdditionalDataSource)
- throws AlgebricksException {
+ AccessMethodAnalysisContext analysisCtx, IOptimizationContext context, boolean fromAdditionalDataSource,
+ IAccessMethod accessMethod) throws AlgebricksException {
MutableInt mutableFieldSource = new MutableInt(0);
for (int varIndex = 0; varIndex < dsVarList.size(); varIndex++) {
LogicalVariable var = dsVarList.get(varIndex);
@@ -923,13 +965,15 @@
optFuncExpr.setFieldName(funcVarIndex, fieldName, fieldSource);
optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
optFuncExpr.setSourceVar(funcVarIndex, var);
- VariableReferenceExpression varRef = new VariableReferenceExpression(var);
- varRef.setSourceLocation(subTree.getDataSourceRef().getValue().getSourceLocation());
- optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
+ ILogicalExpression expr = optFuncExpr.getArgument(funcVarIndex).getValue();
+ if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ optFuncExpr.addStepExpr(funcVarIndex, ((AbstractFunctionCallExpression) expr));
+ }
+ optFuncExpr.setLogicalExpr(funcVarIndex, expr);
setTypeTag(context, subTree, optFuncExpr, funcVarIndex);
if (subTree.hasDataSourceScan()) {
fillIndexExprs(datasetIndexes, fieldName, fieldType, optFuncExpr, optFuncExprIndex, funcVarIndex,
- subTree, analysisCtx, fieldSource);
+ subTree, analysisCtx, fieldSource, accessMethod);
}
}
}
@@ -966,7 +1010,7 @@
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
fieldSource.setValue(0);
- List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree,
+ List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
if (fieldName != null && !fieldName.isEmpty()) {
@@ -981,7 +1025,7 @@
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
fieldSource.setValue(0);
- fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, 0,
+ fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree, assignOrUnnestIndex, 0,
subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource,
false);
if (fieldName != null && !fieldName.isEmpty()) {
@@ -1009,7 +1053,7 @@
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
fieldSource.setValue(0);
- List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree,
+ List<String> fieldName = AccessMethodUtils.getFieldNameSetStepsFromSubTree(null, subTree,
assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
if (fieldName != null && !fieldName.isEmpty()) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index 924c679..3d4986d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -26,6 +26,7 @@
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.optimizer.rules.am.array.ArrayIndexStructureMatcher;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Quadruple;
@@ -36,6 +37,7 @@
* Context for analyzing the applicability of a single access method.
*/
public class AccessMethodAnalysisContext {
+ private final ArrayIndexStructureMatcher arrayIndexStructureMatcher = new ArrayIndexStructureMatcher();
private List<IOptimizableFuncExpr> matchedFuncExprs = new ArrayList<IOptimizableFuncExpr>();
@@ -55,7 +57,7 @@
// See AccessMethodUtils#removeUnjoinedDuplicatesInLOJ() for a definition of a special GroupBy
// and extra output processing steps needed when it's not available.
private Mutable<ILogicalOperator> lojSpecialGroupByOpRef = null;
- private ScalarFunctionCallExpression lojIsMissingFuncInSpecialGroupBy = null;
+ private ScalarFunctionCallExpression lojIsMissingNullFuncInSpecialGroupBy = null;
// For a secondary index, if we use only PK and secondary key field in a plan, it is an index-only plan.
// Contains information about index-only plan
@@ -144,12 +146,12 @@
return lojSpecialGroupByOpRef;
}
- public void setLOJIsMissingFuncInSpecialGroupBy(ScalarFunctionCallExpression isMissingFunc) {
- lojIsMissingFuncInSpecialGroupBy = isMissingFunc;
+ public void setLOJIsMissingNullFuncInSpecialGroupBy(ScalarFunctionCallExpression isMissingNullFunc) {
+ lojIsMissingNullFuncInSpecialGroupBy = isMissingNullFunc;
}
- public ScalarFunctionCallExpression getLOJIsMissingFuncInSpecialGroupBy() {
- return lojIsMissingFuncInSpecialGroupBy;
+ public ScalarFunctionCallExpression getLOJIsMissingNullFuncInSpecialGroupBy() {
+ return lojIsMissingNullFuncInSpecialGroupBy;
}
public Dataset getDatasetFromIndexDatasetMap(Index idx) {
@@ -176,4 +178,7 @@
this.indexDatasetMap = indexDatasetMap;
}
+ public ArrayIndexStructureMatcher getArrayIndexStructureMatcher() {
+ return arrayIndexStructureMatcher;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index c78e89f..3b39d9d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -19,6 +19,10 @@
package org.apache.asterix.optimizer.rules.am;
+import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_BY_INDEX;
+import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_BY_NAME;
+import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_NESTED;
+
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -28,7 +32,6 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Set;
-import java.util.Stack;
import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -44,6 +47,7 @@
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.AInt32;
@@ -62,6 +66,7 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.commons.lang3.mutable.MutableObject;
@@ -79,6 +84,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -106,6 +112,8 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -125,14 +133,19 @@
CONDITIONAL_SPLIT_VAR
}
- // Function Identifier sets that retain the original field variable through each function's arguments
- private final static ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
- ImmutableSet.of(BuiltinFunctions.WORD_TOKENS, BuiltinFunctions.GRAM_TOKENS, BuiltinFunctions.SUBSTRING,
- BuiltinFunctions.SUBSTRING_BEFORE, BuiltinFunctions.SUBSTRING_AFTER,
- BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
- BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
- BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
- BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
+ public final static ImmutableSet<FunctionIdentifier> CAST_NULL_TYPE_CONSTRUCTORS = ImmutableSet.of(
+ BuiltinFunctions.BOOLEAN_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.INT8_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.INT16_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.INT32_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.INT64_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.FLOAT_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.DOUBLE_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.STRING_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT,
+ BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT,
+ BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT,
+ BuiltinFunctions.DURATION_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
+ BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
List<Object> target) throws AlgebricksException {
@@ -177,10 +190,12 @@
public static boolean analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(
AbstractFunctionCallExpression funcExpr, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
+ IOptimizationContext context, IVariableTypeEnvironment typeEnvironment, boolean allowFunctionExprArg)
+ throws AlgebricksException {
ILogicalExpression constExpression = null;
IAType constantExpressionType = null;
LogicalVariable fieldVar = null;
+ int varIndex;
ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
// One of the args must be a runtime constant, and the other arg must be a variable.
@@ -204,6 +219,7 @@
constExpression = arg1;
VariableReferenceExpression varExpr = (VariableReferenceExpression) arg2;
fieldVar = varExpr.getVariableReference();
+ varIndex = 1;
} else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
IAType expressionType = constantRuntimeResultType(arg2, context, typeEnvironment);
if (expressionType == null) {
@@ -223,27 +239,96 @@
VariableReferenceExpression varExpr = (VariableReferenceExpression) arg1;
fieldVar = varExpr.getVariableReference();
+ varIndex = 0;
} else {
- return false;
+ if (!allowFunctionExprArg) {
+ return false;
+ }
+ // extract the variable argument.
+ if (acceptExpressionArg(arg1, context, typeEnvironment)) {
+ // arg1 = expr, arg2 should be a constant
+ Pair<LogicalVariable, IAType> varConstType =
+ getVarAndConstExprType(arg1, arg2, context, typeEnvironment);
+ if (varConstType == null) {
+ return false;
+ }
+ fieldVar = varConstType.first;
+ constantExpressionType = varConstType.second;
+ constExpression = arg2;
+ varIndex = 0;
+ } else if (acceptExpressionArg(arg2, context, typeEnvironment)) {
+ // arg2 = expr, arg1 should be a constant
+ Pair<LogicalVariable, IAType> varConstType =
+ getVarAndConstExprType(arg2, arg1, context, typeEnvironment);
+ if (varConstType == null) {
+ return false;
+ }
+ fieldVar = varConstType.first;
+ constantExpressionType = varConstType.second;
+ constExpression = arg1;
+ varIndex = 1;
+ } else {
+ return false;
+ }
}
// Updates the given Analysis Context by adding a new optimizable function expression.
constructNewOptFuncExprAndAddToAnalysisCtx(funcExpr, fieldVar, constExpression, constantExpressionType,
- analysisCtx);
+ analysisCtx, varIndex);
return true;
}
+ private static boolean acceptExpressionArg(ILogicalExpression expr, IOptimizationContext context,
+ IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression funExpr = (AbstractFunctionCallExpression) expr;
+ List<Mutable<ILogicalExpression>> funArgs = funExpr.getArguments();
+ if (funArgs.size() <= 0) {
+ return false;
+ }
+ // first arg must be a variable and the rest are constants
+ if (funArgs.get(0).getValue().getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return false;
+ }
+ for (int i = 1; i < funArgs.size(); i++) {
+ IAType constExprType = constantRuntimeResultType(funArgs.get(i).getValue(), context, typeEnvironment);
+ if (constExprType == null) {
+ // not constant at runtime
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private static Pair<LogicalVariable, IAType> getVarAndConstExprType(ILogicalExpression exprWithVar,
+ ILogicalExpression constExpr, IOptimizationContext context, IVariableTypeEnvironment typeEnvironment)
+ throws AlgebricksException {
+ IAType constExprType = constantRuntimeResultType(constExpr, context, typeEnvironment);
+ if (constExprType == null) {
+ // not constant at runtime
+ return null;
+ }
+ AbstractFunctionCallExpression funExpr = (AbstractFunctionCallExpression) exprWithVar;
+ LogicalVariable varFromExpr =
+ ((VariableReferenceExpression) funExpr.getArguments().get(0).getValue()).getVariableReference();
+ return new Pair<>(varFromExpr, constExprType);
+ }
+
private static void constructNewOptFuncExprAndAddToAnalysisCtx(AbstractFunctionCallExpression funcExpr,
LogicalVariable fieldVar, ILogicalExpression expression, IAType expressionType,
- AccessMethodAnalysisContext analysisCtx) {
- OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr, fieldVar, expression, expressionType);
+ AccessMethodAnalysisContext analysisCtx, int varIndex) {
+ OptimizableFuncExpr newOptFuncExpr =
+ new OptimizableFuncExpr(funcExpr, fieldVar, expression, expressionType, varIndex);
addNewOptFuncExprToAnalysisCtx(funcExpr, newOptFuncExpr, analysisCtx);
}
private static void constructNewOptFuncExprAndAddToAnalysisCtx(AbstractFunctionCallExpression funcExpr,
- LogicalVariable[] fieldVars, ILogicalExpression[] expressions, IAType[] expressionTypes,
- AccessMethodAnalysisContext analysisCtx) {
- OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr, fieldVars, expressions, expressionTypes);
+ LogicalVariable[] fieldVars, int[] fieldVarsIdxes, ILogicalExpression[] expressions,
+ IAType[] expressionTypes, AccessMethodAnalysisContext analysisCtx) {
+ OptimizableFuncExpr newOptFuncExpr =
+ new OptimizableFuncExpr(funcExpr, fieldVars, fieldVarsIdxes, expressions, expressionTypes);
addNewOptFuncExprToAnalysisCtx(funcExpr, newOptFuncExpr, analysisCtx);
}
@@ -330,7 +415,7 @@
// Updates the given Analysis Context by adding a new optimizable function expression.
constructNewOptFuncExprAndAddToAnalysisCtx(funcExpr, new LogicalVariable[] { fieldVar1, fieldVar2 },
- new ILogicalExpression[0], new IAType[0], analysisCtx);
+ new int[] { 0, 1 }, new ILogicalExpression[0], new IAType[0], analysisCtx);
return true;
}
@@ -483,6 +568,25 @@
return primaryKeyVars;
}
+ public static class SearchKeyRoundingFunctionProvider {
+ public TypeCastingMathFunctionType getRoundingFunction(ComparisonKind cKind, Index chosenIndex,
+ IAType indexedFieldType, IAObject constantValue, boolean realTypeConvertedToIntegerType)
+ throws CompilationException {
+ switch (cKind) {
+ case LT:
+ case GE:
+ // round-up
+ return TypeCastingMathFunctionType.CEIL;
+ case LE:
+ case GT:
+ // round-down
+ return TypeCastingMathFunctionType.FLOOR;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, cKind.toString());
+ }
+ }
+ }
+
/**
* Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query
* then this method returns the a ConstantExpression from the first constant value in the optimizable function
@@ -493,8 +597,8 @@
* @throws AlgebricksException
*/
public static Triple<ILogicalExpression, ILogicalExpression, Boolean> createSearchKeyExpr(Index index,
- IOptimizableFuncExpr optFuncExpr, IAType indexedFieldType, OptimizableOperatorSubTree probeSubTree)
- throws AlgebricksException {
+ IOptimizableFuncExpr optFuncExpr, IAType indexedFieldType, OptimizableOperatorSubTree probeSubTree,
+ SearchKeyRoundingFunctionProvider roundingFunctionProvider) throws AlgebricksException {
SourceLocation sourceLoc = optFuncExpr.getFuncExpr().getSourceLocation();
if (probeSubTree == null) {
// We are optimizing a selection query. Search key is a constant.
@@ -570,18 +674,15 @@
switch (cKind) {
case LT:
- case GE:
- // round-up
- replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
- constantValueTag, indexedFieldTypeTag, index.isEnforced(),
- TypeCastingMathFunctionType.CEIL, sourceLoc);
- break;
case LE:
case GT:
- // round-down
- replacedConstantValue = getReplacedConstantValue(constantValue.getObject(),
- constantValueTag, indexedFieldTypeTag, index.isEnforced(),
- TypeCastingMathFunctionType.FLOOR, sourceLoc);
+ case GE:
+ TypeCastingMathFunctionType roundingFunction =
+ roundingFunctionProvider.getRoundingFunction(cKind, index, indexedFieldType,
+ constantValue.getObject(), realTypeConvertedToIntegerType);
+ replacedConstantValue =
+ getReplacedConstantValue(constantValue.getObject(), constantValueTag,
+ indexedFieldTypeTag, index.isEnforced(), roundingFunction, sourceLoc);
break;
case EQ:
// equality case - both CEIL and FLOOR need to be applied.
@@ -744,26 +845,28 @@
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, AccessMethodAnalysisContext analysisCtx,
IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
- ILogicalOperator indexSearchOp, LogicalVariable newNullPlaceHolderVar,
- Mutable<ILogicalExpression> conditionRef, Dataset dataset) throws AlgebricksException {
+ IAlgebricksConstantValue leftOuterMissingValue, ILogicalOperator indexSearchOp,
+ LogicalVariable newMissingNullPlaceHolderVar, Mutable<ILogicalExpression> conditionRef, Dataset dataset,
+ Index chosenIndex) throws AlgebricksException {
boolean isIndexOnlyPlan = analysisCtx.getIndexOnlyPlanInfo().getFirst();
List<LogicalVariable> probePKVars = null;
ILogicalOperator finalIndexSearchOp = indexSearchOp;
if (isLeftOuterJoin) {
if (isLeftOuterJoinWithSpecialGroupBy) {
- ScalarFunctionCallExpression lojFuncExprs = analysisCtx.getLOJIsMissingFuncInSpecialGroupBy();
- List<LogicalVariable> lojMissingVariables = new ArrayList<>();
- lojFuncExprs.getUsedVariables(lojMissingVariables);
- boolean lojMissingVarExist = !lojMissingVariables.isEmpty();
+ ScalarFunctionCallExpression lojFuncExprs = analysisCtx.getLOJIsMissingNullFuncInSpecialGroupBy();
+ List<LogicalVariable> lojMissingNullVariables = new ArrayList<>();
+ lojFuncExprs.getUsedVariables(lojMissingNullVariables);
+ boolean lojMissingNullVarExist = !lojMissingNullVariables.isEmpty();
// Resets the missing place holder variable.
- AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+ AccessMethodUtils.resetLOJMissingNullPlaceholderVarInGroupByOp(analysisCtx,
+ newMissingNullPlaceHolderVar, context);
- // For the index-only plan, if newNullPlaceHolderVar is not in the variable map of the union operator
+ // For the index-only plan, if newMissingNullPlaceHolderVar is not in the variable map of the union operator
// or if the variable is removed during the above method, we need to refresh the variable mapping in UNION.
if (isIndexOnlyPlan) {
finalIndexSearchOp = AccessMethodUtils.resetVariableMappingInUnionOpInIndexOnlyPlan(
- lojMissingVarExist, lojMissingVariables, indexSearchOp, afterJoinRefs, context);
+ lojMissingNullVarExist, lojMissingNullVariables, indexSearchOp, afterJoinRefs, context);
}
} else {
// We'll need to remove unjoined duplicates after the left outer join if there is no special GroupBy,
@@ -775,6 +878,25 @@
probePKVars = new ArrayList<>();
probeSubTree.getPrimaryKeyVars(null, probePKVars);
}
+ if (chosenIndex.getIndexType() == IndexType.ARRAY) {
+ ILogicalOperator probeRoot = probeSubTree.getRoot();
+ Triple<Set<LogicalVariable>, ILogicalOperator, FunctionalDependency> primaryKeyOpAndVars =
+ EquivalenceClassUtils.findOrCreatePrimaryKeyOpAndVariables(probeRoot, false, context);
+ probePKVars = new ArrayList<>(primaryKeyOpAndVars.first);
+ if (primaryKeyOpAndVars.third != null) {
+ context.addPrimaryKey(primaryKeyOpAndVars.third);
+ }
+ if (primaryKeyOpAndVars.second != null) {
+ // Update all previous usages of the probe subtree root to include this new ID op.
+ Mutable<ILogicalOperator> assignIdOpRef = new MutableObject<>(primaryKeyOpAndVars.second);
+ assignIdOpRef.getValue().getInputs().clear();
+ assignIdOpRef.getValue().getInputs().add(new MutableObject<>(probeRoot));
+ probeSubTree.setRoot(assignIdOpRef.getValue());
+ probeSubTree.setRootRef(assignIdOpRef);
+ context.computeAndSetTypeEnvironmentForOperator(assignIdOpRef.getValue());
+ finalIndexSearchOp = assignIdOpRef.getValue();
+ }
+ }
if (probePKVars == null || probePKVars.isEmpty()) {
return false;
}
@@ -817,7 +939,9 @@
} else {
// Non-index only plan case
indexSubTree.getDataSourceRef().setValue(finalIndexSearchOp);
- SelectOperator topSelectOp = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
+ SelectOperator topSelectOp = isLeftOuterJoin
+ ? new SelectOperator(conditionRef, leftOuterMissingValue, newMissingNullPlaceHolderVar)
+ : new SelectOperator(conditionRef);
topSelectOp.setSourceLocation(sourceLoc);
topSelectOp.getInputs().add(indexSubTree.getRootRef());
topSelectOp.setExecutionMode(ExecutionMode.LOCAL);
@@ -833,7 +957,8 @@
}
if (isLeftOuterJoin && !isLeftOuterJoinWithSpecialGroupBy) {
- finalOp = removeUnjoinedDuplicatesInLOJ(finalOp, probePKVars, newNullPlaceHolderVar, context, sourceLoc);
+ finalOp = removeUnjoinedDuplicatesInLOJ(finalOp, probePKVars, newMissingNullPlaceHolderVar,
+ leftOuterMissingValue, context, sourceLoc);
}
joinRef.setValue(finalOp);
@@ -845,12 +970,12 @@
* (see {@link IntroduceJoinAccessMethodRule#checkAndApplyJoinTransformation(Mutable, IOptimizationContext)}.
* A "Special GroupBy" is a GroupBy that eliminates unjoined duplicates that might be produced by the secondary
* index probe. We probe secondary indexes on each index partition and return a tuple with a right branch variable
- * set to MISSING if there's no match on that partition. Therefore if there's more than one partition where
- * nothing joined then the index probe plan will produce several such MISSING tuples, however the join result
- * must have a single MISSING tuple for each unjoined left branch tuple. If the special GroupBy is available then
- * it'll eliminate those MISSING duplicates, otherwise this method is called to produce additional operators to
- * achieve that. The special GroupBy operators are introduced by the optimizer when rewriting FROM-LET or
- * equivalent patterns into a left outer join with parent a group by.
+ * set to MISSING (or NULL) if there's no match on that partition. Therefore if there's more than one partition
+ * where nothing joined then the index probe plan will produce several such MISSING (or NULL) tuples, however the
+ * join result must have a single MISSING (or NULL) tuple for each unjoined left branch tuple. If the special
+ * GroupBy is available then it'll eliminate those MISSING (or NULL) duplicates, otherwise this method is
+ * called to produce additional operators to achieve that. The special GroupBy operators are introduced by
+ * the optimizer when rewriting FROM-LET or equivalent patterns into a left outer join with parent a group by.
* <p>
* The plan generated by this method to eliminate unjoined duplicates is as follows:
* <ul>
@@ -868,8 +993,9 @@
* the Select operator eliminates those unjoined duplicate tuples.
*/
private static SelectOperator removeUnjoinedDuplicatesInLOJ(ILogicalOperator inputOp,
- List<LogicalVariable> probePKVars, LogicalVariable newNullPlaceHolderVar, IOptimizationContext context,
- SourceLocation sourceLoc) throws AlgebricksException {
+ List<LogicalVariable> probePKVars, LogicalVariable newNullPlaceHolderVar,
+ IAlgebricksConstantValue lojMissingValue, IOptimizationContext context, SourceLocation sourceLoc)
+ throws AlgebricksException {
if (probePKVars == null || probePKVars.isEmpty()) {
throw new IllegalArgumentException();
}
@@ -882,16 +1008,17 @@
VariableReferenceExpression winOrderByVarRef = new VariableReferenceExpression(newNullPlaceHolderVar);
winOrderByVarRef.setSourceLocation(sourceLoc);
- /* Sort in DESC order, so all MISSING values are at the end */
+ /* Sort in DESC order, so all MISSING (or NULL) values are at the end */
Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> winOrderByPair =
new Pair<>(OrderOperator.DESC_ORDER, new MutableObject<>(winOrderByVarRef));
LogicalVariable winVar = context.newVar();
VariableReferenceExpression winOrderByVarRef2 = new VariableReferenceExpression(newNullPlaceHolderVar);
winOrderByVarRef2.setSourceLocation(sourceLoc);
- AbstractFunctionCallExpression winExpr =
- BuiltinFunctions.makeWindowFunctionExpression(BuiltinFunctions.WIN_MARK_FIRST_MISSING_IMPL,
- Collections.singletonList(new MutableObject<>(winOrderByVarRef2)));
+ FunctionIdentifier winMarkFirstUnknownValueFn = lojMissingValue.isNull()
+ ? BuiltinFunctions.WIN_MARK_FIRST_NULL_IMPL : BuiltinFunctions.WIN_MARK_FIRST_MISSING_IMPL;
+ AbstractFunctionCallExpression winExpr = BuiltinFunctions.makeWindowFunctionExpression(
+ winMarkFirstUnknownValueFn, Collections.singletonList(new MutableObject<>(winOrderByVarRef2)));
WindowOperator winOp = new WindowOperator(winPartitionByList, Collections.singletonList(winOrderByPair));
winOp.getVariables().add(winVar);
@@ -903,7 +1030,7 @@
VariableReferenceExpression winVarRef = new VariableReferenceExpression(winVar);
winVarRef.setSourceLocation(sourceLoc);
- SelectOperator selectOp = new SelectOperator(new MutableObject<>(winVarRef), false, null);
+ SelectOperator selectOp = new SelectOperator(new MutableObject<>(winVarRef));
selectOp.getInputs().add(new MutableObject<>(winOp));
selectOp.setExecutionMode(ExecutionMode.LOCAL);
selectOp.setSourceLocation(sourceLoc);
@@ -915,7 +1042,8 @@
public static ILogicalOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType,
ARecordType metaRecordType, Index index, ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams,
IOptimizationContext context, boolean retainInput, boolean retainNull,
- boolean generateInstantTrylockResultFromIndexSearch) throws AlgebricksException {
+ boolean generateInstantTrylockResultFromIndexSearch, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
SourceLocation sourceLoc = inputOp.getSourceLocation();
// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<>();
@@ -945,7 +1073,7 @@
if (retainInput) {
LeftOuterUnnestMapOperator secondaryIndexLeftOuterUnnestOp = new LeftOuterUnnestMapOperator(
secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc),
- secondaryIndexOutputTypes, true);
+ secondaryIndexOutputTypes, leftOuterMissingValue);
secondaryIndexLeftOuterUnnestOp.setSourceLocation(sourceLoc);
secondaryIndexLeftOuterUnnestOp
.setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
@@ -976,7 +1104,8 @@
ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput,
boolean retainMissing, boolean requiresBroadcast, boolean requiresDistinct,
List<LogicalVariable> primaryKeyVars, List<LogicalVariable> primaryIndexUnnestVars,
- List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes) throws AlgebricksException {
+ List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
SourceLocation sourceLoc = inputOp.getSourceLocation();
// Sanity check: requiresDistinct and sortPrimaryKeys are mutually exclusive.
@@ -1028,11 +1157,11 @@
// Creates the primary-index search unnest-map operator.
AbstractUnnestMapOperator primaryIndexUnnestMapOp =
createPrimaryIndexUnnestMapOp(dataset, retainInput, retainMissing, requiresBroadcast, primaryKeyVars,
- primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc);
+ primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc, leftOuterMissingValue);
if (requiresDistinct) {
- primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(distinct));
+ primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(distinct));
} else if (sortPrimaryKeys) {
- primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+ primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(order));
} else {
primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(inputOp));
}
@@ -1047,8 +1176,9 @@
ARecordType metaRecordType, ILogicalOperator inputOp, IOptimizationContext context, boolean retainInput,
boolean retainMissing, boolean requiresBroadcast, Index secondaryIndex,
AccessMethodAnalysisContext analysisCtx, OptimizableOperatorSubTree subTree,
- LogicalVariable newMissingPlaceHolderForLOJ, List<LogicalVariable> pkVarsFromSIdxUnnestMapOp,
- List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
+ LogicalVariable newMissingPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue,
+ List<LogicalVariable> pkVarsFromSIdxUnnestMapOp, List<LogicalVariable> primaryIndexUnnestVars,
+ List<Object> primaryIndexOutputTypes, boolean anyRealTypeConvertedToIntegerType)
throws AlgebricksException {
SourceLocation sourceLoc = inputOp.getSourceLocation();
Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
@@ -1111,8 +1241,8 @@
if (idxType == IndexType.RTREE && (skFieldUsedAfterTopOp || requireVerificationAfterSIdxSearch)) {
IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(secondaryIndex, analysisCtx);
int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(secondaryIndex, analysisCtx);
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFuncExpr.getFieldType(optFieldIdx),
- optFuncExpr.getFieldName(optFieldIdx), recordType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryIndex,
+ optFuncExpr.getFieldType(optFieldIdx), optFuncExpr.getFieldName(optFieldIdx), recordType);
if (keyPairType == null) {
return null;
}
@@ -1425,7 +1555,7 @@
// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
retainMissing, requiresBroadcast, pkVarsInLeftPathFromSIdxSearchBeforeSplit,
- pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc);
+ pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc, leftOuterMissingValue);
primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(origVarsToLeftPathVarsAssignOp));
context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
@@ -1442,8 +1572,9 @@
ILogicalExpression conditionRefExpr = conditionRef.getValue().cloneExpression();
// The retainMissing variable contains the information whether we are optimizing a left-outer join or not.
LogicalVariable newMissingPlaceHolderVar = retainMissing ? newMissingPlaceHolderForLOJ : null;
- newSelectOpInLeftPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr),
- retainMissing, newMissingPlaceHolderVar);
+ newSelectOpInLeftPath =
+ retainMissing ? new SelectOperator(new MutableObject<>(conditionRefExpr), leftOuterMissingValue,
+ newMissingPlaceHolderVar) : new SelectOperator(new MutableObject<>(conditionRefExpr));
newSelectOpInLeftPath.setSourceLocation(conditionRefExpr.getSourceLocation());
VariableUtilities.substituteVariables(newSelectOpInLeftPath, origVarToNewVarInLeftPathMap, context);
@@ -1497,13 +1628,17 @@
// other than joining fields, then those conditions need to be applied to filter out
// false positive results in the right path.
// (e.g., where $a.authors /*+ indexnl */ = $b.authors and $a.id = $b.id <- authors:SK, id:PK)
- if ((idxType == IndexType.RTREE || uniqueUsedVarsInTopOp.size() > 1) && requireVerificationAfterSIdxSearch) {
+ if (((idxType == IndexType.RTREE || uniqueUsedVarsInTopOp.size() > 1) && requireVerificationAfterSIdxSearch)
+ || anyRealTypeConvertedToIntegerType) {
// Creates a new SELECT operator by deep-copying the SELECT operator in the left path
// since we need to change the variable reference in the SELECT operator.
// For the index-nested-loop join case, we copy the condition of the join operator.
ILogicalExpression conditionRefExpr2 = conditionRef.getValue().cloneExpression();
- newSelectOpInRightPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr2),
- retainMissing, newMissingPlaceHolderVar);
+ newSelectOpInRightPath =
+ retainMissing
+ ? new SelectOperator(new MutableObject<>(conditionRefExpr2), leftOuterMissingValue,
+ newMissingPlaceHolderVar)
+ : new SelectOperator(new MutableObject<>(conditionRefExpr2));
newSelectOpInRightPath.setSourceLocation(conditionRefExpr2.getSourceLocation());
newSelectOpInRightPath.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
VariableUtilities.substituteVariables(newSelectOpInRightPath, origVarToSIdxUnnestMapOpVarMap, context);
@@ -1553,7 +1688,7 @@
private static AbstractUnnestMapOperator createPrimaryIndexUnnestMapOp(Dataset dataset, boolean retainInput,
boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes,
- SourceLocation sourceLoc) throws AlgebricksException {
+ SourceLocation sourceLoc, IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
@@ -1573,12 +1708,11 @@
// This is the operator that jobgen will be looking for. It contains an unnest function that has
// all necessary arguments to determine which index to use, which variables contain the index-search keys,
// what is the original dataset, etc.
- AbstractUnnestMapOperator primaryIndexUnnestMapOp = null;
+ AbstractUnnestMapOperator primaryIndexUnnestMapOp;
if (retainMissing) {
if (retainInput) {
primaryIndexUnnestMapOp = new LeftOuterUnnestMapOperator(primaryIndexUnnestVars,
- new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
- retainInput);
+ new MutableObject<>(primaryIndexSearchFunc), primaryIndexOutputTypes, leftOuterMissingValue);
primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
} else {
// Left-outer-join without retainNull and retainInput doesn't make sense.
@@ -1587,8 +1721,7 @@
}
} else {
primaryIndexUnnestMapOp = new UnnestMapOperator(primaryIndexUnnestVars,
- new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
- retainInput);
+ new MutableObject<>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
}
return primaryIndexUnnestMapOp;
@@ -1619,7 +1752,8 @@
IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainMissing,
boolean requiresBroadcast, Index secondaryIndex, AccessMethodAnalysisContext analysisCtx,
OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
- LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue,
+ boolean anyRealTypeConvertedToIntegerType) throws AlgebricksException {
// Common part for the non-index-only plan and index-only plan
// Variables and types for the primary-index search.
List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
@@ -1638,23 +1772,42 @@
// Non-index-only plan case: creates (ORDER)? -> (DISTINCT)? -> UNNEST-MAP(PIDX) and return that unnest-map op.
if (!isIndexOnlyPlan) {
- // If we have a join + an array index, we need add the join source PK to the DISTINCT + ORDER.
- List<LogicalVariable> joinPKVars = Collections.emptyList();
+ // If we have a join + an array index, we need add the join source PK(s) to the DISTINCT + ORDER.
+ List<LogicalVariable> joinPKVars;
if (isArrayIndex && probeSubTree != null) {
- joinPKVars = probeSubTree.getDataSourceVariables().subList(0,
- probeSubTree.getDataSourceVariables().size() - 1);
+ ILogicalOperator probeRoot = probeSubTree.getRoot();
+ Triple<Set<LogicalVariable>, ILogicalOperator, FunctionalDependency> primaryKeyOpAndVars =
+ EquivalenceClassUtils.findOrCreatePrimaryKeyOpAndVariables(probeRoot, false, context);
+ joinPKVars = new ArrayList<>(primaryKeyOpAndVars.first);
+ if (primaryKeyOpAndVars.third != null) {
+ context.addPrimaryKey(primaryKeyOpAndVars.third);
+ }
+ if (primaryKeyOpAndVars.second != null) {
+ // Update all previous usages of the probe subtree root to include this new ID op.
+ Mutable<ILogicalOperator> assignIdOpRef = new MutableObject<>(primaryKeyOpAndVars.second);
+ OperatorManipulationUtil.substituteOpInInput(topOpRef.getValue(), probeRoot, assignIdOpRef);
+ OperatorManipulationUtil.substituteOpInInput(inputOp, probeRoot, assignIdOpRef);
+ probeSubTree.setRoot(primaryKeyOpAndVars.second);
+ probeSubTree.setRootRef(assignIdOpRef);
+ context.computeAndSetTypeEnvironmentForOperator(primaryKeyOpAndVars.second);
+ context.computeAndSetTypeEnvironmentForOperator(topOpRef.getValue());
+ }
+
+ } else {
+ joinPKVars = Collections.emptyList();
}
return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, !isArrayIndex && sortPrimaryKeys,
retainInput, retainMissing, requiresBroadcast, isArrayIndex, pkVarsFromSIdxUnnestMapOp,
- primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes);
+ primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes, leftOuterMissingValue);
} else if (!isArrayIndex) {
// Index-only plan case: creates a UNIONALL operator that has two paths after the secondary unnest-map op,
// and returns it.
return createFinalIndexOnlySearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignsBeforeTopOpRef,
dataset, recordType, metaRecordType, inputOp, context, retainInput, retainMissing,
requiresBroadcast, secondaryIndex, analysisCtx, indexSubTree, newMissingPlaceHolderForLOJ,
- pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
+ leftOuterMissingValue, pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes,
+ anyRealTypeConvertedToIntegerType);
} else {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, inputOp.getSourceLocation(),
"Cannot use index-only plan with array indexes.");
@@ -1689,23 +1842,23 @@
return createRectangleExpr;
}
- private static ScalarFunctionCallExpression getNestedIsMissingCall(AbstractFunctionCallExpression call,
- OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
- ScalarFunctionCallExpression isMissingFuncExpr;
+ private static ScalarFunctionCallExpression getNestedIsMissingNullCall(AbstractFunctionCallExpression call,
+ OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+ ScalarFunctionCallExpression isMissingNullFuncExpr;
if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
if (call.getArguments().get(0).getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
if (((AbstractFunctionCallExpression) call.getArguments().get(0).getValue()).getFunctionIdentifier()
- .equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
- isMissingFuncExpr = (ScalarFunctionCallExpression) call.getArguments().get(0).getValue();
- if (isMissingFuncExpr.getArguments().get(0).getValue()
+ .equals(funId)) {
+ isMissingNullFuncExpr = (ScalarFunctionCallExpression) call.getArguments().get(0).getValue();
+ if (isMissingNullFuncExpr.getArguments().get(0).getValue()
.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
LogicalVariable var =
- ((VariableReferenceExpression) isMissingFuncExpr.getArguments().get(0).getValue())
+ ((VariableReferenceExpression) isMissingNullFuncExpr.getArguments().get(0).getValue())
.getVariableReference();
List<LogicalVariable> liveSubplanVars = new ArrayList<>();
VariableUtilities.getSubplanLocalLiveVariables(rightSubTree.getRoot(), liveSubplanVars);
if (liveSubplanVars.contains(var)) {
- return isMissingFuncExpr;
+ return isMissingNullFuncExpr;
}
}
}
@@ -1714,9 +1867,9 @@
return null;
}
- public static ScalarFunctionCallExpression findIsMissingInSubplan(AbstractLogicalOperator inputOp,
- OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
- ScalarFunctionCallExpression isMissingFuncExpr = null;
+ public static ScalarFunctionCallExpression findIsMissingNullInSubplan(AbstractLogicalOperator inputOp,
+ OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+ ScalarFunctionCallExpression isMissingNullFuncExpr = null;
AbstractLogicalOperator currentOp = inputOp;
while (currentOp != null) {
if (currentOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
@@ -1727,27 +1880,27 @@
if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
for (Mutable<ILogicalExpression> mexpr : call.getArguments()) {
if (mexpr.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
- isMissingFuncExpr = getNestedIsMissingCall(
- (AbstractFunctionCallExpression) mexpr.getValue(), rightSubTree);
- if (isMissingFuncExpr != null) {
- return isMissingFuncExpr;
+ isMissingNullFuncExpr = getNestedIsMissingNullCall(
+ (AbstractFunctionCallExpression) mexpr.getValue(), rightSubTree, funId);
+ if (isMissingNullFuncExpr != null) {
+ return isMissingNullFuncExpr;
}
}
}
}
- isMissingFuncExpr = getNestedIsMissingCall(call, rightSubTree);
- if (isMissingFuncExpr != null) {
- return isMissingFuncExpr;
+ isMissingNullFuncExpr = getNestedIsMissingNullCall(call, rightSubTree, funId);
+ if (isMissingNullFuncExpr != null) {
+ return isMissingNullFuncExpr;
}
}
} else if (currentOp.hasNestedPlans()) {
AbstractOperatorWithNestedPlans nestedPlanOp = (AbstractOperatorWithNestedPlans) currentOp;
for (ILogicalPlan nestedPlan : nestedPlanOp.getNestedPlans()) {
for (Mutable<ILogicalOperator> root : nestedPlan.getRoots()) {
- isMissingFuncExpr =
- findIsMissingInSubplan((AbstractLogicalOperator) root.getValue(), rightSubTree);
- if (isMissingFuncExpr != null) {
- return isMissingFuncExpr;
+ isMissingNullFuncExpr = findIsMissingNullInSubplan((AbstractLogicalOperator) root.getValue(),
+ rightSubTree, funId);
+ if (isMissingNullFuncExpr != null) {
+ return isMissingNullFuncExpr;
}
}
}
@@ -1755,27 +1908,29 @@
currentOp = currentOp.getInputs().isEmpty() ? null
: (AbstractLogicalOperator) currentOp.getInputs().get(0).getValue();
}
- return isMissingFuncExpr;
+ return isMissingNullFuncExpr;
}
- public static ScalarFunctionCallExpression findLOJIsMissingFuncInGroupBy(GroupByOperator lojGroupbyOp,
- OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
- //find IS_MISSING function of which argument has the nullPlaceholder variable in the nested plan of groupby.
+ public static ScalarFunctionCallExpression findLOJIsMissingNullFuncInGroupBy(GroupByOperator lojGroupbyOp,
+ OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+ //find IS_MISSING or IS_NULL function of which argument has the nullPlaceholder variable in the nested plan of groupby.
ALogicalPlanImpl subPlan = (ALogicalPlanImpl) lojGroupbyOp.getNestedPlans().get(0);
Mutable<ILogicalOperator> subPlanRootOpRef = subPlan.getRoots().get(0);
AbstractLogicalOperator subPlanRootOp = (AbstractLogicalOperator) subPlanRootOpRef.getValue();
- return findIsMissingInSubplan(subPlanRootOp, rightSubTree);
+ return findIsMissingNullInSubplan(subPlanRootOp, rightSubTree, funId);
}
- public static void resetLOJMissingPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx,
- LogicalVariable newMissingPlaceholderVaraible, IOptimizationContext context) throws AlgebricksException {
+ public static void resetLOJMissingNullPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx,
+ LogicalVariable newMissingNullPlaceholderVaraible, IOptimizationContext context)
+ throws AlgebricksException {
//reset the missing placeholder variable in groupby operator
- ScalarFunctionCallExpression isMissingFuncExpr = analysisCtx.getLOJIsMissingFuncInSpecialGroupBy();
- isMissingFuncExpr.getArguments().clear();
- VariableReferenceExpression newMissingVarRef = new VariableReferenceExpression(newMissingPlaceholderVaraible);
- newMissingVarRef.setSourceLocation(isMissingFuncExpr.getSourceLocation());
- isMissingFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingVarRef));
+ ScalarFunctionCallExpression isMissingNullFuncExpr = analysisCtx.getLOJIsMissingNullFuncInSpecialGroupBy();
+ isMissingNullFuncExpr.getArguments().clear();
+ VariableReferenceExpression newMissingNullVarRef =
+ new VariableReferenceExpression(newMissingNullPlaceholderVaraible);
+ newMissingNullVarRef.setSourceLocation(isMissingNullFuncExpr.getSourceLocation());
+ isMissingNullFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingNullVarRef));
//recompute type environment.
OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJSpecialGroupByOpRef(), context);
@@ -1968,7 +2123,8 @@
// an inverted index contains a part of a field value, not all of it.
if (noIndexOnlyPlanOption || dataset.getDatasetType() == DatasetType.EXTERNAL || chosenIndex.isPrimaryIndex()
|| chosenIndex.getIndexDetails().isOverridingKeyFieldTypes() || chosenIndex.isEnforced()
- || isInvertedIndex(chosenIndex) || chosenIndex.getIndexType() == IndexType.ARRAY) {
+ || isInvertedIndex(chosenIndex) || chosenIndex.getIndexType() == IndexType.ARRAY
+ || IndexUtil.includesUnknowns(chosenIndex)) {
indexOnlyPlanInfo.setFirst(false);
return;
}
@@ -2374,8 +2530,8 @@
}
int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(chosenIndex, analysisCtx);
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFuncExpr.getFieldType(optFieldIdx),
- optFuncExpr.getFieldName(optFieldIdx), recordType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(chosenIndex,
+ optFuncExpr.getFieldType(optFieldIdx), optFuncExpr.getFieldName(optFieldIdx), recordType);
if (keyPairType == null) {
return false;
}
@@ -2634,10 +2790,10 @@
*
* @throws AlgebricksException
*/
- public static ILogicalOperator resetVariableMappingInUnionOpInIndexOnlyPlan(boolean LOJVarExist,
- List<LogicalVariable> LOJMissingVariables, ILogicalOperator unionAllOp,
+ public static ILogicalOperator resetVariableMappingInUnionOpInIndexOnlyPlan(boolean lojVarExist,
+ List<LogicalVariable> lojMissingNullVariables, ILogicalOperator unionAllOp,
List<Mutable<ILogicalOperator>> aboveTopRefs, IOptimizationContext context) throws AlgebricksException {
- // For an index-only plan, if newNullPlaceHolderVar is not in the variable map of the UNIONALL operator,
+ // For an index-only plan, if newMissingNullPlaceHolderVar is not in the variable map of the UNIONALL operator,
// we need to add this variable to the map.
// Also, we need to delete replaced variables in the map if it was used only in the group-by operator.
if (unionAllOp.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
@@ -2646,15 +2802,15 @@
// First, check whether the given old variable can be deleted. If it is used somewhere else
// except the group-by operator, we can't delete it since we need to propagate it.
- boolean LOJVarCanBeDeleted = true;
- if (LOJVarExist) {
+ boolean lojVarCanBeDeleted = true;
+ if (lojVarExist) {
List<LogicalVariable> usedVars = new ArrayList<>();
for (int i = 0; i < aboveTopRefs.size(); i++) {
usedVars.clear();
ILogicalOperator lOp = aboveTopRefs.get(i).getValue();
VariableUtilities.getUsedVariables(lOp, usedVars);
- if (usedVars.containsAll(LOJMissingVariables) && lOp.getOperatorTag() != LogicalOperatorTag.GROUP) {
- LOJVarCanBeDeleted = false;
+ if (usedVars.containsAll(lojMissingNullVariables) && lOp.getOperatorTag() != LogicalOperatorTag.GROUP) {
+ lojVarCanBeDeleted = false;
break;
}
}
@@ -2663,20 +2819,20 @@
List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap =
((UnionAllOperator) unionAllOp).getVariableMappings();
- if (LOJVarExist && LOJVarCanBeDeleted) {
+ if (lojVarExist && lojVarCanBeDeleted) {
// Delete old variables from the map.
for (Iterator<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> it = varMap.iterator(); it
.hasNext();) {
Triple<LogicalVariable, LogicalVariable, LogicalVariable> tripleVars = it.next();
- if (tripleVars.first.equals(LOJMissingVariables.get(0))
- || tripleVars.second.equals(LOJMissingVariables.get(0))
- || tripleVars.third.equals(LOJMissingVariables.get(0))) {
+ if (tripleVars.first.equals(lojMissingNullVariables.get(0))
+ || tripleVars.second.equals(lojMissingNullVariables.get(0))
+ || tripleVars.third.equals(lojMissingNullVariables.get(0))) {
it.remove();
}
}
}
- if (LOJVarExist && LOJVarCanBeDeleted) {
+ if (lojVarExist && lojVarCanBeDeleted) {
UnionAllOperator newUnionAllOp = new UnionAllOperator(varMap);
newUnionAllOp.getInputs()
.add(new MutableObject<ILogicalOperator>(unionAllOp.getInputs().get(0).getValue()));
@@ -2742,6 +2898,19 @@
return ann == null ? null : ann.getIndexNames();
}
+ public static List<String> getFieldNameSetStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
+ OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
+ int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
+ MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+ if (optFuncExpr != null) {
+ if (parentFuncExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ optFuncExpr.addStepExpr(funcVarIndex, ((AbstractFunctionCallExpression) parentFuncExpr));
+ }
+ }
+ return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, opIndex, assignVarIndex, recordType, funcVarIndex,
+ parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ }
+
/**
* Returns the field name corresponding to the assigned variable at
* varIndex. Returns Collections.emptyList() if the expr at varIndex does not yield to a field
@@ -2749,7 +2918,7 @@
*
* @throws AlgebricksException
*/
- public static List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr,
+ private static List<String> getFieldNameAndStepsFromSubTree(IOptimizableFuncExpr optFuncExpr,
OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
@@ -2761,7 +2930,7 @@
AssignOperator assignOp = (AssignOperator) op;
expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
// Can't get a field name from a constant expression. So, return null.
- if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return Collections.emptyList();
}
childFuncExpr = (AbstractFunctionCallExpression) expr;
@@ -2827,6 +2996,7 @@
}
if (optFuncExpr != null) {
optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
+ optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
}
int[] assignAndExpressionIndexes = null;
@@ -2849,7 +3019,7 @@
for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
LogicalVariable var = varList.get(varIndex);
ArrayList<LogicalVariable> parentVars = new ArrayList<>();
- expr.getUsedVariables(parentVars);
+ funcExpr.getUsedVariables(parentVars);
if (parentVars.contains(var)) {
//Found the variable we are looking for.
@@ -2863,7 +3033,7 @@
//We found the nested assign
//Recursive call on nested assign
- List<String> parentFieldNames = getFieldNameFromSubTree(optFuncExpr, subTree,
+ List<String> parentFieldNames = getFieldNameAndStepsFromSubTree(optFuncExpr, subTree,
assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
@@ -2924,20 +3094,29 @@
}
- if (!funcIDSetThatRetainFieldName.contains(funcIdent)) {
- return Collections.emptyList();
- }
// We use a part of the field in edit distance computation
if (optFuncExpr != null
&& optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
optFuncExpr.setPartialField(true);
}
+ List<Mutable<ILogicalExpression>> funcArgs = funcExpr.getArguments();
+ if (funcArgs.isEmpty()) {
+ return Collections.emptyList();
+ }
// We expect the function's argument to be a variable, otherwise we
// cannot apply an index.
- ILogicalExpression argExpr = funcExpr.getArguments().get(0).getValue();
+ ILogicalExpression argExpr = funcArgs.get(0).getValue();
if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
return Collections.emptyList();
}
+ for (int i = 1; i < funcArgs.size(); i++) {
+ if (funcArgs.get(i).getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return Collections.emptyList();
+ }
+ }
+ if (optFuncExpr != null) {
+ optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
+ }
LogicalVariable curVar = ((VariableReferenceExpression) argExpr).getVariableReference();
// We look for the assign or unnest operator that produces curVar below
// the current operator
@@ -2951,29 +3130,36 @@
LogicalVariable var = varList.get(varIndex);
if (var.equals(curVar) && optFuncExpr != null) {
optFuncExpr.setSourceVar(funcVarIndex, var);
- return getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex, recordType,
- funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ return getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
+ recordType, funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource,
+ isUnnestOverVarAllowed);
}
}
} else {
UnnestOperator unnestOp = (UnnestOperator) curOp;
LogicalVariable var = unnestOp.getVariable();
if (var.equals(curVar)) {
- getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType, funcVarIndex,
- childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType,
+ funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
}
}
}
return Collections.emptyList();
}
- /**
- * Determine whether an array index can be used for the given variable.
- */
- public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(AssignOperator assignOp,
- IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree, LogicalVariable datasetMetaVar,
- IOptimizationContext context, List<Index> datasetIndexes, List<IOptimizableFuncExpr> matchedFuncExprs,
- int assignVarIndex) throws AlgebricksException {
+ public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(List<Index> datasetIndexes,
+ IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree, IOptimizationContext context,
+ LogicalVariable assignVar, AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+ // Set the logical expression we are working with.
+ final int lastMatchedDataSourceVar = subTree.getLastMatchedDataSourceVars().second;
+ if (lastMatchedDataSourceVar < 0) {
+ return null;
+ }
+ final ILogicalExpression optVarExpr = optFuncExpr.getArgument(lastMatchedDataSourceVar).getValue();
+ if (optVarExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ optFuncExpr.addStepExpr(lastMatchedDataSourceVar, ((AbstractFunctionCallExpression) optVarExpr));
+ }
+ optFuncExpr.setLogicalExpr(lastMatchedDataSourceVar, optVarExpr);
for (Index index : datasetIndexes) {
if (index.getIndexType() != IndexType.ARRAY) {
@@ -2988,199 +3174,44 @@
// We have found the array field for an array index.
for (List<String> project : e.getProjectList()) {
- List<String> flattenedFieldName =
- ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project);
- List<Integer> arrayIndicator = ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project);
+ List<String> flatName = ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project);
+ List<Boolean> unnestFlags = ArrayIndexUtil.getUnnestFlags(e.getUnnestList(), project);
+ analysisCtx.getArrayIndexStructureMatcher().reset(assignVar, subTree);
+ ArrayIndexUtil.walkArrayPath(index, subTree.getRecordType(), flatName, unnestFlags,
+ analysisCtx.getArrayIndexStructureMatcher());
- Triple<Integer, List<String>, IAType> fieldTriplet =
- matchAssignFieldInUnnestAssignStack(assignOp.getVariables().get(assignVarIndex),
- assignVarIndex, optFuncExpr, subTree, datasetMetaVar, context, arrayIndicator,
- flattenedFieldName, arrayIndexDetails.isOverridingKeyFieldTypes());
-
- // This specific field aligns with our array index.
- if (fieldTriplet.first > -1) {
- int optVarIndex = fieldTriplet.first;
- List<String> fieldName = fieldTriplet.second;
- IAType fieldType = fieldTriplet.third;
+ LogicalVariable varAfterWalk = analysisCtx.getArrayIndexStructureMatcher().getEndVar();
+ ILogicalOperator opAfterWalk = analysisCtx.getArrayIndexStructureMatcher().getEndOperator();
+ if (varAfterWalk != null && opAfterWalk != null) {
+ // This specific field aligns with an array index. Verify that this variable actually exists
+ // in our function expression.
+ int optVarIndex = optFuncExpr.findLogicalVar(varAfterWalk);
+ if (optVarIndex == -1) {
+ continue;
+ }
+ IAType fieldType =
+ (IAType) context.getOutputTypeEnvironment(opAfterWalk).getVarType(varAfterWalk);
+ optFuncExpr.setSourceVar(optVarIndex, varAfterWalk);
// Remember matching subtree.
optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
MutableInt fieldSource = new MutableInt(0);
- optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
+ optFuncExpr.setFieldName(optVarIndex, flatName, fieldSource.intValue());
optFuncExpr.setFieldType(optVarIndex, fieldType);
IAType type = (IAType) context.getOutputTypeEnvironment(subTree.getRoot())
.getVarType(optFuncExpr.getLogicalVar(optVarIndex));
optFuncExpr.setFieldType(optVarIndex, type);
- return fieldTriplet;
+ return new Triple<>(optVarIndex, flatName, fieldType);
}
}
}
}
-
return null;
}
- /**
- * @param assignVar Variable from lowest assign that we are trying to match (i.e. the first array step var).
- * @param assignVarIndex Index of the variable from the lowest assign.
- * @param optFuncExpr The function expression we are trying to optimize.
- * @param subTree Subtree for the function expression {@code optFunExpr}.
- * @param datasetMetaVar Meta-variable from our subtree, if any exist.
- * @param context Context required to get the type of the found variable.
- * @param indexArrayIndicators Depth indicators of index to match our unnest/assign stack to.
- * @param indexFieldNames Field names of index to match our unnest/assign stack to.
- * @param areFieldNamesInAssign True if we have an open index. False otherwise.
- */
- private static Triple<Integer, List<String>, IAType> matchAssignFieldInUnnestAssignStack(LogicalVariable assignVar,
- int assignVarIndex, IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
- LogicalVariable datasetMetaVar, IOptimizationContext context, List<Integer> indexArrayIndicators,
- List<String> indexFieldNames, boolean areFieldNamesInAssign) throws AlgebricksException {
- Triple<Integer, List<String>, IAType> resultantTriple = new Triple<>(-1, new ArrayList<>(), null);
- final int optVarIndex = subTree.getLastMatchedDataSourceVars().second;
- if (optVarIndex < 0) {
- return resultantTriple;
- }
- final ILogicalExpression optVarExpr = optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue();
- optFuncExpr.setLogicalExpr(optVarIndex, optVarExpr);
-
- // Build our assign / unnest stack. Do not include the very last assign (this is handled in the parent).
- int indexOfWorkingOp = subTree.getAssignsAndUnnests().size() - 1;
- Stack<AbstractLogicalOperator> logicalOperatorStack = new Stack<>();
- logicalOperatorStack.addAll(subTree.getAssignsAndUnnests().subList(0, indexOfWorkingOp));
- if (logicalOperatorStack.empty()) {
- return resultantTriple;
- }
-
- // Aggregate our record paths, and pair these with their respective array indexes.
- Pair<List<List<String>>, List<Integer>> unnestPairs =
- ArrayIndexUtil.unnestComplexRecordPath(indexFieldNames, indexArrayIndicators);
- AbstractLogicalOperator workingOp = null;
- List<String> fieldNameForWorkingUnnest;
- MutableInt fieldSource = new MutableInt(0);
- ARecordType workingRecordType = subTree.getRecordType();
-
- // TODO: (GLENN) Refactor this to use ArrayIndexUtil.
- // Iterate through our array index structure. We must match the depth and field names for the caller's variable
- // to qualify for an array-index optimization.
- LogicalVariable varFromParent = assignVar;
- for (int pairsIndex = 0; pairsIndex < unnestPairs.first.size(); pairsIndex++) {
- if (logicalOperatorStack.empty()) {
- return resultantTriple;
- }
- workingOp = logicalOperatorStack.pop();
-
- // Explore our UNNEST path.
- if (unnestPairs.second.get(pairsIndex) > 0) {
- for (int i = (pairsIndex == 0) ? 1 : 0; i < unnestPairs.first.get(pairsIndex).size(); i++) {
- // Match our parent assign variable to a variable used in our working assign.
- assignVarIndex = findAssignVarIndex(workingOp, varFromParent);
- if (logicalOperatorStack.empty() || assignVarIndex == -1) {
- return resultantTriple;
- }
- varFromParent = ((AssignOperator) workingOp).getVariables().get(assignVarIndex);
- indexOfWorkingOp--;
- workingOp = logicalOperatorStack.pop();
- }
-
- // Get the field name associated with the current UNNEST.
- if (workingOp.getOperatorTag() != LogicalOperatorTag.UNNEST) {
- return resultantTriple;
- }
- fieldNameForWorkingUnnest = getFieldNameFromSubTree(null, subTree, indexOfWorkingOp, assignVarIndex,
- workingRecordType, 0, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource, true);
-
- if (!fieldNameForWorkingUnnest.equals(unnestPairs.first.get(pairsIndex))) {
- return resultantTriple;
- }
- resultantTriple.second.addAll(fieldNameForWorkingUnnest);
-
- IAType typeIntermediate = workingRecordType.getSubFieldType(fieldNameForWorkingUnnest);
- for (int i = 0; i < unnestPairs.second.get(pairsIndex); i++) {
- // If we are working with a closed index, then update our record type. For open types, we do not
- // need to do this as the field name is stored in the expression itself.
- if (!areFieldNamesInAssign && pairsIndex != unnestPairs.first.size() - 1) {
- typeIntermediate = TypeComputeUtils.extractListItemType(typeIntermediate);
- if (typeIntermediate == null) {
- return resultantTriple;
- }
- }
- boolean isIntermediateUnnestInPath = (i != unnestPairs.second.get(pairsIndex) - 1);
- if (!areFieldNamesInAssign && !isIntermediateUnnestInPath) {
- if (typeIntermediate.getTypeTag().equals(ATypeTag.OBJECT)) {
- workingRecordType = (ARecordType) typeIntermediate;
- } else if (!typeIntermediate.getTypeTag().isListType()) {
- return resultantTriple;
- }
- }
-
- // Update our parent variable. If we are in-between UNNESTs, we need to fetch the next UNNEST.
- if (isIntermediateUnnestInPath) {
- workingOp = logicalOperatorStack.pop();
- indexOfWorkingOp--;
- }
- varFromParent = ((UnnestOperator) workingOp).getVariable();
- }
- } else if (pairsIndex != 0) {
- // We have explored an UNNEST array-path previously, and must now match a field name.
- AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
- indexOfWorkingOp -= unnestPairs.first.get(pairsIndex).size();
- for (assignVarIndex = 0; assignVarIndex < workingOpAsAssign.getVariables().size(); assignVarIndex++) {
- // Iterate through each of our ASSIGN's field names, and try to match the index field names.
- fieldNameForWorkingUnnest = getFieldNameFromSubTree(null, subTree, indexOfWorkingOp, assignVarIndex,
- workingRecordType, 0, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource, true);
-
- if (fieldNameForWorkingUnnest.equals(unnestPairs.first.get(pairsIndex))) {
- resultantTriple.second.addAll(fieldNameForWorkingUnnest);
- break;
- }
- }
-
- // We have exhausted all of our ASSIGN variables, but have not matched the field name. Exit early.
- if (assignVarIndex == workingOpAsAssign.getVariables().size()) {
- return resultantTriple;
- }
- }
-
- indexOfWorkingOp--;
- }
-
- // We have found an applicable array index. Determine our optFuncIndex and fieldType.
- if (workingOp != null && workingOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
- AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
- LogicalVariable matchedVar = workingOpAsAssign.getVariables().get(assignVarIndex);
- if (optFuncExpr.findLogicalVar(matchedVar) > -1) {
- resultantTriple.first = optFuncExpr.findLogicalVar(matchedVar);
- resultantTriple.third = (IAType) context.getOutputTypeEnvironment(workingOp).getVarType(matchedVar);
- optFuncExpr.setSourceVar(resultantTriple.first, matchedVar);
- }
-
- } else if (workingOp != null) {
- UnnestOperator workingOpAsUnnest = (UnnestOperator) workingOp;
- resultantTriple.first = optFuncExpr.findLogicalVar(workingOpAsUnnest.getVariable());
- resultantTriple.third =
- (IAType) context.getOutputTypeEnvironment(workingOp).getVarType(workingOpAsUnnest.getVariable());
- optFuncExpr.setSourceVar(resultantTriple.first, workingOpAsUnnest.getVariable());
- }
-
- return resultantTriple;
- }
-
- private static int findAssignVarIndex(AbstractLogicalOperator workingOp, LogicalVariable varFromParentAssign) {
- if (workingOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
- return -1;
- }
- AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
-
- // Match our parent assign variable to a variable used in our working assign.
- List<LogicalVariable> variablesUsedInWorkingAssign = new ArrayList<>();
- for (Mutable<ILogicalExpression> assignExpr : workingOpAsAssign.getExpressions()) {
- assignExpr.getValue().getUsedVariables(variablesUsedInWorkingAssign);
- int pos = variablesUsedInWorkingAssign.indexOf(varFromParentAssign);
- if (pos != -1) {
- return pos;
- }
- }
- return -1;
+ public static boolean isFieldAccess(FunctionIdentifier funId) {
+ return funId.equals(FIELD_ACCESS_BY_NAME) || funId.equals(FIELD_ACCESS_BY_INDEX)
+ || funId.equals(FIELD_ACCESS_NESTED);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
index 44e4a18..d01a2cb 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
@@ -19,22 +19,35 @@
package org.apache.asterix.optimizer.rules.am;
+import java.util.ArrayDeque;
import java.util.ArrayList;
+import java.util.Deque;
import java.util.List;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
// TODO (GLENN): Refactor the BTreeAccessMethod class and this class to extend a new "AbstractBTreeAccessMethod" class.
/**
@@ -45,8 +58,8 @@
@Override
public boolean matchAllIndexExprs(Index index) {
- // Similar to BTree "matchAllIndexExprs", we only require all expressions to be matched if this is a composite
- // key index with an unknowable field.
+ // We only require all expressions to be matched if this is a composite key index with an unknowable field.
+ // TODO (GLENN): When nulls become stored in array indexes, this should return false.
return ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
.map(e -> e.getProjectList().size()).reduce(0, Integer::sum) > 1 && hasUnknownableField(index);
}
@@ -71,12 +84,128 @@
}
@Override
+ public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
+ Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
+ OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
+ Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
+ Dataset dataset = analysisCtx.getIndexDatasetMap().get(chosenIndex);
+ OptimizableOperatorSubTree indexSubTree, probeSubTree;
+
+ // We assume that the left subtree is the outer branch and the right subtree is the inner branch. This
+ // assumption holds true since we only use an index from the right subtree. The following is just a sanity
+ // check.
+ if (rightSubTree.hasDataSourceScan()
+ && dataset.getDatasetName().equals(rightSubTree.getDataset().getDatasetName())) {
+ indexSubTree = rightSubTree;
+ probeSubTree = leftSubTree;
+ } else {
+ return false;
+ }
+
+ // TODO (GLENN): There is a bug with nested-loop joins originating from the probe. Disable this case for now.
+ Deque<ILogicalOperator> opStack = new ArrayDeque<>();
+ List<ILogicalOperator> visited = new ArrayList<>();
+ opStack.add(probeSubTree.getRoot());
+ while (!opStack.isEmpty()) {
+ ILogicalOperator workingOp = opStack.pop();
+ if (!visited.contains(workingOp)) {
+ if (workingOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN
+ || workingOp.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+ AbstractBinaryJoinOperator joinOperator = (AbstractBinaryJoinOperator) workingOp;
+ ILogicalExpression joinCondition = joinOperator.getCondition().getValue();
+ List<Mutable<ILogicalExpression>> conjuncts = new ArrayList<>();
+ if (joinCondition.splitIntoConjuncts(conjuncts)) {
+ for (Mutable<ILogicalExpression> conjunct : conjuncts) {
+ if (conjunct.getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression expr = (AbstractFunctionCallExpression) joinCondition;
+ if (expr.getFunctionIdentifier() != BuiltinFunctions.EQ) {
+ return false;
+ }
+ }
+ } else if (joinCondition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ } else {
+ AbstractFunctionCallExpression expr = (AbstractFunctionCallExpression) joinCondition;
+ if (expr.getFunctionIdentifier() != BuiltinFunctions.EQ) {
+ return false;
+ }
+ }
+ }
+ visited.add(workingOp);
+ }
+ for (Mutable<ILogicalOperator> opRef : workingOp.getInputs()) {
+ if (!visited.contains(opRef.getValue())) {
+ opStack.push(opRef.getValue());
+ }
+ }
+ }
+
+ LogicalVariable newNullPlaceHolderVar = null;
+ if (isLeftOuterJoin) {
+ // Gets a new null place holder variable that is the first field variable of the primary key from the
+ // indexSubTree's datasourceScanOp. We need this for all left outer joins, even those that do not have
+ // a special GroupBy.
+ newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+
+ // For all INNER-UNNESTs associated with the inner subtree (i.e. the index subtree) to extract the
+ // secondary keys, transform these UNNESTs to LEFT-OUTER-UNNESTs. This is to ensure that probe entries w/o
+ // a corresponding secondary key entry are not incorrectly removed. This will not invalidate our fetched
+ // entries because *all* index entries have a non-empty array.
+ ILogicalOperator workingOp = indexSubTree.getRoot(), rootOp = indexSubTree.getRoot(), previousOp = null;
+ while (!workingOp.getInputs().isEmpty()) {
+ if (workingOp.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+ UnnestOperator oldUnnest = (UnnestOperator) workingOp;
+ LeftOuterUnnestOperator newUnnest = new LeftOuterUnnestOperator(oldUnnest.getVariable(),
+ new MutableObject<>(oldUnnest.getExpressionRef().getValue()),
+ ConstantExpression.MISSING.getValue());
+ newUnnest.setSourceLocation(oldUnnest.getSourceLocation());
+ newUnnest.getInputs().addAll(oldUnnest.getInputs());
+ newUnnest.setExecutionMode(oldUnnest.getExecutionMode());
+ context.computeAndSetTypeEnvironmentForOperator(newUnnest);
+ if (workingOp.equals(rootOp)) {
+ rootOp = newUnnest;
+ workingOp = newUnnest;
+ } else if (previousOp != null) {
+ previousOp.getInputs().clear();
+ previousOp.getInputs().add(new MutableObject<>(newUnnest));
+ context.computeAndSetTypeEnvironmentForOperator(previousOp);
+ }
+ }
+ previousOp = workingOp;
+ workingOp = workingOp.getInputs().get(0).getValue();
+ }
+ indexSubTree.setRoot(rootOp);
+ indexSubTree.setRootRef(new MutableObject<>(rootOp));
+ joinOp.getInputs().remove(1);
+ joinOp.getInputs().add(1, new MutableObject<>(rootOp));
+ context.computeAndSetTypeEnvironmentForOperator(joinOp);
+ }
+
+ ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
+ indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
+ isLeftOuterJoin, true, context, newNullPlaceHolderVar, leftOuterMissingValue);
+ if (indexSearchOp == null) {
+ return false;
+ }
+
+ return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
+ analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+ indexSearchOp, newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
+ }
+
+ @Override
public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
Index.ArrayIndexDetails chosenIndexDetails = (Index.ArrayIndexDetails) chosenIndex.getIndexDetails();
List<List<String>> chosenIndexKeyFieldNames = new ArrayList<>();
@@ -93,26 +222,8 @@
return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
- newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
- chosenIndexKeyFieldSourceIndicators);
- }
-
- @Override
- protected IAType getIndexedKeyType(Index.IIndexDetails chosenIndexDetails, int keyPos) throws CompilationException {
- // TODO (GLENN): This assumes a flattened key list. Refactor / clarify this when removing depth indicators.
- Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) chosenIndexDetails;
- int elementPos = 0;
- for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
- for (int i = 0; i < e.getProjectList().size(); i++) {
- if (elementPos == keyPos) {
- return e.getTypeList().get(i);
- }
- elementPos++;
- }
- }
-
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
- "No array index element found, but using " + "an array access method.");
+ newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, chosenIndexKeyFieldNames,
+ chosenIndexKeyFieldTypes, chosenIndexKeyFieldSourceIndicators);
}
@Override
@@ -126,8 +237,21 @@
}
@Override
+ public boolean acceptsFunction(AbstractFunctionCallExpression functionExpr, Index index, IAType indexedFieldType,
+ boolean defaultNull, boolean finalStep) throws CompilationException {
+ if (defaultNull) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "CAST modifier not allowed");
+ }
+ return AccessMethodUtils.isFieldAccess(functionExpr.getFunctionIdentifier());
+ }
+
+ @Override
public int compareTo(IAccessMethod o) {
return this.getName().compareTo(o.getName());
}
+ @Override
+ protected boolean allowFunctionExpressionArg() {
+ return false;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index cad5f12..5dd0135 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -19,6 +19,8 @@
package org.apache.asterix.optimizer.rules.am;
+import static org.apache.asterix.optimizer.rules.am.AccessMethodUtils.CAST_NULL_TYPE_CONSTRUCTORS;
+
import java.util.ArrayList;
import java.util.Arrays;
import java.util.BitSet;
@@ -27,6 +29,7 @@
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -36,14 +39,21 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ComparatorUtil;
import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.IndexUtil;
+import org.apache.asterix.metadata.utils.TypeUtil;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
@@ -58,6 +68,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -117,19 +128,20 @@
List<AbstractLogicalOperator> assignsAndUnnests, AccessMethodAnalysisContext analysisCtx,
IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(funcExpr,
- analysisCtx, context, typeEnvironment);
+ analysisCtx, context, typeEnvironment, allowFunctionExpressionArg());
if (!matches) {
matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVarsAndUpdateAnalysisCtx(funcExpr, analysisCtx);
}
return matches;
}
+ protected boolean allowFunctionExpressionArg() {
+ return true;
+ }
+
@Override
public boolean matchAllIndexExprs(Index index) {
- // require all expressions to be matched if this is a composite key index which has an unknownable key field.
- // because we only add a tuple to the index if all its key fields are not null/missing.
- return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes().size() > 1
- && hasUnknownableField(index);
+ return false;
}
@Override
@@ -137,18 +149,6 @@
return !matchAllIndexExprs(index);
}
- private boolean hasUnknownableField(Index index) {
- if (index.isSecondaryIndex() && index.getIndexDetails().isOverridingKeyFieldTypes() && !index.isEnforced()) {
- return true;
- }
- for (IAType fieldType : ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes()) {
- if (NonTaggedFormatUtil.isOptional(fieldType)) {
- return true;
- }
- }
- return false;
- }
-
@Override
public boolean applySelectPlanTransformation(List<Mutable<ILogicalOperator>> afterSelectRefs,
Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree, Index chosenIndex,
@@ -204,7 +204,7 @@
afterSelectRefs),
false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
.getExecutionMode() == ExecutionMode.UNPARTITIONED,
- context, null);
+ context, null, null);
if (primaryIndexUnnestOp == null) {
return false;
@@ -257,8 +257,8 @@
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException {
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
@@ -283,12 +283,12 @@
return false;
}
- LogicalVariable newNullPlaceHolderVar = null;
+ LogicalVariable newMissingNullPlaceHolderVar = null;
if (isLeftOuterJoin) {
- // Gets a new null place holder variable that is the first field variable of the primary key
+ // Gets a new missing/null place holder variable that is the first field variable of the primary key
// from the indexSubTree's datasourceScanOp.
// We need this for all left outer joins, even those that do not have a special GroupBy
- newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+ newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
}
boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
@@ -299,15 +299,15 @@
ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
- isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+ isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
if (indexSearchOp == null) {
return false;
}
return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
- analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
- newNullPlaceHolderVar, conditionRef, dataset);
+ analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+ indexSearchOp, newMissingNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
}
/**
@@ -321,7 +321,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
Index.ValueIndexDetails chosenIndexDetails = (Index.ValueIndexDetails) chosenIndex.getIndexDetails();
List<List<String>> chosenIndexKeyFieldNames = chosenIndexDetails.getKeyFieldNames();
@@ -330,8 +331,8 @@
return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
- newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
- chosenIndexKeyFieldSourceIndicators);
+ newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, chosenIndexKeyFieldNames,
+ chosenIndexKeyFieldTypes, chosenIndexKeyFieldSourceIndicators);
}
protected ILogicalOperator createBTreeIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
@@ -339,9 +340,9 @@
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newMissingPlaceHolderForLOJ, List<List<String>> chosenIndexKeyFieldNames,
- List<IAType> chosenIndexKeyFieldTypes, List<Integer> chosenIndexKeyFieldSourceIndicators)
- throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue,
+ List<List<String>> chosenIndexKeyFieldNames, List<IAType> chosenIndexKeyFieldTypes,
+ List<Integer> chosenIndexKeyFieldSourceIndicators) throws AlgebricksException {
Dataset dataset = indexSubTree.getDataset();
ARecordType recordType = indexSubTree.getRecordType();
ARecordType metaRecordType = indexSubTree.getMetaRecordType();
@@ -382,6 +383,7 @@
boolean couldntFigureOut = false;
boolean doneWithExprs = false;
boolean isEqCondition = false;
+ boolean anyRealTypeConvertedToIntegerType = false;
BitSet setLowKeys = new BitSet(numSecondaryKeys);
BitSet setHighKeys = new BitSet(numSecondaryKeys);
// Go through the func exprs listed as optimizable by the chosen index,
@@ -409,10 +411,12 @@
// This is required because of type-casting. Refer to AccessMethodUtils.createSearchKeyExpr for details.
IAType indexedFieldType = chosenIndexKeyFieldTypes.get(keyPos);
Triple<ILogicalExpression, ILogicalExpression, Boolean> returnedSearchKeyExpr =
- AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, indexedFieldType, probeSubTree);
+ AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, indexedFieldType, probeSubTree,
+ SEARCH_KEY_ROUNDING_FUNCTION_COMPUTER);
ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
ILogicalExpression searchKeyEQExpr = null;
boolean realTypeConvertedToIntegerType = returnedSearchKeyExpr.third;
+ anyRealTypeConvertedToIntegerType |= realTypeConvertedToIntegerType;
LimitType limit = getLimitType(optFuncExpr, probeSubTree);
if (limit == null) {
@@ -426,7 +430,7 @@
}
// Deals with the non-enforced index case here.
- if (relaxLimitTypeToInclusive(chosenIndex, keyPos, realTypeConvertedToIntegerType)) {
+ if (relaxLimitTypeToInclusive(chosenIndex, indexedFieldType, realTypeConvertedToIntegerType)) {
if (limit == LimitType.HIGH_EXCLUSIVE) {
limit = LimitType.HIGH_INCLUSIVE;
} else if (limit == LimitType.LOW_EXCLUSIVE) {
@@ -674,7 +678,7 @@
// The result: SK, PK, [Optional - the result of an instantTrylock on PK]
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainMissing,
- generateInstantTrylockResultFromIndexSearch);
+ generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
// Generate the rest of the upstream plan which feeds the search results into the primary index.
ILogicalOperator indexSearchOp = null;
@@ -691,7 +695,8 @@
indexSearchOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef,
assignBeforeTheOpRefs, dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp,
context, true, retainInput, retainMissing, false, chosenIndex, analysisCtx, indexSubTree,
- probeSubTree, newMissingPlaceHolderForLOJ);
+ probeSubTree, newMissingNullPlaceHolderForLOJ, leftOuterMissingValue,
+ anyRealTypeConvertedToIntegerType);
// Replaces the datasource scan with the new plan rooted at
// Get dataSourceRef operator -
@@ -768,7 +773,7 @@
} else {
unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
- true);
+ leftOuterMissingValue);
}
unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
indexSearchOp = unnestMapOp;
@@ -781,7 +786,7 @@
} else {
unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
((LeftOuterUnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(),
- primaryIndexOutputTypes, true);
+ primaryIndexOutputTypes, leftOuterMissingValue);
}
unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
indexSearchOp = unnestMapOp;
@@ -917,8 +922,41 @@
return limit;
}
- private boolean relaxLimitTypeToInclusive(Index chosenIndex, int keyPos, boolean realTypeConvertedToIntegerType)
- throws CompilationException {
+ private static final BTreeSearchKeyRoundingFunctionProvider SEARCH_KEY_ROUNDING_FUNCTION_COMPUTER =
+ new BTreeSearchKeyRoundingFunctionProvider();
+
+ private static class BTreeSearchKeyRoundingFunctionProvider
+ extends AccessMethodUtils.SearchKeyRoundingFunctionProvider {
+
+ private final ILogicalBinaryComparator DOUBLE_CMPR =
+ ComparatorUtil.createLogicalComparator(BuiltinType.ADOUBLE, BuiltinType.ADOUBLE, false);
+
+ private static final ADouble ZERO = new ADouble(0);
+
+ @Override
+ public ATypeHierarchy.TypeCastingMathFunctionType getRoundingFunction(ComparisonKind cKind, Index chosenIndex,
+ IAType indexedFieldType, IAObject constantValue, boolean realTypeConvertedToIntegerType)
+ throws CompilationException {
+ switch (cKind) {
+ case GE:
+ return relaxLimitTypeToInclusive(chosenIndex, indexedFieldType, realTypeConvertedToIntegerType)
+ && DOUBLE_CMPR.compare(ZERO, constantValue) == ILogicalBinaryComparator.Result.LT
+ ? ATypeHierarchy.TypeCastingMathFunctionType.FLOOR
+ : ATypeHierarchy.TypeCastingMathFunctionType.CEIL;
+ case LE:
+ return relaxLimitTypeToInclusive(chosenIndex, indexedFieldType, realTypeConvertedToIntegerType)
+ && DOUBLE_CMPR.compare(ZERO, constantValue) == ILogicalBinaryComparator.Result.GT
+ ? ATypeHierarchy.TypeCastingMathFunctionType.CEIL
+ : ATypeHierarchy.TypeCastingMathFunctionType.FLOOR;
+ default:
+ return super.getRoundingFunction(cKind, chosenIndex, indexedFieldType, constantValue,
+ realTypeConvertedToIntegerType);
+ }
+ }
+ }
+
+ private static boolean relaxLimitTypeToInclusive(Index chosenIndex, IAType indexedFieldType,
+ boolean realTypeConvertedToIntegerType) {
// For a non-enforced index or an enforced index that stores a casted value on the given index,
// we need to apply the following transformation.
// For an index on a closed field, this transformation is not necessary since the value between
@@ -946,11 +984,7 @@
}
if (chosenIndex.getIndexDetails().isOverridingKeyFieldTypes() && !chosenIndex.isEnforced()) {
- IAType indexedKeyType = getIndexedKeyType(chosenIndex.getIndexDetails(), keyPos);
- if (NonTaggedFormatUtil.isOptional(indexedKeyType)) {
- indexedKeyType = ((AUnionType) indexedKeyType).getActualType();
- }
- switch (indexedKeyType.getTypeTag()) {
+ switch (TypeComputeUtils.getActualType(indexedFieldType).getTypeTag()) {
case TINYINT:
case SMALLINT:
case INTEGER:
@@ -966,17 +1000,13 @@
return false;
}
- protected IAType getIndexedKeyType(Index.IIndexDetails chosenIndexDetails, int keyPos) throws CompilationException {
- return ((Index.ValueIndexDetails) chosenIndexDetails).getKeyFieldTypes().get(keyPos);
- }
-
private boolean probeIsOnLhs(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree probeSubTree) {
if (probeSubTree == null) {
if (optFuncExpr.getConstantExpressions().length == 0) {
return optFuncExpr.getLogicalExpr(0) == null;
}
// We are optimizing a selection query. Search key is a constant. Return true if constant is on lhs.
- return optFuncExpr.getFuncExpr().getArguments().get(0) == optFuncExpr.getConstantExpr(0);
+ return optFuncExpr.getArgument(0) == optFuncExpr.getConstantExpr(0);
} else {
// We are optimizing a join query. Determine whether the feeding variable is on the lhs.
return (optFuncExpr.getOperatorSubTree(0) == null || optFuncExpr.getOperatorSubTree(0) == probeSubTree);
@@ -1048,6 +1078,30 @@
}
@Override
+ public boolean acceptsFunction(AbstractFunctionCallExpression functionExpr, Index index, IAType indexedFieldType,
+ boolean defaultNull, boolean finalStep) throws CompilationException {
+ FunctionIdentifier funId = functionExpr.getFunctionIdentifier();
+ if (!finalStep) {
+ return AccessMethodUtils.isFieldAccess(funId);
+ }
+ if (defaultNull) {
+ if (!CAST_NULL_TYPE_CONSTRUCTORS.contains(funId)) {
+ return false;
+ }
+ IAType nonNullableType = Index.getNonNullableType(indexedFieldType).first;
+ Pair<FunctionIdentifier, IAObject> constructorWithFmt =
+ IndexUtil.getTypeConstructorDefaultNull(index, nonNullableType, functionExpr.getSourceLocation());
+ FunctionIdentifier indexedFieldConstructorFun = constructorWithFmt.first;
+ IAObject formatInIndex = constructorWithFmt.second;
+ IAObject formatInFunction = TypeUtil.getTemporalFormatArg(functionExpr);
+ // index has CAST (DEFAULT NULL); the applied function should be the same as the indexed field function
+ return funId.equals(indexedFieldConstructorFun) && Objects.equals(formatInIndex, formatInFunction);
+ } else {
+ return AccessMethodUtils.isFieldAccess(funId);
+ }
+ }
+
+ @Override
public int compareTo(IAccessMethod o) {
return this.getName().compareTo(o.getName());
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index b21cf12..5688a70 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -23,6 +23,7 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.types.IAType;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -31,6 +32,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -106,7 +108,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException;
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException;
/**
* Applies the plan transformation to use chosenIndex to optimize a join query.
@@ -116,8 +119,8 @@
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException;
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException;
/**
* Analyzes expr to see whether it is optimizable by the given concrete index.
@@ -130,4 +133,17 @@
public String getName();
+ /**
+ * Checks whether the function applied to an indexed field is acceptable by the access method.
+ *
+ * @param functionExpr applied function
+ * @param index the index definition
+ * @param indexedFieldType the type of the indexed field in the index definition
+ * @param defaultNull true if the candidate index has CAST (DEFAULT NULL) modifier
+ * @param finalStep true if the functionExpr is the final function applied
+ *
+ * @return true if the access method accepts the argument function. False, otherwise.
+ */
+ public boolean acceptsFunction(AbstractFunctionCallExpression functionExpr, Index index, IAType indexedFieldType,
+ boolean defaultNull, boolean finalStep) throws AlgebricksException;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
index 6278865..ae8c8c5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
@@ -21,6 +21,7 @@
import java.util.List;
import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
@@ -80,4 +81,17 @@
void setConstantExpr(int index, ILogicalExpression expr);
ILogicalExpression[] getConstantExpressions();
+
+ void addStepExpr(int index, AbstractFunctionCallExpression funcExpr);
+
+ List<AbstractFunctionCallExpression> getStepsExprs(int index);
+
+ /**
+ * Returns the argument expression from which the logical variable is originating in the optimizable function.
+ *
+ * @param index the index of the logical variable for which to return the expression argument
+ *
+ * @return the argument expression
+ */
+ Mutable<ILogicalExpression> getArgument(int index);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 6a5964d..6f53219 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -23,10 +23,14 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.optimizer.rules.am.array.IIntroduceAccessMethodRuleLocalRewrite;
+import org.apache.asterix.optimizer.rules.am.array.JoinFromSubplanRewrite;
import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -35,6 +39,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -90,6 +95,9 @@
protected IVariableTypeEnvironment typeEnvironment = null;
protected List<Mutable<ILogicalOperator>> afterJoinRefs = null;
+ // For plan rewriting to recognize applicable array indexes.
+ private final JoinFromSubplanRewrite joinFromSubplanRewrite = new JoinFromSubplanRewrite();
+
// Registers access methods.
protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
@@ -98,6 +106,9 @@
registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+ for (Pair<FunctionIdentifier, Boolean> optFunc : BTreeAccessMethod.INSTANCE.getOptimizableFunctions()) {
+ JoinFromSubplanRewrite.addOptimizableFunction(optFunc.first);
+ }
}
/**
@@ -294,6 +305,24 @@
analyzedAMs = new HashMap<>();
}
+ if (continueCheck && context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
+ && JoinFromSubplanRewrite.isApplicableForRewriteCursory(metadataProvider, joinOp)) {
+ // If there exists a SUBPLAN in our plan, and we are conditioning on a variable, attempt to rewrite
+ // this subplan to allow an array-index AM to be introduced. If successful, this rewrite will transform
+ // into an index-nested-loop-join. This rewrite is to be used for pushing the UNNESTs and ASSIGNs from
+ // the subplan into the index branch and giving the join a condition for this rule to optimize.
+ // *No nodes* from this rewrite will be used beyond this point.
+ joinFromSubplanRewrite.findAfterSubplanSelectOperator(afterJoinRefs);
+ if (rewriteLocallyAndTransform(joinRef, context, joinFromSubplanRewrite)) {
+ // Connect the after-join operators to the index subtree root before this rewrite. This also avoids
+ // performing the secondary index validation step twice.
+ ILogicalOperator lastAfterJoinOp = afterJoinRefs.get(afterJoinRefs.size() - 1).getValue();
+ OperatorManipulationUtil.substituteOpInInput(lastAfterJoinOp, joinOp, joinOp.getInputs().get(1));
+ context.computeAndSetTypeEnvironmentForOperator(lastAfterJoinOp);
+ return true;
+ }
+ }
+
// Checks the condition of JOIN operator is a function call since only function call can be transformed
// using available indexes. If so, initializes the subtree information that will be used later to decide
// whether the given plan is truly optimizable or not.
@@ -331,12 +360,8 @@
if (continueCheck && checkRightSubTreeMetadata) {
// Map variables to the applicable indexes and find the field name and type.
// Then find the applicable indexes for the variables used in the JOIN condition.
- if (checkLeftSubTreeMetadata) {
- fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context);
- } else {
- fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context, true);
- }
- fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context);
+ fillSubTreeIndexExprs(leftSubTree, analyzedAMs, context, true, !checkLeftSubTreeMetadata);
+ fillSubTreeIndexExprs(rightSubTree, analyzedAMs, context, false);
// Prunes the access methods based on the function expression and access methods.
pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
@@ -366,6 +391,9 @@
// Applies the plan transformation using chosen index.
AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
+ IAlgebricksConstantValue leftOuterMissingValue =
+ isLeftOuterJoin ? ((LeftOuterJoinOperator) joinOp).getMissingValue() : null;
+
// For a left outer join with a special GroupBy, prepare objects to reset LOJ's
// nullPlaceHolderVariable in that GroupBy's nested plan.
// See AccessMethodUtils#removeUnjoinedDuplicatesInLOJ() for a definition of a special GroupBy
@@ -373,14 +401,16 @@
boolean isLeftOuterJoinWithSpecialGroupBy;
if (isLeftOuterJoin && op.getOperatorTag() == LogicalOperatorTag.GROUP) {
GroupByOperator groupByOp = (GroupByOperator) opRef.getValue();
- ScalarFunctionCallExpression isNullFuncExpr =
- AccessMethodUtils.findLOJIsMissingFuncInGroupBy(groupByOp, rightSubTree);
+ FunctionIdentifier isMissingNullFuncId = Objects
+ .requireNonNull(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue));
+ ScalarFunctionCallExpression isMissingNullFuncExpr = AccessMethodUtils
+ .findLOJIsMissingNullFuncInGroupBy(groupByOp, rightSubTree, isMissingNullFuncId);
// TODO:(dmitry) do we need additional checks to ensure that this is a special GroupBy,
// i.e. that this GroupBy will eliminate unjoined duplicates?
- isLeftOuterJoinWithSpecialGroupBy = isNullFuncExpr != null;
+ isLeftOuterJoinWithSpecialGroupBy = isMissingNullFuncExpr != null;
if (isLeftOuterJoinWithSpecialGroupBy) {
analysisCtx.setLOJSpecialGroupByOpRef(opRef);
- analysisCtx.setLOJIsMissingFuncInSpecialGroupBy(isNullFuncExpr);
+ analysisCtx.setLOJIsMissingNullFuncInSpecialGroupBy(isMissingNullFuncExpr);
}
} else {
isLeftOuterJoinWithSpecialGroupBy = false;
@@ -399,7 +429,7 @@
// Finally, tries to apply plan transformation using the chosen index.
boolean res = chosenIndex.first.applyJoinPlanTransformation(afterJoinRefs, joinRef, leftSubTree,
rightSubTree, chosenIndex.second, analysisCtx, context, isLeftOuterJoin,
- isLeftOuterJoinWithSpecialGroupBy);
+ isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue);
// If the plan transformation is successful, we don't need to traverse the plan
// any more, since if there are more JOIN operators, the next trigger on this plan
@@ -456,4 +486,18 @@
return false;
}
+ private boolean rewriteLocallyAndTransform(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ IIntroduceAccessMethodRuleLocalRewrite<AbstractBinaryJoinOperator> rewriter) throws AlgebricksException {
+ AbstractBinaryJoinOperator joinRewrite = rewriter.createOperator(joinOp, context);
+ boolean transformationResult = false;
+ if (joinRewrite != null) {
+ Mutable<ILogicalOperator> joinRuleInput = new MutableObject<>(joinRewrite);
+ transformationResult = checkAndApplyJoinTransformation(joinRuleInput, context);
+ }
+
+ // Restore our state, so we can look for more optimizations if this transformation failed.
+ joinOp = rewriter.restoreBeforeRewrite(afterJoinRefs, context);
+ joinRef = opRef;
+ return transformationResult;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index ab0ae5f..fc8c3e9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -497,7 +497,7 @@
|| funcIdent == AlgebricksBuiltinFunctions.LT || funcIdent == AlgebricksBuiltinFunctions.GT
|| funcIdent == AlgebricksBuiltinFunctions.EQ) {
AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(funcExpr, analysisCtx, context,
- typeEnvironment);
+ typeEnvironment, false);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
index d221fd1..cdeee0f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
@@ -44,6 +44,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
@@ -235,7 +236,8 @@
AbstractUnnestMapOperator primaryIndexUnnestOperator =
(AbstractUnnestMapOperator) AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, primaryIndex, scanOperator.getInputs().get(0).getValue(),
- newBTreeParameters, context, retainInput, false, false);
+ newBTreeParameters, context, retainInput, false, false,
+ ConstantExpression.MISSING.getValue());
// re-use the PK variables of the original scan operator
primaryIndexUnnestOperator.getVariables().clear();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index c64e517..d9b5da9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -30,7 +30,9 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Index;
-import org.apache.asterix.optimizer.rules.util.SelectInSubplanBranchCreator;
+import org.apache.asterix.optimizer.rules.am.array.IIntroduceAccessMethodRuleLocalRewrite;
+import org.apache.asterix.optimizer.rules.am.array.MergedSelectRewrite;
+import org.apache.asterix.optimizer.rules.am.array.SelectFromSubplanRewrite;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -123,16 +125,22 @@
protected IVariableTypeEnvironment typeEnvironment = null;
protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
protected List<Mutable<ILogicalOperator>> afterSelectRefs = null;
- private final SelectInSubplanBranchCreator selectInSubplanBranchCreator = new SelectInSubplanBranchCreator();
+
+ // For plan rewriting to recognize applicable array indexes.
+ private final SelectFromSubplanRewrite selectFromSubplanRewrite = new SelectFromSubplanRewrite();
+ private final MergedSelectRewrite mergedSelectRewrite = new MergedSelectRewrite();
// Register access methods.
protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
static {
- registerAccessMethod(ArrayBTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+ registerAccessMethod(ArrayBTreeAccessMethod.INSTANCE, accessMethods);
+ for (Pair<FunctionIdentifier, Boolean> f : ArrayBTreeAccessMethod.INSTANCE.getOptimizableFunctions()) {
+ SelectFromSubplanRewrite.addOptimizableFunction(f.first);
+ }
}
/**
@@ -230,7 +238,7 @@
.getExecutionMode() == ExecutionMode.UNPARTITIONED;
ILogicalOperator subRoot = pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx, retainInput, false,
- requiresBroadcast, context, null);
+ requiresBroadcast, context, null, null);
if (subRoot == null) {
return false;
}
@@ -373,22 +381,20 @@
analyzedAMs = new TreeMap<>();
}
- // If there exists a SUBPLAN in our plan, and we are conditioning on a variable,
- // attempt to rewrite this subplan to allow an array-index AM to be introduced.
- // This rewrite is to be used **solely** for the purpose of changing a DATA-SCAN into a
- // non-index-only plan branch. No nodes from this rewrite will be used beyond this point.
- // If successful, this will create a non-index only plan that replaces the subplan's
- // DATA-SCAN with a PIDX SEARCH <- DISTINCT <- ORDER <- SIDX SEARCH.
- if (continueCheck && context.getPhysicalOptimizationConfig().isArrayIndexEnabled()) {
- SelectOperator selectRewrite = selectInSubplanBranchCreator.createSelect(selectOp, context);
- if (selectRewrite != null
- && checkAndApplyTheSelectTransformation(new MutableObject<>(selectRewrite), context)) {
+ if (continueCheck && context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
+ && SelectFromSubplanRewrite.isApplicableForRewriteCursory(metadataProvider, selectOp)) {
+ // If there exists a composite atomic-array index, our conjuncts will be split across multiple
+ // SELECTs. This rewrite is to be used **solely** for the purpose of changing a DATA-SCAN into a
+ // non-index-only plan branch. No nodes introduced from this rewrite will be used beyond this point.
+ if (rewriteLocallyAndTransform(selectRef, context, mergedSelectRewrite)) {
return true;
+ }
- } else {
- // If this optimization or temp-branch creation was not successful, restore our state.
- selectRef = selectRefFromThisOp;
- selectOp = selectInSubplanBranchCreator.getOriginalSelect();
+ // If there exists a SUBPLAN in our plan, and we are conditioning on a variable, attempt to rewrite
+ // this subplan to allow an array-index AM to be introduced. Again, this rewrite is to be used
+ // **solely** for the purpose of changing a DATA-SCAN into a non-index-only plan branch.
+ if (rewriteLocallyAndTransform(selectRef, context, selectFromSubplanRewrite)) {
+ return true;
}
}
@@ -417,7 +423,7 @@
if (continueCheck) {
// Map variables to the applicable indexes and find the field name and type.
// Then find the applicable indexes for the variables used in the SELECT condition.
- fillSubTreeIndexExprs(subTree, analyzedAMs, context);
+ fillSubTreeIndexExprs(subTree, analyzedAMs, context, false);
// Prune the access methods based on the function expression and access methods.
pruneIndexCandidates(analyzedAMs, context, typeEnvironment);
@@ -485,6 +491,21 @@
return accessMethods;
}
+ private boolean rewriteLocallyAndTransform(Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ IIntroduceAccessMethodRuleLocalRewrite<SelectOperator> rewriter) throws AlgebricksException {
+ SelectOperator selectRewrite = rewriter.createOperator(selectOp, context);
+ boolean transformationResult = false;
+ if (selectRewrite != null) {
+ Mutable<ILogicalOperator> selectRuleInput = new MutableObject<>(selectRewrite);
+ transformationResult = checkAndApplyTheSelectTransformation(selectRuleInput, context);
+ }
+
+ // Restore our state, so we can look for more optimizations if this transformation failed.
+ selectOp = rewriter.restoreBeforeRewrite(null, null);
+ selectRef = opRef;
+ return transformationResult;
+ }
+
private void clear() {
afterSelectRefs = null;
selectRef = null;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 8530dee..62812aa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -65,6 +65,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
@@ -149,7 +150,7 @@
|| funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS
|| funcExpr.getFunctionIdentifier() == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(funcExpr,
- analysisCtx, context, typeEnvironment);
+ analysisCtx, context, typeEnvironment, false);
if (!matches) {
matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVarsAndUpdateAnalysisCtx(funcExpr, analysisCtx);
}
@@ -279,9 +280,10 @@
if (fieldVarExpr2 == null) {
return false;
}
- OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr,
- new LogicalVariable[] { fieldVarExpr1, fieldVarExpr2 }, new ILogicalExpression[] { arg3 },
- new IAType[] { (IAType) ExpressionTypeComputer.INSTANCE.getType(arg3, null, null) });
+ OptimizableFuncExpr newOptFuncExpr =
+ new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVarExpr1, fieldVarExpr2 },
+ new int[] { 0, 1 }, new ILogicalExpression[] { arg3 },
+ new IAType[] { (IAType) ExpressionTypeComputer.INSTANCE.getType(arg3, null, null) });
for (IOptimizableFuncExpr optFuncExpr : analysisCtx.getMatchedFuncExprs()) {
//avoid additional optFuncExpressions in case of a join
if (optFuncExpr.getFuncExpr().equals(funcExpr)) {
@@ -306,6 +308,7 @@
// Determine whether one arg is constant, and the other is non-constant.
ILogicalExpression constArg;
ILogicalExpression nonConstArg;
+ int nonConstArgIdx;
if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
&& arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
// The arguments of edit-distance-contains() function are asymmetrical, we can only use index if it is on
@@ -315,10 +318,12 @@
}
constArg = arg1;
nonConstArg = arg2;
+ nonConstArgIdx = 1;
} else if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
&& arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
constArg = arg2;
nonConstArg = arg1;
+ nonConstArgIdx = 0;
} else {
return false;
}
@@ -329,7 +334,7 @@
}
OptimizableFuncExpr newOptFuncExpr = new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVarExpr },
- new ILogicalExpression[] { constArg, arg3 },
+ new int[] { nonConstArgIdx }, new ILogicalExpression[] { constArg, arg3 },
new IAType[] { (IAType) ExpressionTypeComputer.INSTANCE.getType(constArg, null, null),
(IAType) ExpressionTypeComputer.INSTANCE.getType(arg3, null, null) });
for (IOptimizableFuncExpr optFuncExpr : analysisCtx.getMatchedFuncExprs()) {
@@ -396,7 +401,8 @@
List<Mutable<ILogicalOperator>> assignBeforeTopOpRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
// TODO: we currently do not support the index-only plan for the inverted index searches since
// there can be many <SK, PK> pairs for the same PK and we may see two different records with the same PK
// (e.g., the record is deleted and inserted with the same PK). The reason is that there are
@@ -449,13 +455,13 @@
// since it doesn't contain a field value, only part of it.
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainNull,
- generateInstantTrylockResultFromIndexSearch);
+ generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
// Generates the rest of the upstream plan which feeds the search results into the primary index.
ILogicalOperator primaryIndexUnnestOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef,
conditionRef, assignBeforeTopOpRefs, dataSourceScan, dataset, recordType, metaRecordType,
secondaryIndexUnnestOp, context, true, retainInput, retainNull, false, chosenIndex, analysisCtx,
- indexSubTree, null, newNullPlaceHolderForLOJ);
+ indexSubTree, null, newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, false);
return primaryIndexUnnestOp;
}
@@ -483,10 +489,10 @@
SelectOperator selectOp = (SelectOperator) selectRef.getValue();
ILogicalOperator indexPlanRootOp =
createIndexSearchPlan(afterSelectRefs, selectRef, selectOp.getCondition(),
- subTree.getAssignsAndUnnestsRefs(),
- subTree, null, chosenIndex, analysisCtx, false, false, subTree.getDataSourceRef().getValue()
- .getInputs().get(0).getValue().getExecutionMode() == ExecutionMode.UNPARTITIONED,
- context, null);
+ subTree.getAssignsAndUnnestsRefs(), subTree,
+ null, chosenIndex, analysisCtx, false, false, subTree.getDataSourceRef().getValue().getInputs()
+ .get(0).getValue().getExecutionMode() == ExecutionMode.UNPARTITIONED,
+ context, null, null);
// Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
subTree.getDataSourceRef().setValue(indexPlanRootOp);
@@ -497,8 +503,8 @@
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException {
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
OptimizableOperatorSubTree indexSubTree;
OptimizableOperatorSubTree probeSubTree;
@@ -524,16 +530,17 @@
}
//if LOJ, reset null place holder variable
- LogicalVariable newNullPlaceHolderVar = null;
+ LogicalVariable newMissingNullPlaceHolderVar = null;
if (isLeftOuterJoin) {
//get a new null place holder variable that is the first field variable of the primary key
//from the indexSubTree's datasourceScanOp
// We need this for all left outer joins, even those that do not have a special GroupBy
- newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+ newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
if (isLeftOuterJoinWithSpecialGroupBy) {
//reset the null place holder variable
- AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+ AccessMethodUtils.resetLOJMissingNullPlaceholderVarInGroupByOp(analysisCtx,
+ newMissingNullPlaceHolderVar, context);
}
}
@@ -569,14 +576,16 @@
probeSubTree.setRoot(newProbeRootRef.getValue());
}
// Create regular indexed-nested loop join path.
- ILogicalOperator indexPlanRootOp = createIndexSearchPlan(afterJoinRefs, joinRef,
- new MutableObject<ILogicalExpression>(joinCond), indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree,
- probeSubTree, chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+ ILogicalOperator indexPlanRootOp =
+ createIndexSearchPlan(afterJoinRefs, joinRef, new MutableObject<ILogicalExpression>(joinCond),
+ indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx,
+ true, isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
indexSubTree.getDataSourceRef().setValue(indexPlanRootOp);
// Change join into a select with the same condition.
- SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond), isLeftOuterJoin,
- newNullPlaceHolderVar);
+ SelectOperator topSelect = isLeftOuterJoin
+ ? new SelectOperator(new MutableObject<>(joinCond), leftOuterMissingValue, newMissingNullPlaceHolderVar)
+ : new SelectOperator(new MutableObject<>(joinCond));
topSelect.setSourceLocation(indexPlanRootOp.getSourceLocation());
topSelect.getInputs().add(indexSubTree.getRootRef());
topSelect.setExecutionMode(ExecutionMode.LOCAL);
@@ -828,7 +837,7 @@
}
SelectOperator isFilterableSelectOp =
- new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr), false, null);
+ new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr));
isFilterableSelectOp.setSourceLocation(sourceLoc);
isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
@@ -841,7 +850,7 @@
FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), isNotFilterableArgs);
isNotFilterableExpr.setSourceLocation(sourceLoc);
SelectOperator isNotFilterableSelectOp =
- new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr), false, null);
+ new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr));
isNotFilterableSelectOp.setSourceLocation(sourceLoc);
isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
@@ -1088,7 +1097,7 @@
if (targetVar == null) {
continue;
}
- return isJaccardFuncCompatible(optFuncExpr.getFuncExpr().getArguments().get(i).getValue(),
+ return isJaccardFuncCompatible(optFuncExpr.getArgument(i).getValue(),
optFuncExpr.getFieldType(i).getTypeTag(), index.getIndexType());
}
@@ -1346,6 +1355,18 @@
}
@Override
+ public boolean acceptsFunction(AbstractFunctionCallExpression functionExpr, Index index, IAType indexedFieldType,
+ boolean defaultNull, boolean finalStep) throws CompilationException {
+ if (defaultNull) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "CAST modifier not allowed");
+ }
+ FunctionIdentifier funId = functionExpr.getFunctionIdentifier();
+ return AccessMethodUtils.isFieldAccess(funId) || funId.equals(BuiltinFunctions.GRAM_TOKENS)
+ || funId.equals(BuiltinFunctions.WORD_TOKENS) || funId.equals(BuiltinFunctions.SUBSTRING)
+ || funId.equals(BuiltinFunctions.SUBSTRING_BEFORE) || funId.equals(BuiltinFunctions.SUBSTRING_AFTER);
+ }
+
+ @Override
public int compareTo(IAccessMethod o) {
return this.getName().compareTo(o.getName());
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java
index 7b04340..9c3ba68 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableFuncExpr.java
@@ -23,6 +23,7 @@
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
@@ -35,7 +36,9 @@
protected final AbstractFunctionCallExpression funcExpr;
protected final LogicalVariable[] logicalVars;
protected final LogicalVariable[] sourceVars;
+ protected final int[] logicalVarsExprsIndexes;
protected final ILogicalExpression[] logicalExprs;
+ protected final List<List<AbstractFunctionCallExpression>> stepsExprs; // all the transformations of a logical var
protected final List<List<String>> fieldNames;
protected final int[] fieldSources;
protected final IAType[] fieldTypes;
@@ -45,17 +48,22 @@
protected boolean partialField;
public OptimizableFuncExpr(AbstractFunctionCallExpression funcExpr, LogicalVariable[] logicalVars,
- ILogicalExpression[] constantExpressions, IAType[] constantExpressionTypes) {
+ int[] logicalVarsExprsIndexes, ILogicalExpression[] constantExpressions, IAType[] constantExpressionTypes) {
this.funcExpr = funcExpr;
this.logicalVars = logicalVars;
+ this.logicalVarsExprsIndexes = logicalVarsExprsIndexes;
this.sourceVars = new LogicalVariable[logicalVars.length];
this.logicalExprs = new ILogicalExpression[logicalVars.length];
this.constantExpressionTypes = constantExpressionTypes;
this.constantExpressions = constantExpressions;
this.fieldSources = new int[logicalVars.length];
- this.fieldNames = new ArrayList<List<String>>();
+ this.fieldNames = new ArrayList<>();
for (int i = 0; i < logicalVars.length; i++) {
- fieldNames.add(new ArrayList<String>());
+ fieldNames.add(new ArrayList<>());
+ }
+ this.stepsExprs = new ArrayList<>();
+ for (int i = 0; i < logicalVars.length; i++) {
+ stepsExprs.add(new ArrayList<>());
}
this.fieldTypes = new IAType[logicalVars.length];
this.subTrees = new OptimizableOperatorSubTree[logicalVars.length];
@@ -69,7 +77,7 @@
// Special, more convenient c'tor for simple binary functions.
public OptimizableFuncExpr(AbstractFunctionCallExpression funcExpr, LogicalVariable logicalVar,
- ILogicalExpression constantExpression, IAType constantExpressionType) {
+ ILogicalExpression constantExpression, IAType constantExpressionType, int varIndexInOptFunExpr) {
this.funcExpr = funcExpr;
this.logicalVars = new LogicalVariable[] { logicalVar };
this.sourceVars = new LogicalVariable[1];
@@ -77,10 +85,15 @@
this.constantExpressions = new ILogicalExpression[] { constantExpression };
this.constantExpressionTypes = new IAType[] { constantExpressionType };
this.fieldSources = new int[logicalVars.length];
- this.fieldNames = new ArrayList<List<String>>();
+ this.fieldNames = new ArrayList<>();
for (int i = 0; i < logicalVars.length; i++) {
- fieldNames.add(new ArrayList<String>());
+ fieldNames.add(new ArrayList<>());
}
+ this.stepsExprs = new ArrayList<>();
+ for (int i = 0; i < logicalVars.length; i++) {
+ stepsExprs.add(new ArrayList<>());
+ }
+ this.logicalVarsExprsIndexes = new int[] { varIndexInOptFunExpr };
this.fieldTypes = new IAType[logicalVars.length];
this.subTrees = new OptimizableOperatorSubTree[logicalVars.length];
if (funcExpr.getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CONTAINS) {
@@ -157,6 +170,21 @@
}
@Override
+ public void addStepExpr(int index, AbstractFunctionCallExpression funcExpr) {
+ stepsExprs.get(index).add(funcExpr);
+ }
+
+ @Override
+ public List<AbstractFunctionCallExpression> getStepsExprs(int index) {
+ return stepsExprs.get(index);
+ }
+
+ @Override
+ public Mutable<ILogicalExpression> getArgument(int index) {
+ return funcExpr.getArguments().get(logicalVarsExprsIndexes[index]);
+ }
+
+ @Override
public void setConstType(int index, IAType fieldType) {
constantExpressionTypes[index] = fieldType;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index d45e7f2..c55edf9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -431,6 +431,7 @@
}
break;
case EXTERNAL_SCAN:
+ case COLLECTION_SCAN:
break;
case NO_DATASOURCE:
default:
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index 09575e2..6959543 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -27,6 +27,8 @@
import org.apache.asterix.common.annotations.SecondaryIndexSearchPreferenceAnnotation;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
@@ -42,6 +44,7 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Quadruple;
+import org.apache.hyracks.algebricks.common.utils.Triple;
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;
@@ -50,6 +53,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -97,7 +101,7 @@
List<AbstractLogicalOperator> assignsAndUnnests, AccessMethodAnalysisContext analysisCtx,
IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVarAndUpdateAnalysisCtx(funcExpr,
- analysisCtx, context, typeEnvironment);
+ analysisCtx, context, typeEnvironment, false);
if (!matches) {
matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVarsAndUpdateAnalysisCtx(funcExpr, analysisCtx);
}
@@ -127,8 +131,8 @@
IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(chosenIndex, analysisCtx);
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFuncExpr.getFieldType(optFieldIdx),
- optFuncExpr.getFieldName(optFieldIdx), recordType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(chosenIndex,
+ optFuncExpr.getFieldType(optFieldIdx), optFuncExpr.getFieldName(optFieldIdx), recordType);
if (keyPairType == null) {
return false;
}
@@ -188,11 +192,12 @@
analysisCtx.setIndexOnlyPlanInfo(indexOnlyPlanInfo);
- ILogicalOperator primaryIndexUnnestOp = createIndexSearchPlan(afterSelectRefs, selectRef,
- selectOp.getCondition(), subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
- AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(), subTree.getDataSourceRef().getValue(),
- afterSelectRefs),
- false, false, context, null);
+ ILogicalOperator primaryIndexUnnestOp =
+ createIndexSearchPlan(afterSelectRefs, selectRef, selectOp.getCondition(),
+ subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
+ AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
+ subTree.getDataSourceRef().getValue(), afterSelectRefs),
+ false, false, context, null, null);
if (primaryIndexUnnestOp == null) {
return false;
@@ -212,13 +217,17 @@
return true;
}
+ private static final AccessMethodUtils.SearchKeyRoundingFunctionProvider SEARCH_KEY_ROUNDING_FUNCTION_PROVIDER =
+ new AccessMethodUtils.SearchKeyRoundingFunctionProvider();
+
@Override
public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopRefs,
Mutable<ILogicalOperator> topRef, Mutable<ILogicalExpression> conditionRef,
List<Mutable<ILogicalOperator>> assignBeforeTopRefs, OptimizableOperatorSubTree indexSubTree,
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
- LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+ LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
// TODO: We can probably do something smarter here based on selectivity or MBR area.
IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
@@ -229,7 +238,8 @@
int optFieldIdx = AccessMethodUtils.chooseFirstOptFuncVar(chosenIndex, analysisCtx);
IAType optFieldType = optFuncExpr.getFieldType(optFieldIdx);
List<String> optFieldName = optFuncExpr.getFieldName(optFieldIdx);
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(optFieldType, optFieldName, recordType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(chosenIndex, optFieldType, optFieldName, recordType);
if (keyPairType == null) {
return null;
}
@@ -257,8 +267,9 @@
ArrayList<LogicalVariable> keyVarList = new ArrayList<>();
// List of expressions for the assign.
ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<>();
- ILogicalExpression returnedSearchKeyExpr =
- AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, optFieldType, probeSubTree).first;
+ Triple<ILogicalExpression, ILogicalExpression, Boolean> returnedSearchKeyExpr =
+ AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, optFieldType, probeSubTree,
+ SEARCH_KEY_ROUNDING_FUNCTION_PROVIDER);
for (int i = 0; i < numSecondaryKeys; i++) {
// The create MBR function "extracts" one field of an MBR around the given spatial object.
@@ -266,7 +277,7 @@
new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
createMBR.setSourceLocation(optFuncExpr.getFuncExpr().getSourceLocation());
// Spatial object is the constant from the func expr we are optimizing.
- createMBR.getArguments().add(new MutableObject<>(returnedSearchKeyExpr.cloneExpression()));
+ createMBR.getArguments().add(new MutableObject<>(returnedSearchKeyExpr.first.cloneExpression()));
// The number of dimensions
createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
new ConstantExpression(new AsterixConstantValue(new AInt32(numDimensions)))));
@@ -300,7 +311,7 @@
ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
metaRecordType, chosenIndex, assignSearchKeys, jobGenParams, context, retainInput, retainNull,
- generateInstantTrylockResultFromIndexSearch);
+ generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
// Generates the rest of the upstream plan which feeds the search results into the primary index.
return dataset.getDatasetType() == DatasetType.EXTERNAL
@@ -309,15 +320,15 @@
: AccessMethodUtils.createRestOfIndexSearchPlan(afterTopRefs, topRef, conditionRef, assignBeforeTopRefs,
dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp, context, true,
retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree, null,
- newNullPlaceHolderForLOJ);
+ newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, returnedSearchKeyExpr.third);
}
@Override
public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
- IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
- throws AlgebricksException {
+ IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
@@ -342,12 +353,12 @@
return false;
}
- LogicalVariable newNullPlaceHolderVar = null;
+ LogicalVariable newMissingNullPlaceHolderVar = null;
if (isLeftOuterJoin) {
- // Gets a new null place holder variable that is the first field variable of the primary key
+ // Gets a new missing (or null) place holder variable that is the first field variable of the primary key
// from the indexSubTree's datasourceScanOp.
// We need this for all left outer joins, even those that do not have a special GroupBy
- newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+ newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
}
boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
@@ -358,15 +369,15 @@
ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
- isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+ isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
if (indexSearchOp == null) {
return false;
}
return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
- analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
- newNullPlaceHolderVar, conditionRef, dataset);
+ analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+ indexSearchOp, newMissingNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
}
@Override
@@ -395,6 +406,15 @@
}
@Override
+ public boolean acceptsFunction(AbstractFunctionCallExpression functionExpr, Index index, IAType indexedFieldType,
+ boolean defaultNull, boolean finalStep) throws CompilationException {
+ if (defaultNull) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "CAST modifier not allowed");
+ }
+ return AccessMethodUtils.isFieldAccess(functionExpr.getFunctionIdentifier());
+ }
+
+ @Override
public int compareTo(IAccessMethod o) {
return this.getName().compareTo(o.getName());
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
new file mode 100644
index 0000000..ba70aff
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
@@ -0,0 +1,518 @@
+/*
+ * 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.am.array;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.optimizer.base.AnalysisUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+abstract public class AbstractOperatorFromSubplanRewrite<T> implements IIntroduceAccessMethodRuleLocalRewrite<T> {
+ private final static List<IAlgebricksConstantValue> ZEROS_AS_ASTERIX_CONSTANTS =
+ Arrays.asList(new IAlgebricksConstantValue[] { new AsterixConstantValue(new AInt64(0)),
+ new AsterixConstantValue(new AInt32(0)), new AsterixConstantValue(new AInt16((short) 0)),
+ new AsterixConstantValue(new AInt8((byte) 0)) });
+
+ private Set<FunctionIdentifier> optimizableFunctions;
+ private IOptimizationContext context;
+ private SourceLocation sourceLocation;
+
+ public static boolean isApplicableForRewriteCursory(MetadataProvider metadataProvider, ILogicalOperator workingOp)
+ throws AlgebricksException {
+ boolean isApplicableForRewrite = false;
+ if (workingOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator dataSourceScanOperator = (DataSourceScanOperator) workingOp;
+ Pair<DataverseName, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScanOperator);
+ DataverseName dataverseName = datasetInfo.first;
+ String datasetName = datasetInfo.second;
+ if (metadataProvider.getDatasetIndexes(dataverseName, datasetName).stream()
+ .anyMatch(i -> i.getIndexType() == DatasetConfig.IndexType.ARRAY)) {
+ return true;
+ }
+ }
+
+ for (Mutable<ILogicalOperator> inputOp : workingOp.getInputs()) {
+ isApplicableForRewrite |= isApplicableForRewriteCursory(metadataProvider, inputOp.getValue());
+ }
+ return isApplicableForRewrite;
+ }
+
+ protected void reset(SourceLocation sourceLocation, IOptimizationContext context,
+ Set<FunctionIdentifier> optimizableFunctions) {
+ this.optimizableFunctions = optimizableFunctions;
+ this.sourceLocation = sourceLocation;
+ this.context = context;
+ }
+
+ protected LogicalVariable getConditioningVariable(ILogicalExpression condition) {
+ List<Mutable<ILogicalExpression>> selectConjuncts = new ArrayList<>();
+ if (splitIntoConjuncts(condition, selectConjuncts)) {
+ for (Mutable<ILogicalExpression> conjunct : selectConjuncts) {
+ if (conjunct.getValue().getExpressionTag().equals(LogicalExpressionTag.VARIABLE)) {
+ return ((VariableReferenceExpression) conjunct.getValue()).getVariableReference();
+ }
+ }
+
+ } else if (condition.getExpressionTag().equals(LogicalExpressionTag.VARIABLE)) {
+ return ((VariableReferenceExpression) condition).getVariableReference();
+
+ }
+ return null;
+ }
+
+ protected Pair<SelectOperator, UnnestOperator> traverseSubplanBranch(SubplanOperator subplanOperator,
+ ILogicalOperator parentInput, boolean isConnectInput) throws AlgebricksException {
+ AggregateOperator workingSubplanRootAsAggregate = getAggregateFromSubplan(subplanOperator);
+ if (workingSubplanRootAsAggregate == null) {
+ return null;
+ }
+
+ // Find (or create, in the SOME AND EVERY case) a SELECT that we can potentially optimize.
+ SelectOperator optimizableSelect = getSelectFromPlan(workingSubplanRootAsAggregate);
+ if (optimizableSelect == null) {
+ return null;
+ }
+
+ // Ensure that this SELECT represents a predicate for an existential query, and is a query we can optimize.
+ ILogicalExpression normalizedSelectCondition =
+ normalizeCondition(workingSubplanRootAsAggregate, optimizableSelect.getCondition().getValue());
+ normalizedSelectCondition = keepOptimizableFunctions(normalizedSelectCondition).cloneExpression();
+
+ // Create a copy of this SELECT, and set this to our rewrite root.
+ SelectOperator rewriteRootSelect = new SelectOperator(new MutableObject<>(normalizedSelectCondition),
+ optimizableSelect.getRetainMissingAsValue(), optimizableSelect.getMissingPlaceholderVariable());
+ rewriteRootSelect.setSourceLocation(sourceLocation);
+ rewriteRootSelect.setExecutionMode(optimizableSelect.getExecutionMode());
+
+ // Follow this SELECT to the root of our nested-plan branch (i.e. the NESTED-TUPLE-SOURCE).
+ ILogicalOperator workingNewOperator = rewriteRootSelect;
+ UnnestOperator bottommostNewUnnest = null;
+ ILogicalOperator workingOriginalOperator = optimizableSelect.getInputs().get(0).getValue();
+ while (!workingOriginalOperator.getOperatorTag().equals(LogicalOperatorTag.NESTEDTUPLESOURCE)) {
+ if (workingOriginalOperator.getInputs().isEmpty()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ workingSubplanRootAsAggregate.getSourceLocation(),
+ "NESTED-TUPLE-SOURCE expected in nested plan branch, but not found.");
+ }
+
+ ScalarFunctionCallExpression updatedSelectCond;
+ SelectOperator updatedSelectOperator;
+ switch (workingOriginalOperator.getOperatorTag()) {
+ case UNNEST:
+ UnnestOperator originalUnnest = (UnnestOperator) workingOriginalOperator;
+ UnnestOperator newUnnest =
+ new UnnestOperator(originalUnnest.getVariable(), originalUnnest.getExpressionRef());
+ newUnnest.setSourceLocation(sourceLocation);
+ workingNewOperator.getInputs().add(new MutableObject<>(newUnnest));
+ workingNewOperator = newUnnest;
+ bottommostNewUnnest = (UnnestOperator) workingNewOperator;
+ break;
+
+ case ASSIGN:
+ AssignOperator originalAssign = (AssignOperator) workingOriginalOperator;
+ AssignOperator newAssign =
+ new AssignOperator(originalAssign.getVariables(), originalAssign.getExpressions());
+ newAssign.setSourceLocation(sourceLocation);
+ workingNewOperator.getInputs().add(new MutableObject<>(newAssign));
+ workingNewOperator = newAssign;
+ break;
+
+ case SELECT:
+ // If we encounter another SELECT, then we have multiple quantifiers. Transform our new SELECT to
+ // include this condition.
+ updatedSelectCond = coalesceConditions(rewriteRootSelect, workingOriginalOperator);
+ updatedSelectOperator = new SelectOperator(new MutableObject<>(updatedSelectCond),
+ rewriteRootSelect.getRetainMissingAsValue(),
+ rewriteRootSelect.getMissingPlaceholderVariable());
+ updatedSelectOperator.setSourceLocation(sourceLocation);
+ updatedSelectOperator.getInputs().addAll(rewriteRootSelect.getInputs());
+ rewriteRootSelect = updatedSelectOperator;
+ if (workingNewOperator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ workingNewOperator = rewriteRootSelect;
+ }
+ break;
+
+ case SUBPLAN:
+ // If we encounter another subplan, then we must look to include any SELECTs from here as well.
+ Pair<SelectOperator, UnnestOperator> traversalOutput =
+ traverseSubplanBranch((SubplanOperator) workingOriginalOperator, optimizableSelect, false);
+ if (traversalOutput != null) {
+ updatedSelectCond = coalesceConditions(rewriteRootSelect, traversalOutput.first);
+ updatedSelectOperator = new SelectOperator(new MutableObject<>(updatedSelectCond),
+ rewriteRootSelect.getRetainMissingAsValue(),
+ rewriteRootSelect.getMissingPlaceholderVariable());
+ updatedSelectOperator.setSourceLocation(sourceLocation);
+ updatedSelectOperator.getInputs().addAll(rewriteRootSelect.getInputs());
+ rewriteRootSelect = updatedSelectOperator;
+ if (workingNewOperator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ workingNewOperator = rewriteRootSelect;
+ }
+
+ // Add the inputs from our subplan.
+ Mutable<ILogicalOperator> traversalOperator = traversalOutput.first.getInputs().get(0);
+ while (traversalOperator != null) {
+ workingNewOperator.getInputs().add(new MutableObject<>(
+ OperatorManipulationUtil.deepCopy(traversalOperator.getValue())));
+ workingNewOperator = workingNewOperator.getInputs().get(0).getValue();
+ traversalOperator = (traversalOperator.getValue().getInputs().isEmpty()) ? null
+ : traversalOperator.getValue().getInputs().get(0);
+ }
+ workingNewOperator.getInputs().clear();
+ bottommostNewUnnest = traversalOutput.second;
+ break;
+ }
+
+ default:
+ return null;
+ }
+
+ workingOriginalOperator = workingOriginalOperator.getInputs().get(0).getValue();
+ }
+
+ // Sanity check: we should always be working with an UNNEST at this stage.
+ if (bottommostNewUnnest == null) {
+ return null;
+ }
+
+ // If we are working with strict universal quantification, then we must also check whether we have a
+ // conjunct that asserts that the array should also be non-empty.
+ if (isStrictUniversalQuantification(workingSubplanRootAsAggregate)
+ && isArrayNonEmptyConjunctMissing(bottommostNewUnnest, subplanOperator.getInputs().get(0).getValue())
+ && (parentInput == null || isArrayNonEmptyConjunctMissing(bottommostNewUnnest, parentInput))) {
+ return null;
+ }
+
+ // We have added everything we need in our nested-plan branch. Now, connect the input of our SUBPLAN to our
+ // current working branch.
+ if (isConnectInput) {
+ bottommostNewUnnest.getInputs().addAll(subplanOperator.getInputs());
+ OperatorManipulationUtil.computeTypeEnvironmentBottomUp(rewriteRootSelect, context);
+ }
+
+ return new Pair<>(rewriteRootSelect, bottommostNewUnnest);
+ }
+
+ protected ScalarFunctionCallExpression coalesceConditions(SelectOperator selectOp, ILogicalOperator auxOp) {
+ ScalarFunctionCallExpression combinedCondition =
+ new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND));
+ combinedCondition.setSourceLocation(selectOp.getSourceLocation());
+
+ List<Mutable<ILogicalExpression>> conjuncts = new ArrayList<>();
+ if (splitIntoConjuncts(selectOp.getCondition().getValue(), conjuncts)) {
+ combinedCondition.getArguments().addAll(conjuncts);
+ conjuncts.clear();
+ } else {
+ combinedCondition.getArguments().add(selectOp.getCondition());
+ }
+
+ switch (auxOp.getOperatorTag()) {
+ case LEFTOUTERJOIN:
+ case INNERJOIN:
+ AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) auxOp;
+ if (splitIntoConjuncts(joinOp.getCondition().getValue(), conjuncts)) {
+ combinedCondition.getArguments().addAll(conjuncts);
+ } else {
+ combinedCondition.getArguments().add(joinOp.getCondition());
+ }
+ break;
+
+ case SELECT:
+ SelectOperator selectOp2 = (SelectOperator) auxOp;
+ if (splitIntoConjuncts(selectOp2.getCondition().getValue(), conjuncts)) {
+ combinedCondition.getArguments().addAll(conjuncts);
+ } else {
+ combinedCondition.getArguments().add(selectOp2.getCondition());
+ }
+ break;
+ }
+
+ return combinedCondition;
+ }
+
+ private SelectOperator getSelectFromPlan(AggregateOperator subplanRoot) {
+ ILogicalExpression aggregateCondition = null;
+ boolean isNonEmptyStream = false;
+ for (Mutable<ILogicalExpression> expression : subplanRoot.getExpressions()) {
+ AggregateFunctionCallExpression aggExpression = (AggregateFunctionCallExpression) expression.getValue();
+ if (aggExpression.getFunctionIdentifier().equals(BuiltinFunctions.NON_EMPTY_STREAM)) {
+ isNonEmptyStream = true;
+
+ } else if (aggExpression.isTwoStep()
+ && aggExpression.getStepOneAggregate().getFunctionIdentifier().equals(BuiltinFunctions.SQL_COUNT)
+ && aggExpression.getStepTwoAggregate().getFunctionIdentifier().equals(BuiltinFunctions.SQL_SUM)
+ && aggExpression.getArguments().get(0).getValue().getExpressionTag()
+ .equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression switchExpression =
+ (AbstractFunctionCallExpression) aggExpression.getArguments().get(0).getValue();
+
+ ILogicalExpression arg1 = switchExpression.getArguments().get(0).getValue();
+ ILogicalExpression arg2 = switchExpression.getArguments().get(1).getValue();
+ ILogicalExpression arg3 = switchExpression.getArguments().get(2).getValue();
+ ILogicalExpression arg4 = switchExpression.getArguments().get(3).getValue();
+ if (arg2.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+ && arg3.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+ && arg4.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+ && ((ConstantExpression) arg2).getValue().isTrue()
+ && ((ConstantExpression) arg3).getValue().isNull()
+ && ((ConstantExpression) arg4).getValue().isTrue()) {
+ aggregateCondition = arg1;
+ }
+ }
+ }
+
+ // First, try to create a SELECT from the aggregate itself (i.e. handle the SOME AND EVERY case).
+ if (isNonEmptyStream && aggregateCondition != null) {
+ SelectOperator selectFromAgg = new SelectOperator(new MutableObject<>(aggregateCondition));
+ selectFromAgg.getInputs().addAll(subplanRoot.getInputs());
+ selectFromAgg.setSourceLocation(sourceLocation);
+ return selectFromAgg;
+ }
+
+ // If we could not create a SELECT from the aggregate, try to find a SELECT inside the subplan itself.
+ ILogicalOperator workingOperator = subplanRoot.getInputs().get(0).getValue();
+ while (workingOperator != null) {
+ if (workingOperator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ return (SelectOperator) workingOperator;
+ }
+ workingOperator =
+ (workingOperator.getInputs().isEmpty()) ? null : workingOperator.getInputs().get(0).getValue();
+ }
+
+ // We could not find a SELECT.
+ return null;
+ }
+
+ private ILogicalExpression keepOptimizableFunctions(ILogicalExpression cond) {
+ if (cond.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ AbstractFunctionCallExpression func = (AbstractFunctionCallExpression) cond;
+ List<Mutable<ILogicalExpression>> conjuncts = new ArrayList<>();
+ if (splitIntoConjuncts(func, conjuncts)) {
+ List<Mutable<ILogicalExpression>> optimizableConjuncts = new ArrayList<>();
+ for (Mutable<ILogicalExpression> conjunct : conjuncts) {
+ if (conjunct.getValue().getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && optimizableFunctions.contains(
+ ((AbstractFunctionCallExpression) conjunct.getValue()).getFunctionIdentifier())) {
+ optimizableConjuncts.add(conjunct);
+ }
+ }
+
+ if (optimizableConjuncts.size() == 1) {
+ return optimizableConjuncts.get(0).getValue();
+
+ } else if (optimizableConjuncts.size() > 1) {
+ ScalarFunctionCallExpression andCond = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND));
+ andCond.setSourceLocation(cond.getSourceLocation());
+ andCond.getArguments().addAll(optimizableConjuncts);
+ return andCond;
+ }
+
+ } else if (func.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && optimizableFunctions.contains(func.getFunctionIdentifier())) {
+ return cond;
+
+ }
+ }
+
+ return ConstantExpression.TRUE;
+ }
+
+ private AggregateOperator getAggregateFromSubplan(SubplanOperator subplanOperator) {
+ // We only expect one plan, and one root.
+ if (subplanOperator.getNestedPlans().size() > 1
+ || subplanOperator.getNestedPlans().get(0).getRoots().size() > 1) {
+ return null;
+ }
+
+ // This root of our "subplan" should always be an aggregate.
+ ILogicalOperator workingSubplanRoot = subplanOperator.getNestedPlans().get(0).getRoots().get(0).getValue();
+ AggregateOperator workingSubplanRootAsAggregate;
+ if (!workingSubplanRoot.getOperatorTag().equals(LogicalOperatorTag.AGGREGATE)) {
+ return null;
+ }
+ workingSubplanRootAsAggregate = (AggregateOperator) workingSubplanRoot;
+ return workingSubplanRootAsAggregate;
+ }
+
+ private boolean isStrictUniversalQuantification(AggregateOperator workingSubplanRoot) {
+ for (Mutable<ILogicalExpression> expression : workingSubplanRoot.getExpressions()) {
+ AggregateFunctionCallExpression funcExpr = (AggregateFunctionCallExpression) expression.getValue();
+ if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.EMPTY_STREAM)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean isArrayNonEmptyConjunctMissing(UnnestOperator firstUnnestInNTS, ILogicalOperator subplanInput) {
+ UnnestingFunctionCallExpression unnestFunction =
+ (UnnestingFunctionCallExpression) firstUnnestInNTS.getExpressionRef().getValue();
+ VariableReferenceExpression unnestVarExpr =
+ (VariableReferenceExpression) unnestFunction.getArguments().get(0).getValue();
+ LogicalVariable arrayVariable = unnestVarExpr.getVariableReference();
+
+ if (!subplanInput.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ return true;
+ }
+
+ ILogicalExpression selectCondExpr =
+ normalizeCondition(null, ((SelectOperator) subplanInput).getCondition().getValue());
+ List<Mutable<ILogicalExpression>> conjunctsFromSelect = new ArrayList<>();
+ if (splitIntoConjuncts(selectCondExpr, conjunctsFromSelect)) {
+ // We have a collection of conjuncts. Analyze each conjunct w/ a function.
+ for (Mutable<ILogicalExpression> mutableConjunct : conjunctsFromSelect) {
+ ILogicalExpression workingConjunct = mutableConjunct.getValue();
+ if (workingConjunct.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && analyzeConjunctForArrayNonEmptiness(arrayVariable,
+ (ScalarFunctionCallExpression) workingConjunct)) {
+ return false;
+ }
+ }
+
+ // No such conjunct found.
+ return true;
+ }
+
+ if (!selectCondExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ return true;
+ }
+ return !analyzeConjunctForArrayNonEmptiness(arrayVariable, (ScalarFunctionCallExpression) selectCondExpr);
+ }
+
+ private boolean analyzeConjunctForArrayNonEmptiness(LogicalVariable arrayVariable,
+ ScalarFunctionCallExpression workingSelectCondExpr) {
+ // Handle the conjunct: LEN(arrayVar) > 0
+ if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.GT)) {
+ ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
+ ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
+
+ if (firstArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && ((ScalarFunctionCallExpression) firstArg).getFunctionIdentifier().equals(BuiltinFunctions.LEN)) {
+ ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) firstArg;
+ List<LogicalVariable> usedVariables = new ArrayList<>();
+ lenFunction.getUsedVariables(usedVariables);
+
+ return usedVariables.contains(arrayVariable)
+ && secondArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+ && ZEROS_AS_ASTERIX_CONSTANTS.contains(((ConstantExpression) secondArg).getValue());
+ }
+ }
+
+ // Handle the conjunct: 0 < LEN(arrayVar)
+ else if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.LT)) {
+ ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
+ ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
+
+ if (secondArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && ((ScalarFunctionCallExpression) secondArg).getFunctionIdentifier()
+ .equals(BuiltinFunctions.LEN)) {
+ ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) secondArg;
+ List<LogicalVariable> usedVariables = new ArrayList<>();
+ lenFunction.getUsedVariables(usedVariables);
+
+ return usedVariables.contains(arrayVariable)
+ && firstArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+ && ZEROS_AS_ASTERIX_CONSTANTS.contains(((ConstantExpression) firstArg).getValue());
+ }
+ }
+
+ return false;
+ }
+
+ private ILogicalExpression normalizeCondition(AggregateOperator aggregateOperator, ILogicalExpression expr) {
+ // The purpose of this function is to remove the NOT(IF-MISSING-OR-NULL(...)) functions for a strict universal
+ // quantification query. The {@code ArrayBTreeAccessMethod} does not recognize the former as optimizable
+ // functions, so we remove them here. This SELECT will never make it to the final query plan (after the
+ // {@code IntroduceSelectAccessMethodRule}), which allows us to get away with this logically incorrect branch.
+ if (aggregateOperator != null && !isStrictUniversalQuantification(aggregateOperator)) {
+ // We are working with an existential quantification OR an EACH AND EVERY query. Do not modify the SELECT.
+ return expr;
+
+ } else {
+ // We are working with a strict universal quantification query.
+ ScalarFunctionCallExpression notFunction = (ScalarFunctionCallExpression) expr;
+ if (!notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
+ return expr;
+ }
+
+ ScalarFunctionCallExpression ifMissingOrNullFunction =
+ (ScalarFunctionCallExpression) notFunction.getArguments().get(0).getValue();
+ if (!ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
+ return expr;
+ }
+ return ifMissingOrNullFunction.getArguments().get(0).getValue().cloneExpression();
+ }
+ }
+
+ private boolean splitIntoConjuncts(ILogicalExpression expression, List<Mutable<ILogicalExpression>> conjuncts) {
+ List<Mutable<ILogicalExpression>> exprConjuncts = new ArrayList<>();
+ if (expression.splitIntoConjuncts(exprConjuncts)) {
+ for (Mutable<ILogicalExpression> conjunct : exprConjuncts) {
+ List<Mutable<ILogicalExpression>> innerExprConjuncts = new ArrayList<>();
+ if (splitIntoConjuncts(conjunct.getValue(), innerExprConjuncts)) {
+ conjuncts.addAll(innerExprConjuncts);
+ } else {
+ conjuncts.add(conjunct);
+ }
+ }
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/ArrayIndexStructureMatcher.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/ArrayIndexStructureMatcher.java
new file mode 100644
index 0000000..62b266a
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/ArrayIndexStructureMatcher.java
@@ -0,0 +1,212 @@
+/*
+ * 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.am.array;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.optimizer.rules.am.OptimizableOperatorSubTree;
+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.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+
+public class ArrayIndexStructureMatcher implements ArrayIndexUtil.TypeTrackerCommandExecutor {
+ private final List<AbstractLogicalOperator> logicalOperators = new ArrayList<>();
+ private int lastMatchedPosition = -1;
+ private boolean isStructureMatched = true;
+
+ private LogicalVariable varFromParent;
+ private IAType typeFromParent;
+
+ @Override
+ public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+ List<String> fieldName, boolean isFirstArrayStep, boolean isLastUnnestInIntermediateStep) {
+ // Avoiding exploring, we could not match an earlier array step.
+ if (!isStructureMatched) {
+ return;
+ }
+
+ // Match our field name and a set of ASSIGNs.
+ boolean isStructureMatchedForThisStep = matchAssignVarAndFieldName(startingStepRecordType, fieldName);
+
+ // Match an UNNEST operator with the tail of the previously matched ASSIGN.
+ if (isStructureMatchedForThisStep) {
+ isStructureMatchedForThisStep = matchUnnestVar();
+ }
+
+ // Update our flags.
+ isStructureMatched = isStructureMatched && isStructureMatchedForThisStep;
+ }
+
+ @Override
+ public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+ boolean isNonArrayStep, boolean requiresOnlyOneUnnest) {
+ if (isNonArrayStep) {
+ isStructureMatched = isStructureMatched && matchAssignVarAndFieldName(startingStepRecordType, fieldName);
+ }
+
+ if (!isStructureMatched) {
+ // If no match is found, signal this to our caller.
+ varFromParent = null;
+ lastMatchedPosition = -1;
+ }
+ }
+
+ public void reset(LogicalVariable assignVar, OptimizableOperatorSubTree subTree) {
+ varFromParent = assignVar;
+ typeFromParent = null;
+
+ // We start by assuming that the structure is matched, and try to find steps where this does not hold.
+ isStructureMatched = true;
+
+ // Build our list from the ASSIGNs and UNNESTs collected in our subtree.
+ lastMatchedPosition = -1;
+ logicalOperators.clear();
+ logicalOperators.addAll(subTree.getAssignsAndUnnests());
+ Collections.reverse(logicalOperators);
+ }
+
+ public LogicalVariable getEndVar() {
+ return varFromParent;
+ }
+
+ public ILogicalOperator getEndOperator() {
+ return (lastMatchedPosition == -1) ? null : logicalOperators.get(lastMatchedPosition);
+ }
+
+ private boolean matchUnnestVar() {
+ boolean isStructureMatchedFoundForThisStep = false;
+ AbstractLogicalOperator workingOp;
+ int searchPosition = lastMatchedPosition + 1;
+
+ // Match the UNNEST variable. Ignore any ASSIGNs we run into here.
+ while (!isStructureMatchedFoundForThisStep && searchPosition < logicalOperators.size()) {
+ workingOp = logicalOperators.get(searchPosition);
+
+ if (workingOp.getOperatorTag().equals(LogicalOperatorTag.UNNEST)) {
+ UnnestOperator workingOpAsUnnest = (UnnestOperator) workingOp;
+ List<LogicalVariable> expressionUsedVariables = new ArrayList<>();
+ ILogicalExpression unnestExpr = workingOpAsUnnest.getExpressionRef().getValue();
+ unnestExpr.getUsedVariables(expressionUsedVariables);
+
+ if (expressionUsedVariables.contains(varFromParent)) {
+ varFromParent = workingOpAsUnnest.getVariable();
+ lastMatchedPosition = searchPosition;
+ isStructureMatchedFoundForThisStep = true;
+
+ }
+ }
+ searchPosition++;
+ }
+ return isStructureMatchedFoundForThisStep;
+ }
+
+ private boolean matchAssignVarAndFieldName(ARecordType recordType, List<String> fieldName) {
+ boolean isStructureMatchedForThisStep = false;
+ AbstractLogicalOperator workingOp;
+ int searchPosition;
+
+ typeFromParent = recordType;
+ for (String fieldPart : fieldName) {
+ searchPosition = lastMatchedPosition + 1;
+ isStructureMatchedForThisStep = false;
+
+ // Match the ASSIGN variable + field name. Ignore the UNNESTs we find here.
+ while (!isStructureMatchedForThisStep && searchPosition < logicalOperators.size()) {
+ workingOp = logicalOperators.get(searchPosition);
+ if (workingOp.getOperatorTag().equals(LogicalOperatorTag.ASSIGN)) {
+
+ // Keep track of our type between matching field part calls.
+ ARecordType typeForFieldPart =
+ (typeFromParent instanceof ARecordType) ? (ARecordType) typeFromParent : recordType;
+
+ // Match the specific field part.
+ if (matchAssignVarAndFieldPart((AssignOperator) workingOp, typeForFieldPart, fieldPart)) {
+ isStructureMatchedForThisStep = true;
+ lastMatchedPosition = searchPosition;
+ }
+ }
+ searchPosition++;
+ }
+ }
+ return isStructureMatchedForThisStep;
+ }
+
+ private boolean matchAssignVarAndFieldPart(AssignOperator workingOp, ARecordType recordType, String fieldPart) {
+ final List<LogicalVariable> expressionUsedVariables = new ArrayList<>();
+ for (int j = 0; j < workingOp.getExpressions().size(); j++) {
+ ILogicalExpression assignExpr = workingOp.getExpressions().get(j).getValue();
+ assignExpr.getUsedVariables(expressionUsedVariables);
+
+ boolean isVarInExpression = expressionUsedVariables.contains(varFromParent);
+ boolean isVarInOutput = workingOp.getVariables().get(j).equals(varFromParent);
+ if (isVarInExpression || isVarInOutput) {
+ // We have found a potential match. We must now map the field names from the assign to what is actually
+ // specified in the index.
+
+ if (assignExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ ScalarFunctionCallExpression assignFunc = (ScalarFunctionCallExpression) assignExpr;
+ String fieldNameFromAssign = null;
+
+ if (assignFunc.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
+ // Search for the field name inside the expression itself.
+ ConstantExpression assignNameExpr =
+ (ConstantExpression) assignFunc.getArguments().get(1).getValue();
+ IAObject assignNameObj = ((AsterixConstantValue) assignNameExpr.getValue()).getObject();
+ fieldNameFromAssign = ((AString) assignNameObj).getStringValue();
+
+ } else if (assignFunc.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_INDEX)) {
+ // Search for the field name using the type we are tracking.
+ ConstantExpression assignIndexExpr =
+ (ConstantExpression) assignFunc.getArguments().get(1).getValue();
+ IAObject assignIndexObj = ((AsterixConstantValue) assignIndexExpr.getValue()).getObject();
+ int assignIndex = ((AInt32) assignIndexObj).getIntegerValue();
+ fieldNameFromAssign = recordType.getFieldNames()[assignIndex];
+ typeFromParent = recordType.getFieldTypes()[assignIndex];
+ }
+
+ if (fieldNameFromAssign != null && fieldNameFromAssign.equals(fieldPart)) {
+ // We have found a match.
+ varFromParent = workingOp.getVariables().get(j);
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/IIntroduceAccessMethodRuleLocalRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/IIntroduceAccessMethodRuleLocalRewrite.java
new file mode 100644
index 0000000..9942bbe
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/IIntroduceAccessMethodRuleLocalRewrite.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.optimizer.rules.am.array;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+
+public interface IIntroduceAccessMethodRuleLocalRewrite<T> {
+ /**
+ * @param originalOperator Original operator before rewrite. Should be returned by {@code restoreBeforeRewrite}.
+ * @param context Optimization context.
+ * @return Null if no rewrite has occurred. Otherwise, a new plan from the given operator.
+ * @throws AlgebricksException
+ */
+ T createOperator(T originalOperator, IOptimizationContext context) throws AlgebricksException;
+
+ /**
+ * @param afterOperatorRefs Operators after the original operator that should be restored after the rewrite.
+ * @param context Optimization context.
+ * @return The original operator given at {@code createOperator} time.
+ * @throws AlgebricksException
+ */
+ T restoreBeforeRewrite(List<Mutable<ILogicalOperator>> afterOperatorRefs, IOptimizationContext context)
+ throws AlgebricksException;
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java
new file mode 100644
index 0000000..aa80462
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/JoinFromSubplanRewrite.java
@@ -0,0 +1,296 @@
+/*
+ * 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.am.array;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+
+/**
+ * For use in writing a "throwaway" branch which removes NTS and subplan operators. The result of this invocation is to
+ * be given to the {@code IntroduceJoinAccessMethodRule} to check if an array index can be used.
+ * <br>
+ * If we are given the pattern (an existential quantification over a cross product):
+ * <pre>
+ * SELECT_1(some variable)
+ * SUBPLAN_1 -----------------------|
+ * | AGGREGATE(NON-EMPTY-STREAM)
+ * | SELECT_2(some predicate)
+ * | (UNNEST/ASSIGN)*
+ * | UNNEST(on variable)
+ * | NESTED-TUPLE-SOURCE
+ * JOIN(true)
+ * | |----------------- (potential) index branch ...
+ * |----------------- probe branch ...
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * JOIN(some predicate from SELECT_2)
+ * | |----------------- (UNNEST/ASSIGN)*
+ * | UNNEST(on variable)
+ * | (potential) index branch ...
+ * |----------------- probe branch ...
+ * </pre>
+ *
+ * If we are given the pattern (a universal quantification over a cross product):
+ * <pre>
+ * SELECT_1(some variable AND array is not empty)
+ * SUBPLAN_1 -----------------------|
+ * | AGGREGATE(EMPTY-STREAM)
+ * | SELECT_2(NOT(IF-MISSING-OR-NULL(some optimizable predicate)))
+ * | (UNNEST/ASSIGN)*
+ * | UNNEST(on variable)
+ * | NESTED-TUPLE-SOURCE
+ * JOIN(true)
+ * | |----------------- (potential) index branch ...
+ * |----------------- probe branch ...
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * JOIN(some optimizable predicate) <--- removed the NOT(IF-MISSING-OR-NULL(...))!
+ * | |----------------- (UNNEST/ASSIGN)*
+ * | UNNEST(on variable)
+ * | (potential) index branch ...
+ * |----------------- probe branch ...
+ * </pre>
+ *
+ * In the case of nested-subplans, we return a copy of the innermost SELECT followed by all relevant UNNEST/ASSIGNs.
+ */
+public class JoinFromSubplanRewrite extends AbstractOperatorFromSubplanRewrite<AbstractBinaryJoinOperator> {
+ private final static Set<FunctionIdentifier> optimizableFunctions = new HashSet<>();
+ private final Deque<JoinFromSubplanContext> contextStack = new ArrayDeque<>();
+
+ /**
+ * Add an optimizable function from an access method that can take advantage of this throwaway branch.
+ */
+ public static void addOptimizableFunction(FunctionIdentifier functionIdentifier) {
+ optimizableFunctions.add(functionIdentifier);
+ }
+
+ /**
+ * The subplan we want to push to the JOIN operator is located *above/after* the JOIN itself.
+ */
+ public void findAfterSubplanSelectOperator(List<Mutable<ILogicalOperator>> afterJoinRefs)
+ throws AlgebricksException {
+ JoinFromSubplanContext joinContext = new JoinFromSubplanContext();
+ contextStack.push(joinContext);
+
+ // Minimally, we need to have a DISTRIBUTE <- SELECT <- SUBPLAN.
+ if (afterJoinRefs.size() < 3) {
+ return;
+ }
+
+ // We expect a) the operator immediately above to be a SUBPLAN, and b) the next operator above to be a SELECT.
+ Mutable<ILogicalOperator> afterJoinOpRef1 = afterJoinRefs.get(afterJoinRefs.size() - 1);
+ Mutable<ILogicalOperator> afterJoinOpRef2 = afterJoinRefs.get(afterJoinRefs.size() - 2);
+ Mutable<ILogicalOperator> afterJoinOpRef3 = afterJoinRefs.get(afterJoinRefs.size() - 3);
+ ILogicalOperator afterJoinOp1 = afterJoinOpRef1.getValue();
+ ILogicalOperator afterJoinOp2 = afterJoinOpRef2.getValue();
+ ILogicalOperator afterJoinOp3 = afterJoinOpRef3.getValue();
+ if (!afterJoinOp1.getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)
+ || !afterJoinOp2.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ return;
+ }
+
+ // Additionally, verify that our SELECT is conditioning on a variable.
+ joinContext.selectAfterSubplan = (SelectOperator) afterJoinOp2;
+ if (getConditioningVariable(joinContext.selectAfterSubplan.getCondition().getValue()) == null) {
+ return;
+ }
+
+ // Modify the given after-join operators. We will reconnect these after the join-rule transformation.
+ joinContext.removedAfterJoinOperators = new ArrayList<>();
+ joinContext.removedAfterJoinOperators.add(afterJoinOpRef2);
+ joinContext.removedAfterJoinOperators.add(afterJoinOpRef1);
+ afterJoinRefs.remove(afterJoinOpRef2);
+ afterJoinRefs.remove(afterJoinOpRef1);
+
+ // Connect our inputs here. We will compute the type environment for this copy in {@code createOperator}.
+ joinContext.afterJoinOpForRewrite = OperatorManipulationUtil.deepCopy(afterJoinOp3);
+ joinContext.afterJoinOpForRewrite.getInputs().clear();
+ joinContext.afterJoinOpForRewrite.getInputs().addAll(afterJoinOp3.getInputs());
+ }
+
+ /**
+ * Create a new branch to match that of the form:
+ *
+ * <pre>
+ * JOIN(...)
+ * | |----------------- (UNNEST/ASSIGN)*
+ * | UNNEST
+ * | (potential) index branch ...
+ * |----------------- probe branch ...
+ * </pre>
+ * <p>
+ * Operators are *created* here, rather than just reconnected from the original branch.
+ */
+ @Override
+ public AbstractBinaryJoinOperator createOperator(AbstractBinaryJoinOperator originalOperator,
+ IOptimizationContext context) throws AlgebricksException {
+ // Reset our context.
+ this.reset(originalOperator.getSourceLocation(), context, optimizableFunctions);
+ JoinFromSubplanContext joinContext = contextStack.getFirst();
+ joinContext.originalJoinRoot = originalOperator;
+ if (joinContext.removedAfterJoinOperators == null) {
+ return null;
+ }
+
+ // Traverse our subplan and generate a SELECT branch if applicable.
+ SubplanOperator subplanOperator =
+ (SubplanOperator) joinContext.selectAfterSubplan.getInputs().get(0).getValue();
+ List<Mutable<ILogicalOperator>> originalOpInputs = originalOperator.getInputs();
+ Pair<SelectOperator, UnnestOperator> traversalOutput =
+ traverseSubplanBranch(subplanOperator, originalOpInputs.get(1).getValue(), true);
+ if (traversalOutput == null) {
+ return null;
+ }
+
+ // We have successfully generated a SELECT branch. Create the new JOIN operator.
+ ScalarFunctionCallExpression newCond = coalesceConditions(traversalOutput.first, joinContext.originalJoinRoot);
+ joinContext.newJoinRoot = new InnerJoinOperator(new MutableObject<>(newCond));
+ joinContext.newJoinRoot.getInputs().add(0, new MutableObject<>(originalOpInputs.get(0).getValue()));
+
+ // Connect the join branches together.
+ traversalOutput.second.getInputs().clear();
+ traversalOutput.second.getInputs().add(new MutableObject<>(originalOpInputs.get(1).getValue()));
+ joinContext.newJoinRoot.getInputs().add(1, traversalOutput.first.getInputs().get(0));
+ context.computeAndSetTypeEnvironmentForOperator(joinContext.newJoinRoot);
+
+ // To support type casting that is performed on the index subtree and still make this expression recognizable,
+ // push all function calls to a lower ASSIGN.
+ List<Mutable<ILogicalExpression>> conjuncts = new ArrayList<>();
+ if (newCond.splitIntoConjuncts(conjuncts)) {
+ for (Mutable<ILogicalExpression> conjunct : conjuncts) {
+ extractFunctionCallToAssign(joinContext.newJoinRoot, context, conjunct.getValue());
+ }
+
+ } else {
+ extractFunctionCallToAssign(joinContext.newJoinRoot, context, newCond);
+ }
+
+ // Reconnect our after-join operator to our new join.
+ OperatorManipulationUtil.substituteOpInInput(joinContext.afterJoinOpForRewrite,
+ joinContext.removedAfterJoinOperators.get(0).getValue(), new MutableObject<>(joinContext.newJoinRoot));
+ context.computeAndSetTypeEnvironmentForOperator(joinContext.afterJoinOpForRewrite);
+ return joinContext.newJoinRoot;
+ }
+
+ /**
+ * To undo this process is to return what was passed to us at {@code createOperator} time. If we removed any
+ * after-join references, add them back in the order they were originally given.
+ */
+ @Override
+ public AbstractBinaryJoinOperator restoreBeforeRewrite(List<Mutable<ILogicalOperator>> afterOperatorRefs,
+ IOptimizationContext context) throws AlgebricksException {
+ JoinFromSubplanContext joinContext = contextStack.pop();
+ if (joinContext.removedAfterJoinOperators != null) {
+ afterOperatorRefs.addAll(joinContext.removedAfterJoinOperators);
+ }
+
+ return joinContext.originalJoinRoot;
+ }
+
+ private void extractFunctionCallToAssign(AbstractBinaryJoinOperator joinOp, IOptimizationContext context,
+ ILogicalExpression condition) throws AlgebricksException {
+ if (!condition.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ return;
+ }
+ AbstractFunctionCallExpression conditionAsFuncCall = (AbstractFunctionCallExpression) condition;
+ if (!AlgebricksBuiltinFunctions.isComparisonFunction(conditionAsFuncCall.getFunctionIdentifier())) {
+ return;
+ }
+
+ for (Mutable<ILogicalExpression> arg : conditionAsFuncCall.getArguments()) {
+ if (!arg.getValue().getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ continue;
+ }
+
+ LogicalVariable newVar = context.newVar();
+ VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+ newVarRef.setSourceLocation(joinOp.getSourceLocation());
+ AssignOperator newAssign =
+ new AssignOperator(newVar, new MutableObject<>(arg.getValue().cloneExpression()));
+ newAssign.setSourceLocation(arg.getValue().getSourceLocation());
+ newAssign.setExecutionMode(joinOp.getExecutionMode());
+
+ // Place the new ASSIGN in the appropriate join branch.
+ ILogicalOperator leftBranchRoot = joinOp.getInputs().get(0).getValue();
+ ILogicalOperator rightBranchRoot = joinOp.getInputs().get(1).getValue();
+ List<LogicalVariable> usedVarsFromFunc = new ArrayList<>();
+ List<LogicalVariable> varsFromLeftBranch = new ArrayList<>();
+ List<LogicalVariable> varsFromRightBranch = new ArrayList<>();
+ VariableUtilities.getUsedVariables(newAssign, usedVarsFromFunc);
+ VariableUtilities.getProducedVariablesInDescendantsAndSelf(leftBranchRoot, varsFromLeftBranch);
+ VariableUtilities.getProducedVariablesInDescendantsAndSelf(rightBranchRoot, varsFromRightBranch);
+ if (varsFromLeftBranch.containsAll(usedVarsFromFunc)) {
+ newAssign.getInputs().add(new MutableObject<>(leftBranchRoot));
+ context.computeAndSetTypeEnvironmentForOperator(newAssign);
+ joinOp.getInputs().get(0).setValue(newAssign);
+ context.computeAndSetTypeEnvironmentForOperator(joinOp);
+ arg.setValue(newVarRef);
+
+ } else if (varsFromRightBranch.containsAll(usedVarsFromFunc)) {
+ newAssign.getInputs().add(new MutableObject<>(rightBranchRoot));
+ context.computeAndSetTypeEnvironmentForOperator(newAssign);
+ joinOp.getInputs().get(1).setValue(newAssign);
+ context.computeAndSetTypeEnvironmentForOperator(joinOp);
+ arg.setValue(newVarRef);
+
+ }
+ }
+ }
+
+ /**
+ * All state associated with a single call of {@code createOperator}.
+ */
+ private static class JoinFromSubplanContext {
+ private List<Mutable<ILogicalOperator>> removedAfterJoinOperators;
+ private AbstractBinaryJoinOperator originalJoinRoot;
+ private AbstractBinaryJoinOperator newJoinRoot;
+ private SelectOperator selectAfterSubplan;
+ private ILogicalOperator afterJoinOpForRewrite;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
new file mode 100644
index 0000000..75168f5
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
@@ -0,0 +1,161 @@
+/*
+ * 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.am.array;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+
+/**
+ * For all expressions that pertain to a single dataset but are spread across various SELECTs due to requiring an
+ * intermediate UNNEST, "push up" the lower SELECT expressions into a single unifying SELECT. This is used to
+ * recognize composite atomic-array index applicability.
+ */
+public class MergedSelectRewrite implements IIntroduceAccessMethodRuleLocalRewrite<SelectOperator> {
+ private final Set<Mutable<ILogicalExpression>> selectConjuncts = new LinkedHashSet<>();
+ private final Deque<SelectOperator> selectRootStack = new ArrayDeque<>();
+ private Mutable<ILogicalOperator> originalDataSourceRef;
+
+ @Override
+ public SelectOperator createOperator(SelectOperator originalOperator, IOptimizationContext context)
+ throws AlgebricksException {
+ // Initialize the conjuncts for our SELECT.
+ selectConjuncts.clear();
+ List<Mutable<ILogicalExpression>> thisSelectConjuncts = new ArrayList<>();
+ if (!originalOperator.getCondition().getValue().splitIntoConjuncts(thisSelectConjuncts)) {
+ thisSelectConjuncts.add(originalOperator.getCondition());
+ }
+ selectConjuncts.addAll(thisSelectConjuncts);
+
+ // Explore all operators below this SELECT until the first data source operator.
+ selectRootStack.push(originalOperator);
+ if (!collectSelectConjuncts(originalOperator.getInputs().get(0))) {
+ // We encountered a JOIN operator. Exit early.
+ return null;
+ }
+
+ if (thisSelectConjuncts.size() == selectConjuncts.size()) {
+ // No other SELECTs were found. Return null to indicate there were no SELECTs to merge.
+ return null;
+
+ } else {
+ // We have found additional SELECTs. Form a conjunction.
+ AbstractFunctionCallExpression andCond = new ScalarFunctionCallExpression(
+ context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND));
+ andCond.setSourceLocation(originalOperator.getSourceLocation());
+ for (Mutable<ILogicalExpression> conjunct : selectConjuncts) {
+ andCond.getArguments().add(conjunct);
+ }
+
+ // Return a new plan that removes all SELECTs that were pushed up.
+ SelectOperator newSelectOperator = new SelectOperator(new MutableObject<>(andCond),
+ originalOperator.getRetainMissingAsValue(), originalOperator.getMissingPlaceholderVariable());
+ newSelectOperator.setSourceLocation(originalOperator.getSourceLocation());
+ ILogicalPlan newSelectInputPlan = OperatorManipulationUtil
+ .deepCopy(new ALogicalPlanImpl(originalOperator.getInputs().get(0)), context);
+ newSelectOperator.getInputs().add(newSelectInputPlan.getRoots().get(0));
+ removeSelectsFromPlan(newSelectOperator, newSelectInputPlan.getRoots().get(0));
+ OperatorManipulationUtil.computeTypeEnvironmentBottomUp(newSelectOperator, context);
+ return newSelectOperator;
+ }
+ }
+
+ @Override
+ public SelectOperator restoreBeforeRewrite(List<Mutable<ILogicalOperator>> afterOperatorRefs,
+ IOptimizationContext context) throws AlgebricksException {
+ return selectRootStack.pop();
+ }
+
+ private boolean collectSelectConjuncts(Mutable<ILogicalOperator> workingOp) {
+ switch (workingOp.getValue().getOperatorTag()) {
+ case DATASOURCESCAN:
+ case EMPTYTUPLESOURCE:
+ case UNNEST_MAP:
+ // If we have reached a datasource operator, stop our search.
+ originalDataSourceRef = workingOp;
+ break;
+
+ case INNERJOIN:
+ case LEFTOUTERJOIN:
+ // We are not interested in exploring joins in this class.
+ return false;
+
+ case SELECT:
+ SelectOperator selectOperator = (SelectOperator) workingOp.getValue();
+ List<Mutable<ILogicalExpression>> thisSelectConjuncts = new ArrayList<>();
+ if (!selectOperator.getCondition().getValue().splitIntoConjuncts(thisSelectConjuncts)) {
+ thisSelectConjuncts.add(selectOperator.getCondition());
+ }
+ selectConjuncts.addAll(thisSelectConjuncts);
+
+ default:
+ // Explore the rest of our plan in our DFS fashion.
+ for (Mutable<ILogicalOperator> input : workingOp.getValue().getInputs()) {
+ if (!collectSelectConjuncts(input)) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+
+ private void removeSelectsFromPlan(ILogicalOperator parentOp, Mutable<ILogicalOperator> workingOp) {
+ Mutable<ILogicalOperator> workingOpInParent =
+ parentOp.getInputs().stream().filter(i -> i.equals(workingOp)).collect(Collectors.toList()).get(0);
+ int indexOfWorkingOpInParent = parentOp.getInputs().indexOf(workingOpInParent);
+
+ switch (workingOp.getValue().getOperatorTag()) {
+ case DATASOURCESCAN:
+ case EMPTYTUPLESOURCE:
+ case UNNEST_MAP:
+ // If we have reached a datasource operator, stop and replace this with our original datasource.
+ // (IntroduceSelectAccessMethodRule replaces this specific operator, so this must be the original.)
+ parentOp.getInputs().set(indexOfWorkingOpInParent, originalDataSourceRef);
+ break;
+
+ case SELECT:
+ parentOp.getInputs().set(indexOfWorkingOpInParent, workingOp.getValue().getInputs().get(0));
+
+ default:
+ // Explore the rest of our plan in our DFS fashion.
+ for (Mutable<ILogicalOperator> input : workingOp.getValue().getInputs()) {
+ removeSelectsFromPlan(workingOp.getValue(), input);
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/SelectFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/SelectFromSubplanRewrite.java
new file mode 100644
index 0000000..594ba41
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/SelectFromSubplanRewrite.java
@@ -0,0 +1,133 @@
+/*
+ * 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.am.array;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+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.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+
+/**
+ * For use in writing a "throwaway" branch which removes NTS and subplan operators. The result of this invocation is to
+ * be given to the {@code IntroduceSelectAccessMethodRule} to check if an array index can be used.
+ * <br>
+ * If we are given the pattern (an existential quantification query):
+ * <pre>
+ * SELECT_1(some variable)
+ * SUBPLAN_1 -------------------------------|
+ * (parent branch input) AGGREGATE(NON-EMPTY-STREAM)
+ * SELECT_2(some predicate)
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * NESTED-TUPLE-SOURCE
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * SELECT_2(some predicate)
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * (parent branch input)
+ * </pre>
+ *
+ * If we are given the pattern (a universal quantification query):
+ * <pre>
+ * SELECT_1(some variable AND array is not empty)
+ * SUBPLAN_1 -------------------------------|
+ * (parent branch input) AGGREGATE(EMPTY-STREAM)
+ * SELECT_2(NOT(IF-MISSING-OR-NULL(some optimizable predicate)))
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * NESTED-TUPLE-SOURCE
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * SELECT_2(some optimizable predicate) <--- removed the NOT(IF-MISSING-OR-NULL(...))!
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * (parent branch input)
+ * </pre>
+ *
+ * In the case of nested-subplans, we return a copy of the innermost SELECT followed by all relevant UNNEST/ASSIGNs.
+ */
+public class SelectFromSubplanRewrite extends AbstractOperatorFromSubplanRewrite<SelectOperator> {
+ private final static Set<FunctionIdentifier> optimizableFunctions = new HashSet<>();
+ private final Deque<SelectOperator> selectRootStack = new ArrayDeque<>();
+
+ /**
+ * Add an optimizable function from an access method that can take advantage of this throwaway branch.
+ */
+ public static void addOptimizableFunction(FunctionIdentifier functionIdentifier) {
+ optimizableFunctions.add(functionIdentifier);
+ }
+
+ /**
+ * Create a new branch to match that of the form:
+ *
+ * <pre>
+ * SELECT (...)
+ * (UNNEST/ASSIGN)*
+ * UNNEST
+ * ...
+ * </pre>
+ *
+ * Operators are *created* here, rather than just reconnected from the original branch.
+ */
+ @Override
+ public SelectOperator createOperator(SelectOperator originalOperator, IOptimizationContext context)
+ throws AlgebricksException {
+ // Reset our context.
+ selectRootStack.push(originalOperator);
+ reset(originalOperator.getSourceLocation(), context, optimizableFunctions);
+
+ // We expect a) a SUBPLAN as input to this SELECT, and b) our SELECT to be conditioning on a variable.
+ LogicalVariable originalSelectVar = getConditioningVariable(originalOperator.getCondition().getValue());
+ if (!originalOperator.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)
+ || originalSelectVar == null) {
+ return null;
+ }
+
+ // Traverse our subplan and generate a SELECT branch if applicable.
+ SubplanOperator subplanOperator = (SubplanOperator) originalOperator.getInputs().get(0).getValue();
+ Pair<SelectOperator, UnnestOperator> traversalOutput = traverseSubplanBranch(subplanOperator, null, true);
+ return (traversalOutput == null) ? null : traversalOutput.first;
+ }
+
+ /**
+ * To undo this process is to return what was passed to us at {@code createOperator} time. We do not touch the
+ * operators after the SELECT.
+ */
+ @Override
+ public SelectOperator restoreBeforeRewrite(List<Mutable<ILogicalOperator>> afterOperatorRefs,
+ IOptimizationContext context) throws AlgebricksException {
+ return selectRootStack.pop();
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
new file mode 100644
index 0000000..b7632db
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaBuilder.java
@@ -0,0 +1,218 @@
+/*
+ * 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.pushdown;
+
+import static org.apache.asterix.optimizer.rules.pushdown.ExpressionValueAccessPushdownVisitor.ARRAY_FUNCTIONS;
+import static org.apache.asterix.optimizer.rules.pushdown.ExpressionValueAccessPushdownVisitor.SUPPORTED_FUNCTIONS;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.optimizer.rules.pushdown.schema.AbstractComplexExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.AnyExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ArrayExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ExpectedSchemaNodeType;
+import org.apache.asterix.optimizer.rules.pushdown.schema.IExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ObjectExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.RootExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.UnionExpectedSchemaNode;
+import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+
+/**
+ * This class takes a value access expression and produces an expected schema (given the expression).
+ * Example:
+ * - $$t.getField("hashtags").getItem(0)
+ * We expect:
+ * 1- $$t is OBJECT
+ * 2- the output type of getField("hashtags") is ARRAY
+ * 3- the output type of getItem(0) is ANY node
+ */
+class ExpectedSchemaBuilder {
+ //Registered Variables
+ private final Map<LogicalVariable, IExpectedSchemaNode> varToNode;
+ private final ExpectedSchemaNodeToIATypeTranslatorVisitor typeBuilder;
+
+ public ExpectedSchemaBuilder() {
+ varToNode = new HashMap<>();
+ typeBuilder = new ExpectedSchemaNodeToIATypeTranslatorVisitor();
+ }
+
+ public DataProjectionInfo createProjectionInfo(LogicalVariable recordVariable) {
+ IExpectedSchemaNode rootNode = varToNode.get(recordVariable);
+ Map<String, FunctionCallInformation> sourceInformation = new HashMap<>();
+ typeBuilder.reset(sourceInformation);
+ ARecordType recordType = (ARecordType) rootNode.accept(typeBuilder, null);
+ return new DataProjectionInfo(recordType, sourceInformation);
+ }
+
+ public boolean setSchemaFromExpression(AbstractFunctionCallExpression expr, LogicalVariable producedVar) {
+ //Parent always nested
+ AbstractComplexExpectedSchemaNode parent = (AbstractComplexExpectedSchemaNode) buildNestedNode(expr);
+ if (parent != null) {
+ IExpectedSchemaNode leaf =
+ new AnyExpectedSchemaNode(parent, expr.getSourceLocation(), expr.getFunctionIdentifier().getName());
+ addChild(expr, parent, leaf);
+ if (producedVar != null) {
+ //Register the node if a variable is produced
+ varToNode.put(producedVar, leaf);
+ }
+ }
+ return parent != null;
+ }
+
+ public void registerDataset(LogicalVariable recordVar, RootExpectedSchemaNode rootNode) {
+ varToNode.put(recordVar, rootNode);
+ }
+
+ public void unregisterVariable(LogicalVariable variable) {
+ //Remove the node so no other expression will pushdown any expression in the future
+ IExpectedSchemaNode node = varToNode.remove(variable);
+ AbstractComplexExpectedSchemaNode parent = node.getParent();
+ if (parent == null) {
+ //It is a root node. Request the entire record
+ varToNode.put(variable, RootExpectedSchemaNode.ALL_FIELDS_ROOT_NODE);
+ } else {
+ //It is a nested node. Replace the node to a LEAF node
+ node.replaceIfNeeded(ExpectedSchemaNodeType.ANY, parent.getSourceLocation(), parent.getFunctionName());
+ }
+ }
+
+ public boolean isVariableRegistered(LogicalVariable recordVar) {
+ return varToNode.containsKey(recordVar);
+ }
+
+ public boolean containsRegisteredDatasets() {
+ return !varToNode.isEmpty();
+ }
+
+ private IExpectedSchemaNode buildNestedNode(ILogicalExpression expr) {
+ //The current node expression
+ AbstractFunctionCallExpression myExpr = (AbstractFunctionCallExpression) expr;
+ if (!SUPPORTED_FUNCTIONS.contains(myExpr.getFunctionIdentifier())) {
+ //Return null if the function is not supported.
+ return null;
+ }
+
+ //The parent expression
+ ILogicalExpression parentExpr = myExpr.getArguments().get(0).getValue();
+ if (isVariable(parentExpr)) {
+ //A variable could be the record's originated from data-scan or an expression from assign
+ LogicalVariable sourceVar = VariableUtilities.getVariable(parentExpr);
+ return changeNodeForVariable(sourceVar, myExpr);
+ }
+
+ //Recursively create the parent nodes. Parent is always a nested node
+ AbstractComplexExpectedSchemaNode newParent = (AbstractComplexExpectedSchemaNode) buildNestedNode(parentExpr);
+ //newParent could be null if the expression is not supported
+ if (newParent != null) {
+ //Parent expression must be a function call (as parent is a nested node)
+ AbstractFunctionCallExpression parentFuncExpr = (AbstractFunctionCallExpression) parentExpr;
+ //Get 'myType' as we will create the child type of the newParent
+ ExpectedSchemaNodeType myType = getExpectedNestedNodeType(myExpr);
+ /*
+ * Create 'myNode'. It is a nested node because the function is either getField() or supported array
+ * function
+ */
+ AbstractComplexExpectedSchemaNode myNode = AbstractComplexExpectedSchemaNode.createNestedNode(myType,
+ newParent, myExpr.getSourceLocation(), myExpr.getFunctionIdentifier().getName());
+ //Add myNode to the parent
+ addChild(parentFuncExpr, newParent, myNode);
+ return myNode;
+ }
+ return null;
+ }
+
+ private IExpectedSchemaNode changeNodeForVariable(LogicalVariable sourceVar,
+ AbstractFunctionCallExpression myExpr) {
+ //Get the associated node with the sourceVar (if any)
+ IExpectedSchemaNode oldNode = varToNode.get(sourceVar);
+ if (oldNode == null) {
+ //Variable is not associated with a node. No pushdown is possible
+ return null;
+ }
+ //What is the expected type of the variable
+ ExpectedSchemaNodeType varExpectedType = getExpectedNestedNodeType(myExpr);
+ // Get the node associated with the variable (or change its type if needed).
+ IExpectedSchemaNode newNode = oldNode.replaceIfNeeded(varExpectedType, myExpr.getSourceLocation(),
+ myExpr.getFunctionIdentifier().getName());
+ //Map the sourceVar to the node
+ varToNode.put(sourceVar, newNode);
+ return newNode;
+ }
+
+ private void addChild(AbstractFunctionCallExpression parentExpr, AbstractComplexExpectedSchemaNode parent,
+ IExpectedSchemaNode child) {
+ switch (parent.getType()) {
+ case OBJECT:
+ handleObject(parentExpr, parent, child);
+ break;
+ case ARRAY:
+ handleArray(parent, child);
+ break;
+ case UNION:
+ handleUnion(parentExpr, parent, child);
+ break;
+ default:
+ throw new IllegalStateException("Node " + parent.getType() + " is not nested");
+
+ }
+ }
+
+ private void handleObject(AbstractFunctionCallExpression parentExpr, AbstractComplexExpectedSchemaNode parent,
+ IExpectedSchemaNode child) {
+ ObjectExpectedSchemaNode objectNode = (ObjectExpectedSchemaNode) parent;
+ objectNode.addChild(ConstantExpressionUtil.getStringArgument(parentExpr, 1), child);
+ }
+
+ private void handleArray(AbstractComplexExpectedSchemaNode parent, IExpectedSchemaNode child) {
+ ArrayExpectedSchemaNode arrayNode = (ArrayExpectedSchemaNode) parent;
+ arrayNode.addChild(child);
+ }
+
+ private void handleUnion(AbstractFunctionCallExpression parentExpr, AbstractComplexExpectedSchemaNode parent,
+ IExpectedSchemaNode child) {
+ UnionExpectedSchemaNode unionNode = (UnionExpectedSchemaNode) parent;
+ ExpectedSchemaNodeType parentType = getExpectedNestedNodeType(parentExpr);
+ addChild(parentExpr, unionNode.getChild(parentType), child);
+ }
+
+ private static ExpectedSchemaNodeType getExpectedNestedNodeType(AbstractFunctionCallExpression funcExpr) {
+ FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
+ if (BuiltinFunctions.FIELD_ACCESS_BY_NAME.equals(fid)) {
+ return ExpectedSchemaNodeType.OBJECT;
+ } else if (ARRAY_FUNCTIONS.contains(fid)) {
+ return ExpectedSchemaNodeType.ARRAY;
+ }
+ throw new IllegalStateException("Function " + fid + " should not be pushed down");
+ }
+
+ private static boolean isVariable(ILogicalExpression expr) {
+ return expr.getExpressionTag() == LogicalExpressionTag.VARIABLE;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
new file mode 100644
index 0000000..c746994
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpectedSchemaNodeToIATypeTranslatorVisitor.java
@@ -0,0 +1,114 @@
+/*
+ * 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.pushdown;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.optimizer.rules.pushdown.schema.AbstractComplexExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.AnyExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ArrayExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ExpectedSchemaNodeType;
+import org.apache.asterix.optimizer.rules.pushdown.schema.IExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.IExpectedSchemaNodeVisitor;
+import org.apache.asterix.optimizer.rules.pushdown.schema.ObjectExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.RootExpectedSchemaNode;
+import org.apache.asterix.optimizer.rules.pushdown.schema.UnionExpectedSchemaNode;
+import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+
+/**
+ * This visitor translates the {@link IExpectedSchemaNode} to {@link IAType} record.
+ * The {@link IAType#getTypeName()} is used to map each {@link IAType} to its {@link FunctionCallInformation}
+ */
+class ExpectedSchemaNodeToIATypeTranslatorVisitor implements IExpectedSchemaNodeVisitor<IAType, String> {
+ //Map typeName to source information
+ private Map<String, FunctionCallInformation> sourceInformationMap;
+ //To give a unique name for each type
+ private int counter;
+
+ public void reset(Map<String, FunctionCallInformation> sourceInformationMap) {
+ this.sourceInformationMap = sourceInformationMap;
+ }
+
+ @Override
+ public IAType visit(RootExpectedSchemaNode node, String arg) {
+ if (node.isAllFields()) {
+ return DataProjectionInfo.ALL_FIELDS_TYPE;
+ } else if (node.isEmpty()) {
+ return DataProjectionInfo.EMPTY_TYPE;
+ }
+ return createRecordType(node, String.valueOf(counter++));
+ }
+
+ @Override
+ public IAType visit(ObjectExpectedSchemaNode node, String arg) {
+ IAType recordType = createRecordType(node, arg);
+ sourceInformationMap.put(arg, createFunctionCallInformation(node));
+ return recordType;
+ }
+
+ @Override
+ public IAType visit(ArrayExpectedSchemaNode node, String arg) {
+ IAType itemType = node.getChild().accept(this, String.valueOf(counter++));
+ IAType listType = new AOrderedListType(itemType, arg);
+ sourceInformationMap.put(arg, createFunctionCallInformation(node));
+ return listType;
+ }
+
+ @Override
+ public IAType visit(UnionExpectedSchemaNode node, String arg) {
+ List<IAType> unionTypes = new ArrayList<>();
+ for (Map.Entry<ExpectedSchemaNodeType, AbstractComplexExpectedSchemaNode> child : node.getChildren()) {
+ unionTypes.add(child.getValue().accept(this, String.valueOf(counter++)));
+ }
+ IAType unionType = new AUnionType(unionTypes, arg);
+ sourceInformationMap.put(arg, createFunctionCallInformation(node));
+ return unionType;
+ }
+
+ @Override
+ public IAType visit(AnyExpectedSchemaNode node, String arg) {
+ return BuiltinType.ANY;
+ }
+
+ private ARecordType createRecordType(ObjectExpectedSchemaNode node, String arg) {
+ Set<Map.Entry<String, IExpectedSchemaNode>> children = node.getChildren();
+ String[] childrenFieldNames = new String[children.size()];
+ IAType[] childrenTypes = new IAType[children.size()];
+ int i = 0;
+ for (Map.Entry<String, IExpectedSchemaNode> child : children) {
+ childrenFieldNames[i] = child.getKey();
+ childrenTypes[i++] = child.getValue().accept(this, String.valueOf(counter++));
+ }
+
+ return new ARecordType(arg, childrenFieldNames, childrenTypes, true);
+ }
+
+ private FunctionCallInformation createFunctionCallInformation(IExpectedSchemaNode node) {
+ return new FunctionCallInformation(node.getFunctionName(), node.getSourceLocation());
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
new file mode 100644
index 0000000..a8dfe1e
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
@@ -0,0 +1,183 @@
+/*
+ * 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.pushdown;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+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.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+
+class ExpressionValueAccessPushdownVisitor implements ILogicalExpressionReferenceTransform {
+ //Set of supported type-check functions
+ static final Set<FunctionIdentifier> TYPE_CHECK_FUNCTIONS = createSupportedTypeCheckFunctions();
+ //Set of supported array functions
+ static final Set<FunctionIdentifier> ARRAY_FUNCTIONS = createSupportedArrayFunctions();
+ //Set of supported functions that we can pushdown
+ static final Set<FunctionIdentifier> SUPPORTED_FUNCTIONS = createSupportedFunctions();
+
+ private final ExpectedSchemaBuilder builder;
+ private List<LogicalVariable> producedVariables;
+ private int producedVariableIndex;
+
+ public ExpressionValueAccessPushdownVisitor(ExpectedSchemaBuilder builder) {
+ this.builder = builder;
+ end();
+ }
+
+ public void init(List<LogicalVariable> producedVariables) {
+ this.producedVariables = producedVariables;
+ producedVariableIndex = 0;
+ }
+
+ @Override
+ public boolean transform(Mutable<ILogicalExpression> expression) throws AlgebricksException {
+ if (producedVariableIndex == -1) {
+ //This for ensuring that the produced variables (if any) should be set
+ throw new IllegalStateException("init must be called first");
+ }
+ pushValueAccessExpression(expression, getNextProducedVariable());
+ return false;
+ }
+
+ public void end() {
+ producedVariables = null;
+ producedVariableIndex = -1;
+ }
+
+ private LogicalVariable getNextProducedVariable() {
+ LogicalVariable variable = producedVariables != null ? producedVariables.get(producedVariableIndex) : null;
+ producedVariableIndex++;
+ return variable;
+ }
+
+ /**
+ * Pushdown field access expressions and array access expressions down
+ */
+ private void pushValueAccessExpression(Mutable<ILogicalExpression> exprRef, LogicalVariable producedVar) {
+ final ILogicalExpression expr = exprRef.getValue();
+ if (skipPushdown(expr)) {
+ return;
+ }
+
+ final AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+
+ if (isSuccessfullyPushedDown(funcExpr, producedVar)) {
+ //We successfully pushed down the value access function
+ return;
+ }
+
+ //Check nested arguments if contains any pushable value access
+ pushValueAccessExpressionArg(funcExpr.getArguments());
+ }
+
+ /**
+ * Check if we can pushdown an expression. Also, unregister a variable if we found that a common expression value is
+ * required in its entirety.
+ */
+ private boolean skipPushdown(ILogicalExpression expr) {
+ if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable variable = VariableUtilities.getVariable(expr);
+ unregisterVariableIfNeeded(variable);
+ return true;
+ }
+ return expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL || !builder.containsRegisteredDatasets()
+ || isTypeCheckOnVariable(expr);
+ }
+
+ /**
+ * If the expression is a type-check function on a variable. We should stop as we do not want to unregister
+ * the variable used by the type-check function.
+ * <p>
+ * Example:
+ * SELECT p.personInfo.name
+ * FROM Person p
+ * WHERE p.personInfo IS NOT MISSING;
+ * <p>
+ * Plan:
+ * ...
+ * assign [$$17] <- [$$18.getField(\"name\")]
+ * select (not(is-missing($$18)))
+ * ...
+ * assign [$$18] <- [$$p.getField(\"personInfo\")]
+ * ...
+ * data-scan []<-[$$p] <- test.ParquetDataset project ({personInfo:{name:VALUE}})
+ * <p>
+ * In this case, is-missing($$18) could unregister $$18 since it requires the entire value (personInfo) and we
+ * won't be able to pushdown the access of (personInfo.name). This check would allow (personInfo.name) to be
+ * pushed down to data scan.
+ *
+ * @param expression expression
+ * @return if the function is a type-check function and has a variable argument.
+ */
+ private boolean isTypeCheckOnVariable(ILogicalExpression expression) {
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expression;
+ return TYPE_CHECK_FUNCTIONS.contains(funcExpr.getFunctionIdentifier())
+ && funcExpr.getArguments().get(0).getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE;
+ }
+
+ private void pushValueAccessExpressionArg(List<Mutable<ILogicalExpression>> exprList) {
+ for (Mutable<ILogicalExpression> exprRef : exprList) {
+ /*
+ * We need to set the produced variable as null here as the produced variable will not correspond to the
+ * nested expression.
+ */
+ pushValueAccessExpression(exprRef, null);
+ }
+ }
+
+ private boolean isSuccessfullyPushedDown(AbstractFunctionCallExpression funcExpr, LogicalVariable producedVar) {
+ return SUPPORTED_FUNCTIONS.contains(funcExpr.getFunctionIdentifier())
+ && builder.setSchemaFromExpression(funcExpr, producedVar);
+ }
+
+ private void unregisterVariableIfNeeded(LogicalVariable variable) {
+ if (builder.isVariableRegistered(variable)) {
+ builder.unregisterVariable(variable);
+ }
+ }
+
+ private static Set<FunctionIdentifier> createSupportedArrayFunctions() {
+ return Set.of(BuiltinFunctions.GET_ITEM, BuiltinFunctions.ARRAY_STAR, BuiltinFunctions.SCAN_COLLECTION);
+ }
+
+ private static Set<FunctionIdentifier> createSupportedFunctions() {
+ Set<FunctionIdentifier> supportedFunctions = new HashSet<>();
+ //For objects, only field-access-by-name is supported
+ supportedFunctions.add(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
+ supportedFunctions.addAll(ARRAY_FUNCTIONS);
+ return supportedFunctions;
+ }
+
+ private static Set<FunctionIdentifier> createSupportedTypeCheckFunctions() {
+ return Set.of(BuiltinFunctions.IS_ARRAY, BuiltinFunctions.IS_OBJECT, BuiltinFunctions.IS_ATOMIC,
+ BuiltinFunctions.IS_NUMBER, BuiltinFunctions.IS_BOOLEAN, BuiltinFunctions.IS_STRING,
+ AlgebricksBuiltinFunctions.IS_MISSING, AlgebricksBuiltinFunctions.IS_NULL, BuiltinFunctions.IS_UNKNOWN);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
new file mode 100644
index 0000000..6739384
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/OperatorValueAccessPushdownVisitor.java
@@ -0,0 +1,477 @@
+/*
+ * 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.pushdown;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.DatasetDataSource;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.optimizer.rules.pushdown.schema.RootExpectedSchemaNode;
+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.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * This visitor visits the entire plan and tries to build the information of the required values from all dataset
+ */
+public class OperatorValueAccessPushdownVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+ private final IOptimizationContext context;
+ //Requested schema builder. It is only expected schema not a definite one
+ private final ExpectedSchemaBuilder builder;
+ //To visit every expression in each operator
+ private final ExpressionValueAccessPushdownVisitor pushdownVisitor;
+ //Datasets that allow pushdowns
+ private final Map<LogicalVariable, DataSourceScanOperator> registeredDatasets;
+ //visitedOperators so we do not visit the same operator twice (in case of REPLICATE)
+ private final Set<ILogicalOperator> visitedOperators;
+
+ public OperatorValueAccessPushdownVisitor(IOptimizationContext context) {
+ this.context = context;
+ builder = new ExpectedSchemaBuilder();
+ registeredDatasets = new HashMap<>();
+ pushdownVisitor = new ExpressionValueAccessPushdownVisitor(builder);
+ visitedOperators = new HashSet<>();
+ }
+
+ public void finish() {
+ for (Map.Entry<LogicalVariable, DataSourceScanOperator> scan : registeredDatasets.entrySet()) {
+ scan.getValue().setProjectionInfo(builder.createProjectionInfo(scan.getKey()));
+ }
+ }
+
+ /**
+ * Visit every input of an operator. Then, start pushdown any value expression that the operator has
+ *
+ * @param op the operator to process
+ * @param producedVariables any produced variables by the operator. We only care about the {@link AssignOperator}
+ * and {@link UnnestOperator} variables for now.
+ */
+ private void visitInputs(ILogicalOperator op, List<LogicalVariable> producedVariables) throws AlgebricksException {
+ if (visitedOperators.contains(op)) {
+ return;
+ }
+ for (Mutable<ILogicalOperator> child : op.getInputs()) {
+ child.getValue().accept(this, null);
+ }
+ visitedOperators.add(op);
+ //Initiate the pushdown visitor
+ pushdownVisitor.init(producedVariables);
+ //pushdown any expression the operator has
+ op.acceptExpressionTransform(pushdownVisitor);
+ pushdownVisitor.end();
+ }
+
+ /*
+ * ******************************************************************************
+ * Operators that need to handle special cases
+ * ******************************************************************************
+ */
+
+ @Override
+ public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ if (op.getVariables().isEmpty()) {
+ //If the variables are empty and the next operator is DataSourceScanOperator, then set empty record
+ setEmptyRecord(op.getInputs().get(0).getValue());
+ }
+ return null;
+ }
+
+ /**
+ * From the {@link DataSourceScanOperator}, we need to register the payload variable (record variable) to check
+ * which expression in the plan is using it.
+ */
+ @Override
+ public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ DatasetDataSource datasetDataSource = getDatasetDataSourceIfApplicable(op);
+ if (datasetDataSource != null) {
+ LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(op.getVariables());
+ if (!builder.isVariableRegistered(recordVar)) {
+ /*
+ * This is the first time we see the dataset, and we know we might only need part of the record.
+ * Register the dataset to prepare for value access expression pushdowns.
+ * Initially, we will request the entire record.
+ */
+ builder.registerDataset(recordVar, RootExpectedSchemaNode.ALL_FIELDS_ROOT_NODE);
+ registeredDatasets.put(recordVar, op);
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ if (!op.isGlobal() && isCountConstant(op.getExpressions())) {
+ /*
+ * Optimize the SELECT COUNT(*) case
+ * It is local aggregate and has agg-sql-count function with a constant argument. Set empty record if the
+ * input operator is DataSourceScanOperator
+ */
+ setEmptyRecord(op.getInputs().get(0).getValue());
+ }
+ return null;
+ }
+
+ /*
+ * ******************************************************************************
+ * Helper methods
+ * ******************************************************************************
+ */
+
+ /**
+ * The role of this method is:
+ * 1- Check whether the dataset is an external dataset and allows value access pushdowns
+ * 2- return the actual DatasetDataSource
+ */
+ private DatasetDataSource getDatasetDataSourceIfApplicable(DataSourceScanOperator scan) throws AlgebricksException {
+ DataSource dataSource = (DataSource) scan.getDataSource();
+ if (dataSource == null) {
+ return null;
+ }
+
+ MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
+ DataverseName dataverse = dataSource.getId().getDataverseName();
+ String datasetName = dataSource.getId().getDatasourceName();
+ Dataset dataset = mp.findDataset(dataverse, datasetName);
+
+ //Only external dataset can have pushed down expressions
+ if (dataset == null || dataset.getDatasetType() == DatasetConfig.DatasetType.INTERNAL
+ || dataset.getDatasetType() == DatasetConfig.DatasetType.EXTERNAL && !ExternalDataUtils
+ .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties())) {
+ return null;
+ }
+
+ return (DatasetDataSource) dataSource;
+ }
+
+ /**
+ * If the inputOp is a {@link DataSourceScanOperator}, then set the projected value needed as empty record
+ *
+ * @param inputOp an operator that is potentially a {@link DataSourceScanOperator}
+ * @see #visitAggregateOperator(AggregateOperator, Void)
+ * @see #visitProjectOperator(ProjectOperator, Void)
+ */
+ private void setEmptyRecord(ILogicalOperator inputOp) throws AlgebricksException {
+ if (inputOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator scan = (DataSourceScanOperator) inputOp;
+ DatasetDataSource datasetDataSource = getDatasetDataSourceIfApplicable(scan);
+ if (datasetDataSource != null) {
+ //We know that we only need the count of objects. So return empty objects only
+ LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(scan.getVariables());
+ /*
+ * Set the root node as EMPTY_ROOT_NODE (i.e., no fields will be read from disk). We register the
+ * dataset with EMPTY_ROOT_NODE so that we skip pushdowns on empty node.
+ */
+ builder.registerDataset(recordVar, RootExpectedSchemaNode.EMPTY_ROOT_NODE);
+ }
+ }
+ }
+
+ private boolean isCountConstant(List<Mutable<ILogicalExpression>> expressions) {
+ if (expressions.size() != 1) {
+ return false;
+ }
+ ILogicalExpression expression = expressions.get(0).getValue();
+ if (expression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expression;
+ FunctionIdentifier fid = funcExpr.getFunctionIdentifier();
+ return BuiltinFunctions.SQL_COUNT.equals(fid)
+ && funcExpr.getArguments().get(0).getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT;
+ }
+
+ private void visitSubplans(List<ILogicalPlan> nestedPlans) throws AlgebricksException {
+ for (ILogicalPlan plan : nestedPlans) {
+ for (Mutable<ILogicalOperator> root : plan.getRoots()) {
+ visitInputs(root.getValue());
+ }
+ }
+ }
+
+ /*
+ * ******************************************************************************
+ * Pushdown when possible for each operator
+ * ******************************************************************************
+ */
+
+ @Override
+ public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op, op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ visitSubplans(op.getNestedPlans());
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op, op.getVariables());
+ return null;
+ }
+
+ @Override
+ public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
+ public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ visitSubplans(op.getNestedPlans());
+ return null;
+ }
+
+ @Override
+ public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitDelegateOperator(DelegateOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitIntersectOperator(IntersectOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg)
+ throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitTokenizeOperator(TokenizeOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ return null;
+ }
+
+ @Override
+ public Void visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
+ visitInputs(op);
+ visitSubplans(op.getNestedPlans());
+ return null;
+ }
+
+ private void visitInputs(ILogicalOperator op) throws AlgebricksException {
+ visitInputs(op, null);
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AbstractComplexExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AbstractComplexExpectedSchemaNode.java
new file mode 100644
index 0000000..5da5149
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AbstractComplexExpectedSchemaNode.java
@@ -0,0 +1,97 @@
+/*
+ * 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.pushdown.schema;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractComplexExpectedSchemaNode extends AbstractExpectedSchemaNode {
+
+ AbstractComplexExpectedSchemaNode(AbstractComplexExpectedSchemaNode parent, SourceLocation sourceLocation,
+ String functionName) {
+ super(parent, sourceLocation, functionName);
+ }
+
+ @Override
+ public IExpectedSchemaNode replaceIfNeeded(ExpectedSchemaNodeType expectedNodeType, SourceLocation sourceLocation,
+ String functionName) {
+ //If no change is required, return the same node
+ IExpectedSchemaNode node = this;
+ if (expectedNodeType == ExpectedSchemaNodeType.ANY) {
+ /*
+ * We want to fall back to ANY. This could happen if we needed one nested value in one expression but
+ * another expression, the entire node is needed. So, we fall back to ANY and remove any information
+ * about the nested value. For example:
+ * SELECT t.hashtags[*].text, t.hashtags
+ * FROM Tweets t
+ * In this case, we first saw (t.hashtags[*].text), but the next expression (t.hashtags) requested
+ * the entire hashtags. So, the expected type for hashtags should be ANY
+ */
+ node = new AnyExpectedSchemaNode(getParent(), getSourceLocation(), getFunctionName());
+ getParent().replaceChild(this, node);
+ } else if (expectedNodeType != getType()) {
+ /*
+ * We need to change the type to UNION, as the same value was accessed as an ARRAY and as an OBJECT.
+ * This is possible if we have heterogeneous value access in the query.
+ */
+
+ //Create UNION node and its parent is the parent of this
+ UnionExpectedSchemaNode unionSchemaNode =
+ new UnionExpectedSchemaNode(getParent(), getSourceLocation(), getFunctionName());
+
+ //Add this as a child of UNION
+ unionSchemaNode.addChild(this);
+ /*
+ * Replace the reference of this in its parent with the union node
+ * Before: parent --> this
+ * After: parent --> UNION --> this
+ */
+ getParent().replaceChild(this, unionSchemaNode);
+ /*
+ * Set the parent of this to union
+ * Before: oldParent <-- this
+ * After: oldParent <-- UNION <-- this
+ */
+ setParent(unionSchemaNode);
+ /*
+ * Add the new child with the expected type to union
+ * Before: UNION <-- this
+ * After: UNION <-- (this, newChild)
+ */
+ unionSchemaNode.createChild(expectedNodeType, sourceLocation, functionName);
+ node = unionSchemaNode;
+ }
+ return node;
+ }
+
+ protected abstract void replaceChild(IExpectedSchemaNode oldNode, IExpectedSchemaNode newNode);
+
+ public static AbstractComplexExpectedSchemaNode createNestedNode(ExpectedSchemaNodeType type,
+ AbstractComplexExpectedSchemaNode parent, SourceLocation sourceLocation, String functionName) {
+ switch (type) {
+ case ARRAY:
+ return new ArrayExpectedSchemaNode(parent, sourceLocation, functionName);
+ case OBJECT:
+ return new ObjectExpectedSchemaNode(parent, sourceLocation, functionName);
+ case UNION:
+ return new UnionExpectedSchemaNode(parent, sourceLocation, functionName);
+ default:
+ throw new IllegalStateException(type + " is not nested or unknown");
+ }
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AbstractExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AbstractExpectedSchemaNode.java
new file mode 100644
index 0000000..3c6cc95
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AbstractExpectedSchemaNode.java
@@ -0,0 +1,53 @@
+/*
+ * 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.pushdown.schema;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+abstract class AbstractExpectedSchemaNode implements IExpectedSchemaNode {
+ private AbstractComplexExpectedSchemaNode parent;
+ private final SourceLocation sourceLocation;
+ private final String functionName;
+
+ AbstractExpectedSchemaNode(AbstractComplexExpectedSchemaNode parent, SourceLocation sourceLocation,
+ String functionName) {
+ this.parent = parent;
+ this.sourceLocation = sourceLocation;
+ this.functionName = functionName;
+ }
+
+ @Override
+ public final AbstractComplexExpectedSchemaNode getParent() {
+ return parent;
+ }
+
+ @Override
+ public final SourceLocation getSourceLocation() {
+ return sourceLocation;
+ }
+
+ @Override
+ public final String getFunctionName() {
+ return functionName;
+ }
+
+ protected void setParent(AbstractComplexExpectedSchemaNode parent) {
+ this.parent = parent;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AnyExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AnyExpectedSchemaNode.java
new file mode 100644
index 0000000..834a405
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/AnyExpectedSchemaNode.java
@@ -0,0 +1,58 @@
+/*
+ * 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.pushdown.schema;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class AnyExpectedSchemaNode extends AbstractExpectedSchemaNode {
+
+ public AnyExpectedSchemaNode(AbstractComplexExpectedSchemaNode parent, SourceLocation sourceLocation,
+ String functionName) {
+ super(parent, sourceLocation, functionName);
+ }
+
+ @Override
+ public IExpectedSchemaNode replaceIfNeeded(ExpectedSchemaNodeType expectedNodeType, SourceLocation sourceLocation,
+ String functionName) {
+ if (expectedNodeType == ExpectedSchemaNodeType.ANY) {
+ return this;
+ }
+ /*
+ * ANY node is typeless (i.e., we do not know what is the possible type of ANY node) when we created it.
+ * However, now the query says it is (possibly) a nested value. We know that because there is a field
+ * access expression or an array access expression on that node. So, we should replace the ANY node to
+ * the given nested type.
+ */
+ AbstractComplexExpectedSchemaNode parent = getParent();
+ AbstractComplexExpectedSchemaNode nestedNode = AbstractComplexExpectedSchemaNode
+ .createNestedNode(expectedNodeType, parent, getSourceLocation(), functionName);
+ parent.replaceChild(this, nestedNode);
+ return nestedNode;
+ }
+
+ @Override
+ public ExpectedSchemaNodeType getType() {
+ return ExpectedSchemaNodeType.ANY;
+ }
+
+ @Override
+ public <R, T> R accept(IExpectedSchemaNodeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ArrayExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ArrayExpectedSchemaNode.java
new file mode 100644
index 0000000..b000dc8
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ArrayExpectedSchemaNode.java
@@ -0,0 +1,57 @@
+/*
+ * 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.pushdown.schema;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class ArrayExpectedSchemaNode extends AbstractComplexExpectedSchemaNode {
+ private IExpectedSchemaNode child;
+
+ ArrayExpectedSchemaNode(AbstractComplexExpectedSchemaNode parent, SourceLocation sourceLocation,
+ String functionName) {
+ super(parent, sourceLocation, functionName);
+ }
+
+ @Override
+ public ExpectedSchemaNodeType getType() {
+ return ExpectedSchemaNodeType.ARRAY;
+ }
+
+ public IExpectedSchemaNode getChild() {
+ return child;
+ }
+
+ public void addChild(IExpectedSchemaNode child) {
+ this.child = child;
+ }
+
+ @Override
+ public <R, T> R accept(IExpectedSchemaNodeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public void replaceChild(IExpectedSchemaNode oldNode, IExpectedSchemaNode newNode) {
+ if (oldNode != child) {
+ //this should not happen
+ throw new IllegalStateException("Node " + oldNode.getType() + " is not a child");
+ }
+ child = newNode;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ExpectedSchemaNodeType.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ExpectedSchemaNodeType.java
new file mode 100644
index 0000000..e6c88b4
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ExpectedSchemaNodeType.java
@@ -0,0 +1,26 @@
+/*
+ * 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.pushdown.schema;
+
+public enum ExpectedSchemaNodeType {
+ ARRAY,
+ OBJECT,
+ UNION,
+ ANY
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/IExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/IExpectedSchemaNode.java
new file mode 100644
index 0000000..454e32e
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/IExpectedSchemaNode.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.pushdown.schema;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * An interface of the expected schema of value access expressions in a query.
+ */
+public interface IExpectedSchemaNode {
+
+ /**
+ * @return node type
+ */
+ ExpectedSchemaNodeType getType();
+
+ /**
+ * @return source location of the value access
+ */
+ SourceLocation getSourceLocation();
+
+ /**
+ * @return value access function name
+ */
+ String getFunctionName();
+
+ /**
+ * @return the parent of a node
+ */
+ AbstractComplexExpectedSchemaNode getParent();
+
+ /**
+ * For visiting a node
+ *
+ * @param visitor schema node visitor
+ * @param arg any argument might be needed by the visitor
+ * @param <R> return type
+ * @param <T> argument type
+ */
+ <R, T> R accept(IExpectedSchemaNodeVisitor<R, T> visitor, T arg);
+
+ /**
+ * Replace a node from one type to another
+ * Example:
+ * - {@link ExpectedSchemaNodeType#ANY} to {@link ExpectedSchemaNodeType#OBJECT}
+ * - {@link ExpectedSchemaNodeType#OBJECT} to {@link ExpectedSchemaNodeType#UNION}
+ *
+ * @param expectedNodeType what is the other expected type
+ * @param sourceLocation source location of the value access
+ * @param functionName function name as in {@link FunctionIdentifier#getName()}
+ * @see AbstractComplexExpectedSchemaNode#replaceIfNeeded(ExpectedSchemaNodeType, SourceLocation, String)
+ * @see UnionExpectedSchemaNode#replaceIfNeeded(ExpectedSchemaNodeType, SourceLocation, String)
+ */
+ IExpectedSchemaNode replaceIfNeeded(ExpectedSchemaNodeType expectedNodeType, SourceLocation sourceLocation,
+ String functionName);
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/IExpectedSchemaNodeVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/IExpectedSchemaNodeVisitor.java
new file mode 100644
index 0000000..754503a
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/IExpectedSchemaNodeVisitor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.pushdown.schema;
+
+public interface IExpectedSchemaNodeVisitor<R, T> {
+
+ R visit(RootExpectedSchemaNode node, T arg);
+
+ R visit(ObjectExpectedSchemaNode node, T arg);
+
+ R visit(ArrayExpectedSchemaNode node, T arg);
+
+ R visit(UnionExpectedSchemaNode node, T arg);
+
+ R visit(AnyExpectedSchemaNode node, T arg);
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ObjectExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ObjectExpectedSchemaNode.java
new file mode 100644
index 0000000..02307c0
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/ObjectExpectedSchemaNode.java
@@ -0,0 +1,71 @@
+/*
+ * 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.pushdown.schema;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class ObjectExpectedSchemaNode extends AbstractComplexExpectedSchemaNode {
+ private final Map<String, IExpectedSchemaNode> children;
+
+ ObjectExpectedSchemaNode(AbstractComplexExpectedSchemaNode parent, SourceLocation sourceLocation,
+ String functionName) {
+ super(parent, sourceLocation, functionName);
+ children = new HashMap<>();
+ }
+
+ public Set<Map.Entry<String, IExpectedSchemaNode>> getChildren() {
+ return children.entrySet();
+ }
+
+ public IExpectedSchemaNode addChild(String fieldName, IExpectedSchemaNode child) {
+ children.put(fieldName, child);
+ return child;
+ }
+
+ @Override
+ public ExpectedSchemaNodeType getType() {
+ return ExpectedSchemaNodeType.OBJECT;
+ }
+
+ @Override
+ public <R, T> R accept(IExpectedSchemaNodeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public void replaceChild(IExpectedSchemaNode oldNode, IExpectedSchemaNode newNode) {
+ String key = null;
+ for (Map.Entry<String, IExpectedSchemaNode> child : children.entrySet()) {
+ if (child.getValue() == oldNode) {
+ key = child.getKey();
+ break;
+ }
+ }
+
+ if (key == null) {
+ //this should not happen
+ throw new IllegalStateException("Node " + oldNode.getType() + " is not a child");
+ }
+ children.replace(key, newNode);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/RootExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/RootExpectedSchemaNode.java
new file mode 100644
index 0000000..1a978f3
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/RootExpectedSchemaNode.java
@@ -0,0 +1,70 @@
+/*
+ * 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.pushdown.schema;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class RootExpectedSchemaNode extends ObjectExpectedSchemaNode {
+ //Root with zero fields
+ private static final int EMPTY_ROOT = 0;
+ //Root with the entire fields
+ private static final int ALL_FIELDS_ROOT = 1;
+ //Root with custom fields
+ private static final int CLIPPED_ROOT = 2;
+ public static final RootExpectedSchemaNode ALL_FIELDS_ROOT_NODE = new RootExpectedSchemaNode(ALL_FIELDS_ROOT);
+ public static final RootExpectedSchemaNode EMPTY_ROOT_NODE = new RootExpectedSchemaNode(EMPTY_ROOT);
+
+ private final int rootType;
+
+ RootExpectedSchemaNode() {
+ this(CLIPPED_ROOT);
+ }
+
+ private RootExpectedSchemaNode(int rootType) {
+ super(null, null, null);
+ this.rootType = rootType;
+ }
+
+ public int getRootType() {
+ return rootType;
+ }
+
+ @Override
+ public AbstractComplexExpectedSchemaNode replaceIfNeeded(ExpectedSchemaNodeType expectedNodeType,
+ SourceLocation sourceLocation, String functionName) {
+ if (rootType == ALL_FIELDS_ROOT) {
+ //ALL_FIELDS_ROOT. Return a new CLIPPED_ROOT root
+ return new RootExpectedSchemaNode();
+ }
+ return this;
+ }
+
+ @Override
+ public <R, T> R accept(IExpectedSchemaNodeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
+
+ public boolean isEmpty() {
+ return rootType == EMPTY_ROOT;
+ }
+
+ public boolean isAllFields() {
+ return rootType == ALL_FIELDS_ROOT;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/UnionExpectedSchemaNode.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/UnionExpectedSchemaNode.java
new file mode 100644
index 0000000..3a675b8
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/schema/UnionExpectedSchemaNode.java
@@ -0,0 +1,88 @@
+/*
+ * 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.pushdown.schema;
+
+import java.util.EnumMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class UnionExpectedSchemaNode extends AbstractComplexExpectedSchemaNode {
+ private final Map<ExpectedSchemaNodeType, AbstractComplexExpectedSchemaNode> children;
+
+ protected UnionExpectedSchemaNode(AbstractComplexExpectedSchemaNode parent, SourceLocation sourceLocation,
+ String functionName) {
+ super(parent, sourceLocation, functionName);
+ children = new EnumMap<>(ExpectedSchemaNodeType.class);
+ }
+
+ /**
+ * A UNION type must have both ARRAY and OBJECT when first created - the only possible values. Thus, we cannot
+ * replace a child of a UNION type to ANY. We can only replace the union itself to ANY.
+ */
+ @Override
+ protected void replaceChild(IExpectedSchemaNode oldChildNode, IExpectedSchemaNode newChildNode) {
+ throw new UnsupportedOperationException("Cannot replace a child of UNION");
+ }
+
+ protected void addChild(AbstractComplexExpectedSchemaNode node) {
+ children.put(node.getType(), node);
+ }
+
+ public void createChild(ExpectedSchemaNodeType nodeType, SourceLocation sourceLocation, String functionName) {
+ children.computeIfAbsent(nodeType, k -> createNestedNode(k, this, sourceLocation, functionName));
+ }
+
+ public AbstractComplexExpectedSchemaNode getChild(ExpectedSchemaNodeType type) {
+ return children.get(type);
+ }
+
+ public Set<Map.Entry<ExpectedSchemaNodeType, AbstractComplexExpectedSchemaNode>> getChildren() {
+ return children.entrySet();
+ }
+
+ @Override
+ public ExpectedSchemaNodeType getType() {
+ return ExpectedSchemaNodeType.UNION;
+ }
+
+ @Override
+ public <R, T> R accept(IExpectedSchemaNodeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
+
+ /**
+ * We override this method to handle heterogeneous values while UNION exists. We do not need to create another
+ * UNION type - we simply return this. In case we want to fallback to ANY node, we call the super method.
+ *
+ * @param expectedNodeType the expected type
+ * @param sourceLocation source location of the value access
+ * @param functionName function name of the expression
+ * @return ANY or this
+ */
+ @Override
+ public IExpectedSchemaNode replaceIfNeeded(ExpectedSchemaNodeType expectedNodeType, SourceLocation sourceLocation,
+ String functionName) {
+ if (expectedNodeType == ExpectedSchemaNodeType.ANY) {
+ return super.replaceIfNeeded(expectedNodeType, sourceLocation, functionName);
+ }
+ return this;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 1b7b7d2..4ac44b4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -743,7 +743,7 @@
gbyOp.getNestedPlans().add(nestedPlan);
OperatorManipulationUtil.computeTypeEnvironmentBottomUp(gbyOp, context);
- return op;
+ return gbyOp;
}
private ILogicalOperator visitMultiInputOperator(ILogicalOperator op) throws AlgebricksException {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 8326f85..d4b7853 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -21,6 +21,7 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
import org.apache.commons.lang3.mutable.Mutable;
@@ -33,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
@@ -89,6 +91,7 @@
// The target Nts operator.
private final ILogicalOperator targetNts;
+ private final IAlgebricksConstantValue leftOuterMissingValue;
// The live variables in <code>subplanInputOperator</code> to enforce.
private final Set<LogicalVariable> liveVarsFromSubplanInput = new HashSet<>();
@@ -112,13 +115,16 @@
* the input operator to the target SubplanOperator
* @param nts
* the NestedTupleSourceOperator to be replaced by <code>subplanInputOperator</code>
+ * @param leftOuterMissingValue
* @throws AlgebricksException
*/
public InlineLeftNtsInSubplanJoinFlatteningVisitor(IOptimizationContext context,
- ILogicalOperator subplanInputOperator, ILogicalOperator nts) throws AlgebricksException {
+ ILogicalOperator subplanInputOperator, ILogicalOperator nts, IAlgebricksConstantValue leftOuterMissingValue)
+ throws AlgebricksException {
this.context = context;
this.subplanInputOperator = subplanInputOperator;
this.targetNts = nts;
+ this.leftOuterMissingValue = Objects.requireNonNull(leftOuterMissingValue);
VariableUtilities.getSubplanLocalLiveVariables(subplanInputOperator, liveVarsFromSubplanInput);
}
@@ -192,7 +198,7 @@
AbstractBinaryJoinOperator returnOp = op;
// After rewriting, the original inner join should become an left outer join.
if (rewritten) {
- returnOp = new LeftOuterJoinOperator(op.getCondition());
+ returnOp = new LeftOuterJoinOperator(op.getCondition(), leftOuterMissingValue);
returnOp.setSourceLocation(op.getSourceLocation());
returnOp.getInputs().addAll(op.getInputs());
injectNullCheckVars(returnOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
index ae9a886..6220d06 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
@@ -43,6 +43,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
@@ -59,6 +60,7 @@
import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -300,16 +302,19 @@
return changedAndVarMap;
}
- /**
+ IAlgebricksConstantValue leftOuterMissingValue = ConstantExpression.MISSING.getValue();
+
+ /*
* Apply the special join-based rewriting.
*/
- Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> result = applySpecialFlattening(opRef, context);
+ Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> result =
+ applySpecialFlattening(opRef, context, leftOuterMissingValue);
if (!result.first) {
- /**
+ /*
* If the special join-based rewriting does not apply, apply the general
* rewriting which blindly inlines all NTSs.
*/
- result = applyGeneralFlattening(opRef, context);
+ result = applyGeneralFlattening(opRef, context, leftOuterMissingValue);
}
LinkedHashMap<LogicalVariable, LogicalVariable> returnedMap = new LinkedHashMap<>();
// Adds variable mappings from input operators.
@@ -359,7 +364,8 @@
}
private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applyGeneralFlattening(
- Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
if (!SubplanFlatteningUtil.containsOperators(subplanOp, EnumSet.of(LogicalOperatorTag.DATASOURCESCAN,
LogicalOperatorTag.INNERJOIN, LogicalOperatorTag.LEFTOUTERJOIN))) {
@@ -437,8 +443,8 @@
} else {
joinExpr = joinPredicates.size() > 0 ? joinPredicates.get(0).getValue() : ConstantExpression.TRUE;
}
- LeftOuterJoinOperator leftOuterJoinOp =
- new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef, rightInputOpRef);
+ LeftOuterJoinOperator leftOuterJoinOp = new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef,
+ rightInputOpRef, leftOuterMissingValue);
leftOuterJoinOp.setSourceLocation(sourceLoc);
OperatorManipulationUtil.computeTypeEnvironmentBottomUp(rightInputOp, context);
context.computeAndSetTypeEnvironmentForOperator(leftOuterJoinOp);
@@ -481,7 +487,7 @@
lowestAggregateRefInSubplan.getValue().getInputs().add(currentOpRef);
}
- // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
+ // Adds a select operator into the nested plan for group-by to remove tuples with MISSING on {@code assignVar},
// i.e., subplan input tuples that are filtered out within a subplan.
VariableReferenceExpression assignVarRef = new VariableReferenceExpression(assignVar);
assignVarRef.setSourceLocation(sourceLoc);
@@ -489,14 +495,15 @@
List<Mutable<ILogicalExpression>> args = new ArrayList<>();
args.add(filterVarExpr);
List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
- ScalarFunctionCallExpression isMissingExpr =
- new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+ ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue)),
+ args);
isMissingExpr.setSourceLocation(sourceLoc);
argsForNotFunction.add(new MutableObject<>(isMissingExpr));
ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction);
notExpr.setSourceLocation(sourceLoc);
- SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr), false, null);
+ SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr));
selectOp.setSourceLocation(sourceLoc);
currentOpRef.getValue().getInputs().add(new MutableObject<>(selectOp));
@@ -524,7 +531,8 @@
}
private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applySpecialFlattening(
- Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
SourceLocation sourceLoc = subplanOp.getSourceLocation();
Mutable<ILogicalOperator> inputOpRef = subplanOp.getInputs().get(0);
@@ -554,8 +562,8 @@
Set<LogicalVariable> liveVars = new HashSet<>();
VariableUtilities.getLiveVariables(inputOp, liveVars);
- Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> notNullVarsAndTopJoinRef =
- SubplanFlatteningUtil.inlineLeftNtsInSubplanJoin(subplanOp, context, newPrimaryKeyFd);
+ Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> notNullVarsAndTopJoinRef = SubplanFlatteningUtil
+ .inlineLeftNtsInSubplanJoin(subplanOp, context, newPrimaryKeyFd, leftOuterMissingValue);
if (notNullVarsAndTopJoinRef.first == null) {
inputOpRef.setValue(inputOpBackup);
return new Pair<>(false, replacedVarMap);
@@ -588,7 +596,7 @@
groupbyOp.getInputs().add(new MutableObject<>(topJoinRef.getValue()));
if (!notNullVars.isEmpty()) {
- // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
+ // Adds a select operator into the nested plan for group-by to remove tuples with MISSING on {@code assignVar},
// i.e., subplan input tuples that are filtered out within a subplan.
List<Mutable<ILogicalExpression>> nullCheckExprRefs = new ArrayList<>();
for (LogicalVariable notNullVar : notNullVars) {
@@ -598,8 +606,8 @@
List<Mutable<ILogicalExpression>> args = new ArrayList<>();
args.add(filterVarExpr);
List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
- ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
- FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+ ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(FunctionUtil
+ .getFunctionInfo(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue)), args);
isMissingExpr.setSourceLocation(sourceLoc);
argsForNotFunction.add(new MutableObject<>(isMissingExpr));
ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
@@ -616,7 +624,7 @@
} else {
selectExprRef = nullCheckExprRefs.get(0);
}
- SelectOperator selectOp = new SelectOperator(selectExprRef, false, null);
+ SelectOperator selectOp = new SelectOperator(selectExprRef);
selectOp.setSourceLocation(sourceLoc);
topJoinRef.setValue(selectOp);
NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
@@ -630,9 +638,9 @@
topJoinRef.setValue(ntsOp);
}
opRef.setValue(groupbyOp);
+ VariableUtilities.substituteVariables(groupbyOp, result.second, context);
OperatorManipulationUtil.computeTypeEnvironmentBottomUp(groupbyOp, context);
- VariableUtilities.substituteVariables(groupbyOp, result.second, context);
replacedVarMap.putAll(result.second);
return new Pair<>(true, replacedVarMap);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
index e3c7d69..916d75d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
@@ -101,13 +102,14 @@
* the optimization context
* @param extraPrimaryKeyFd
* extra primary key dependency that needs to be added to the context before performing the rewrite
+ * @param leftOuterMissingValue
* @return A set of variables used for further null-checks, i.e., variables indicating
* whether a tuple produced by a transformed left outer join is a non-match;
* a reference to the top join operator in the nested subplan.
*/
public static Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> inlineLeftNtsInSubplanJoin(
- SubplanOperator subplanOp, IOptimizationContext context, FunctionalDependency extraPrimaryKeyFd)
- throws AlgebricksException {
+ SubplanOperator subplanOp, IOptimizationContext context, FunctionalDependency extraPrimaryKeyFd,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
Pair<Boolean, ILogicalOperator> applicableAndNtsToRewrite =
SubplanFlatteningUtil.isQualifiedForSpecialFlattening(subplanOp);
if (!applicableAndNtsToRewrite.first) {
@@ -120,8 +122,8 @@
ILogicalOperator qualifiedNts = applicableAndNtsToRewrite.second;
ILogicalOperator subplanInputOp = subplanOp.getInputs().get(0).getValue();
- InlineLeftNtsInSubplanJoinFlatteningVisitor specialVisitor =
- new InlineLeftNtsInSubplanJoinFlatteningVisitor(context, subplanInputOp, qualifiedNts);
+ InlineLeftNtsInSubplanJoinFlatteningVisitor specialVisitor = new InlineLeftNtsInSubplanJoinFlatteningVisitor(
+ context, subplanInputOp, qualifiedNts, leftOuterMissingValue);
// Rewrites the query plan.
Mutable<ILogicalOperator> topRef = subplanOp.getNestedPlans().get(0).getRoots().get(0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
index 672cb25..5781d6c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
@@ -18,29 +18,14 @@
*/
package org.apache.asterix.optimizer.rules.util;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.asterix.common.annotations.RangeAnnotation;
-import org.apache.asterix.om.types.ATypeTag;
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.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
public class AsterixJoinUtils {
- private static final int LEFT = 0;
- private static final int RIGHT = 1;
-
private AsterixJoinUtils() {
}
@@ -49,37 +34,14 @@
if (!topLevelOp) {
return;
}
- ILogicalExpression conditionLE = op.getCondition().getValue();
- if (conditionLE.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ ILogicalExpression joinCondition = op.getCondition().getValue();
+ if (joinCondition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return;
}
- List<LogicalVariable> sideLeft = new ArrayList<>(1);
- List<LogicalVariable> sideRight = new ArrayList<>(1);
- List<LogicalVariable> varsLeft = op.getInputs().get(LEFT).getValue().getSchema();
- List<LogicalVariable> varsRight = op.getInputs().get(RIGHT).getValue().getSchema();
- AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) conditionLE;
- FunctionIdentifier fi =
- IntervalJoinUtils.isIntervalJoinCondition(fexp, varsLeft, varsRight, sideLeft, sideRight, LEFT, RIGHT);
- if (fi == null) {
- return;
+
+ boolean intervalJoinChosen = IntervalJoinUtils.tryIntervalJoinAssignment(op, context, joinCondition, 0, 1);
+ if (!intervalJoinChosen) {
+ SpatialJoinUtils.trySpatialJoinAssignment(op, context, joinCondition, 0, 1);
}
- RangeAnnotation rangeAnnotation = IntervalJoinUtils.findRangeAnnotation(fexp);
- if (rangeAnnotation == null) {
- return;
- }
- //Check RangeMap type
- RangeMap rangeMap = rangeAnnotation.getRangeMap();
- if (rangeMap.getTag(0, 0) != ATypeTag.DATETIME.serialize() && rangeMap.getTag(0, 0) != ATypeTag.DATE.serialize()
- && rangeMap.getTag(0, 0) != ATypeTag.TIME.serialize()) {
- IWarningCollector warningCollector = context.getWarningCollector();
- if (warningCollector.shouldWarn()) {
- warningCollector.warn(Warning.of(op.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
- "Date, DateTime, and Time are only range hints types supported for interval joins"));
- }
- return;
- }
- IntervalPartitions intervalPartitions =
- IntervalJoinUtils.createIntervalPartitions(op, fi, sideLeft, sideRight, rangeMap, context, LEFT, RIGHT);
- IntervalJoinUtils.setSortMergeIntervalJoinOp(op, fi, sideLeft, sideRight, context, intervalPartitions);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
index e10f9fb..340c841 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
@@ -32,6 +32,7 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.operators.joins.interval.utils.AfterIntervalJoinUtilFactory;
import org.apache.asterix.runtime.operators.joins.interval.utils.BeforeIntervalJoinUtilFactory;
import org.apache.asterix.runtime.operators.joins.interval.utils.CoveredByIntervalJoinUtilFactory;
@@ -60,6 +61,8 @@
import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty.PartitioningType;
import org.apache.hyracks.algebricks.core.algebra.properties.IntervalColumn;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
public class IntervalJoinUtils {
@@ -76,6 +79,44 @@
INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_OVERLAPS, BuiltinFunctions.INTERVAL_OVERLAPPED_BY);
}
+ protected static boolean tryIntervalJoinAssignment(AbstractBinaryJoinOperator op, IOptimizationContext context,
+ ILogicalExpression joinCondition, int left, int right) throws AlgebricksException {
+ List<LogicalVariable> sideLeft = new ArrayList<>(1);
+ List<LogicalVariable> sideRight = new ArrayList<>(1);
+ List<LogicalVariable> varsLeft = op.getInputs().get(left).getValue().getSchema();
+ List<LogicalVariable> varsRight = op.getInputs().get(right).getValue().getSchema();
+
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) joinCondition;
+ FunctionIdentifier fi = IntervalJoinUtils.isIntervalJoinCondition(funcExpr, varsLeft, varsRight, sideLeft,
+ sideRight, left, right);
+ if (fi == null) {
+ return false;
+ }
+
+ // Existing workflow for interval merge join
+ RangeAnnotation rangeAnnotation = IntervalJoinUtils.findRangeAnnotation(funcExpr);
+ if (rangeAnnotation == null) {
+ return false;
+ }
+
+ //Check RangeMap type
+ RangeMap rangeMap = rangeAnnotation.getRangeMap();
+ if (rangeMap.getTag(0, 0) != ATypeTag.DATETIME.serialize() && rangeMap.getTag(0, 0) != ATypeTag.DATE.serialize()
+ && rangeMap.getTag(0, 0) != ATypeTag.TIME.serialize()) {
+ IWarningCollector warningCollector = context.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(op.getSourceLocation(),
+ org.apache.hyracks.api.exceptions.ErrorCode.INAPPLICABLE_HINT,
+ "Date, DateTime, and Time are only range hints types supported for interval joins"));
+ }
+ return false;
+ }
+ IntervalPartitions intervalPartitions =
+ IntervalJoinUtils.createIntervalPartitions(op, fi, sideLeft, sideRight, rangeMap, context, left, right);
+ IntervalJoinUtils.setSortMergeIntervalJoinOp(op, fi, sideLeft, sideRight, context, intervalPartitions);
+ return true;
+ }
+
protected static RangeAnnotation findRangeAnnotation(AbstractFunctionCallExpression fexp) {
return fexp.getAnnotation(RangeAnnotation.class);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
deleted file mode 100644
index 6efef16..0000000
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
+++ /dev/null
@@ -1,424 +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.
- */
-package org.apache.asterix.optimizer.rules.util;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.om.base.AInt16;
-import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.base.AInt8;
-import org.apache.asterix.om.constants.AsterixConstantValue;
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
-import org.apache.hyracks.api.exceptions.SourceLocation;
-
-/**
- * For use in writing a "throwaway" branch which removes NTS and subplan operators. The result of this invocation is to
- * be given to the {@code IntroduceSelectAccessMethodRule} to check if an array index can be used.
- * <br>
- * If we are given the pattern (an existential query):
- * <pre>
- * SELECT_1(some variable)
- * SUBPLAN_1 -------------------------------|
- * (parent branch input) AGGREGATE(NON-EMPTY-STREAM)
- * SELECT_2(some predicate)
- * (UNNEST/ASSIGN)*
- * UNNEST(on variable)
- * NESTED-TUPLE-SOURCE
- * </pre>
- * We return the following branch:
- * <pre>
- * SELECT_2(some predicate)
- * (UNNEST/ASSIGN)*
- * UNNEST(on variable)
- * (parent branch input)
- * </pre>
- *
- * If we are given the pattern (a universal query):
- * <pre>
- * SELECT_1(some variable AND array is not empty)
- * SUBPLAN_1 -------------------------------|
- * (parent branch input) AGGREGATE(EMPTY-STREAM)
- * SELECT_2(NOT(IF-MISSING-OR-NULL(some predicate)))
- * (UNNEST/ASSIGN)*
- * UNNEST(on variable)
- * NESTED-TUPLE-SOURCE
- * </pre>
- * We return the following branch:
- * <pre>
- * SELECT_2(some predicate) <--- removed the NOT(IF-MISSING-OR-NULL(...))!
- * (UNNEST/ASSIGN)*
- * UNNEST(on variable)
- * (parent branch input)
- * </pre>
- *
- * In the case of nested-subplans, we return a copy of the innermost SELECT followed by all relevant UNNEST/ASSIGNs.
- */
-public class SelectInSubplanBranchCreator {
- private final static List<IAlgebricksConstantValue> zerosAsAsterixConstants =
- Arrays.asList(new IAlgebricksConstantValue[] { new AsterixConstantValue(new AInt64(0)),
- new AsterixConstantValue(new AInt32(0)), new AsterixConstantValue(new AInt16((short) 0)),
- new AsterixConstantValue(new AInt8((byte) 0)) });
-
- private IOptimizationContext context;
- private SourceLocation sourceLocation;
- private SelectOperator originalSelectRoot;
-
- /**
- * Create a new branch to match that of the form:
- *
- * <pre>
- * SELECT (...)
- * (UNNEST/ASSIGN)*
- * UNNEST
- * ...
- * </pre>
- *
- * Operators are *created* here, rather than just reconnected from the original branch.
- */
- public SelectOperator createSelect(SelectOperator originalSelect, IOptimizationContext context)
- throws AlgebricksException {
- // Reset our context.
- this.sourceLocation = originalSelect.getSourceLocation();
- this.originalSelectRoot = originalSelect;
- this.context = context;
-
- // We expect a) a SUBPLAN as input to this SELECT, and b) our SELECT to be conditioning on a variable.
- if (!originalSelect.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)
- || !originalSelect.getCondition().getValue().getExpressionTag().equals(LogicalExpressionTag.VARIABLE)) {
- return null;
- }
- LogicalVariable originalSelectVar =
- ((VariableReferenceExpression) originalSelect.getCondition().getValue()).getVariableReference();
-
- // Additionally, verify that the subplan does not produce any other variable other than the SELECT var above.
- SubplanOperator subplanOperator = (SubplanOperator) originalSelect.getInputs().get(0).getValue();
- List<LogicalVariable> subplanProducedVars = new ArrayList<>();
- VariableUtilities.getProducedVariables(subplanOperator, subplanProducedVars);
- if (subplanProducedVars.size() != 1 || !subplanProducedVars.get(0).equals(originalSelectVar)) {
- return null;
- }
-
- return traverseSubplanBranch(subplanOperator);
- }
-
- /**
- * To undo this process is to return what was passed to us at {@code createSelect} time.
- */
- public SelectOperator getOriginalSelect() {
- return originalSelectRoot;
- }
-
- private SelectOperator traverseSubplanBranch(SubplanOperator subplanOperator) throws AlgebricksException {
- // We only expect one plan, and one root.
- if (subplanOperator.getNestedPlans().size() > 1
- || subplanOperator.getNestedPlans().get(0).getRoots().size() > 1) {
- return null;
- }
-
- // This root of our "subplan" should always be an aggregate.
- ILogicalOperator workingSubplanRoot = subplanOperator.getNestedPlans().get(0).getRoots().get(0).getValue();
- AggregateOperator workingSubplanRootAsAggregate;
- if (!workingSubplanRoot.getOperatorTag().equals(LogicalOperatorTag.AGGREGATE)) {
- return null;
- }
- workingSubplanRootAsAggregate = (AggregateOperator) workingSubplanRoot;
-
- // Try to find a SELECT that we can optimize (i.e. has a function call).
- SelectOperator optimizableSelect = null;
- for (Mutable<ILogicalOperator> opInput : workingSubplanRoot.getInputs()) {
- ILogicalOperator subplanOrSelect = findSubplanOrSelect(opInput.getValue());
- if (subplanOrSelect == null) {
- return null;
-
- } else if (subplanOrSelect.getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)) {
- optimizableSelect = traverseSubplanBranch((SubplanOperator) subplanOrSelect);
-
- } else {
- optimizableSelect = (SelectOperator) subplanOrSelect;
- break;
- }
- }
- if (optimizableSelect == null) {
- return null;
- }
-
- // We have found a SELECT with a variable. Create a copy, and set this to our rewrite root.
- SelectOperator newSelectOperator = new SelectOperator(optimizableSelect.getCondition(),
- optimizableSelect.getRetainMissing(), optimizableSelect.getMissingPlaceholderVariable());
-
- // Ensure that this SELECT represents a predicate for an existential query, and is a query we can optimize.
- newSelectOperator = normalizeSelectCondition(workingSubplanRootAsAggregate, newSelectOperator,
- subplanOperator.getInputs().get(0).getValue());
- if (newSelectOperator == null) {
- return null;
- }
- newSelectOperator.setSourceLocation(sourceLocation);
- newSelectOperator.setExecutionMode(optimizableSelect.getExecutionMode());
-
- // Follow this SELECT to the root of our nested-plan branch (i.e. the NESTED-TUPLE-SOURCE).
- ILogicalOperator workingOriginalOperator = optimizableSelect, workingNewOperator = newSelectOperator;
- UnnestOperator bottommostNewUnnest = null;
- while (!workingOriginalOperator.getOperatorTag().equals(LogicalOperatorTag.NESTEDTUPLESOURCE)) {
- if (workingOriginalOperator.getInputs().isEmpty()) {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
- workingSubplanRoot.getSourceLocation(),
- "NESTED-TUPLE-SOURCE expected in nested plan branch," + " but not found.");
- }
-
- switch (workingOriginalOperator.getOperatorTag()) {
- case UNNEST:
- UnnestOperator originalUnnest = (UnnestOperator) workingOriginalOperator;
- UnnestOperator newUnnest =
- new UnnestOperator(originalUnnest.getVariable(), originalUnnest.getExpressionRef());
- newUnnest.setSourceLocation(sourceLocation);
- workingNewOperator.getInputs().add(new MutableObject<>(newUnnest));
- workingNewOperator = newUnnest;
- bottommostNewUnnest = (UnnestOperator) workingNewOperator;
- break;
-
- case ASSIGN:
- AssignOperator originalAssign = (AssignOperator) workingOriginalOperator;
- AssignOperator newAssign =
- new AssignOperator(originalAssign.getVariables(), originalAssign.getExpressions());
- newAssign.setSourceLocation(sourceLocation);
- workingNewOperator.getInputs().add(new MutableObject<>(newAssign));
- workingNewOperator = newAssign;
- break;
-
- case SUBPLAN:
- // TODO (GLENN): Work on supporting nested universal quantification.
- return null;
-
- case AGGREGATE:
- case SELECT:
- break;
-
- default:
- return null;
- }
-
- workingOriginalOperator = workingOriginalOperator.getInputs().get(0).getValue();
- }
-
- // If we are working with universal quantification, then we must also check whether or not we have a conjunct
- // that asserts that the array should also be non-empty.
- if (isUniversalQuantification(workingSubplanRootAsAggregate)
- && !isArrayNonEmptyConjunctIncluded(bottommostNewUnnest, subplanOperator)) {
- return null;
- }
-
- // We have added everything we need in our nested-plan branch. Now, connect the input of our SUBPLAN to our
- // current working branch.
- bottommostNewUnnest.getInputs().addAll(subplanOperator.getInputs());
- OperatorManipulationUtil.computeTypeEnvironmentBottomUp(newSelectOperator, context);
-
- return newSelectOperator;
- }
-
- private boolean isUniversalQuantification(AggregateOperator workingSubplanRoot) throws CompilationException {
- AggregateFunctionCallExpression aggregateFunctionCallExpression =
- (AggregateFunctionCallExpression) workingSubplanRoot.getExpressions().get(0).getValue();
- if (aggregateFunctionCallExpression.getFunctionIdentifier().equals(BuiltinFunctions.EMPTY_STREAM)) {
- return true;
- } else if (aggregateFunctionCallExpression.getFunctionIdentifier().equals(BuiltinFunctions.NON_EMPTY_STREAM)) {
- return false;
- } else {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, workingSubplanRoot.getSourceLocation(),
- "Unexpected aggregate function: " + aggregateFunctionCallExpression.getFunctionIdentifier());
- }
- }
-
- private boolean isArrayNonEmptyConjunctIncluded(UnnestOperator firstUnnestInNTS, SubplanOperator subplanOperator) {
- UnnestingFunctionCallExpression unnestFunction =
- (UnnestingFunctionCallExpression) firstUnnestInNTS.getExpressionRef().getValue();
- VariableReferenceExpression unnestVarExpr =
- (VariableReferenceExpression) unnestFunction.getArguments().get(0).getValue();
- LogicalVariable arrayVariable = unnestVarExpr.getVariableReference();
-
- // TODO (GLENN): The SELECT directly below the SUBPLAN is the only operator we explore. This does not cover
- // all predicates where the array may be non-empty (say, having an existential predicate located after this
- // subplan).
- if (!subplanOperator.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
- return false;
- }
- SelectOperator subplanInputOperator = (SelectOperator) subplanOperator.getInputs().get(0).getValue();
- ILogicalExpression selectCondExpr = subplanInputOperator.getCondition().getValue();
- List<Mutable<ILogicalExpression>> conjunctsFromSelect = new ArrayList<>();
- if (selectCondExpr.splitIntoConjuncts(conjunctsFromSelect)) {
- // We have a collection of conjuncts. Analyze each conjunct w/ a function.
- for (Mutable<ILogicalExpression> mutableConjuct : conjunctsFromSelect) {
- ILogicalExpression workingConjunct = mutableConjuct.getValue();
- if (workingConjunct.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
- && analyzeConjunctForArrayNonEmptiness(arrayVariable,
- (ScalarFunctionCallExpression) workingConjunct)) {
- return true;
- }
- }
-
- // No such conjunct found.
- return false;
- }
-
- if (!selectCondExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
- return false;
- }
- return analyzeConjunctForArrayNonEmptiness(arrayVariable, (ScalarFunctionCallExpression) selectCondExpr);
- }
-
- private boolean analyzeConjunctForArrayNonEmptiness(LogicalVariable arrayVariable,
- ScalarFunctionCallExpression workingSelectCondExpr) {
- // Handle the conjunct: LEN(arrayVar) > 0
- if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.GT)) {
- ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
- ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
-
- if (firstArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
- && ((ScalarFunctionCallExpression) firstArg).getFunctionIdentifier().equals(BuiltinFunctions.LEN)) {
- ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) firstArg;
- List<LogicalVariable> usedVariables = new ArrayList<>();
- lenFunction.getUsedVariables(usedVariables);
-
- return usedVariables.contains(arrayVariable)
- && secondArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
- && zerosAsAsterixConstants.contains(((ConstantExpression) secondArg).getValue());
- }
- }
-
- // Handle the conjunct: 0 < LEN(arrayVar)
- else if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.LT)) {
- ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
- ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
-
- if (secondArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
- && ((ScalarFunctionCallExpression) secondArg).getFunctionIdentifier()
- .equals(BuiltinFunctions.LEN)) {
- ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) secondArg;
- List<LogicalVariable> usedVariables = new ArrayList<>();
- lenFunction.getUsedVariables(usedVariables);
-
- return usedVariables.contains(arrayVariable)
- && firstArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
- && zerosAsAsterixConstants.contains(((ConstantExpression) firstArg).getValue());
- }
- }
-
- // TODO (GLENN): Handle the cases 1) where the arrayVar is explicitly indexed, 2) the NOT function.
- return false;
- }
-
- private SelectOperator normalizeSelectCondition(AggregateOperator aggregateOperator, SelectOperator selectOperator,
- ILogicalOperator subplanInputOperator) throws AlgebricksException {
- // The purpose of this function is to remove the NOT(IF-MISSING-OR-NULL(...)) functions for a universal
- // quantification query. The {@code ArrayBTreeAccessMethod} does not recognize the former as optimizable
- // functions, so we remove them here. This SELECT will never make it to the final query plan (after the
- // {@code IntroduceSelectAccessMethodRule}), which allows us to get away with this logically incorrect branch.
- if (!isUniversalQuantification(aggregateOperator)) {
- // We are working with an existential quantification query. Do not modify the SELECT.
- return selectOperator;
-
- } else {
- // We are working with a universal quantification query.
- if (!subplanInputOperator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
- return null;
- }
-
- ScalarFunctionCallExpression notFunction =
- (ScalarFunctionCallExpression) selectOperator.getCondition().getValue();
- if (!notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
- return selectOperator;
- }
-
- ScalarFunctionCallExpression ifMissingOrNullFunction =
- (ScalarFunctionCallExpression) notFunction.getArguments().get(0).getValue();
- if (!ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
- return selectOperator;
- }
-
- Mutable<ILogicalExpression> newSelectCondition =
- new MutableObject<>(ifMissingOrNullFunction.getArguments().get(0).getValue().cloneExpression());
- return new SelectOperator(newSelectCondition, selectOperator.getRetainMissing(),
- selectOperator.getMissingPlaceholderVariable());
-
- }
- }
-
- private ILogicalOperator findSubplanOrSelect(ILogicalOperator operator) {
- // We are trying to find a SELECT operator with a function call that is not "NOT(IF-MISSING-OR-NULL(...))".
- if (operator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
- SelectOperator selectOperator = (SelectOperator) operator;
- ILogicalExpression selectCondExpr = selectOperator.getCondition().getValue();
- if (selectCondExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-
- // Follow the chain of NOT(IF-MISSING-OR-NULL(...)) to see if we have a variable at the end.
- ScalarFunctionCallExpression notFunction =
- (ScalarFunctionCallExpression) selectOperator.getCondition().getValue();
- if (notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
- ScalarFunctionCallExpression ifMissingOrNullFunction =
- (ScalarFunctionCallExpression) notFunction.getArguments().get(0).getValue();
- if (ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
- ILogicalExpression finalExpr = ifMissingOrNullFunction.getArguments().get(0).getValue();
- if (finalExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
- return selectOperator;
- }
- }
-
- } else {
- return selectOperator;
- }
- }
- } else if (operator.getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)) {
- // We have found an additional SUBPLAN branch to explore. Recurse w/ caller function.
- return operator;
- }
-
- // No matching operator found. Recurse on current operator input.
- if (operator.getInputs().isEmpty()) {
- return null;
- } else {
- return findSubplanOrSelect(operator.getInputs().get(0).getValue());
- }
- }
-}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SpatialJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SpatialJoinUtils.java
new file mode 100644
index 0000000..ca997e0
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SpatialJoinUtils.java
@@ -0,0 +1,656 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.asterix.algebra.operators.physical.SpatialJoinPOperator;
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.IntersectSpatialJoinUtilFactory;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.ListSet;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.BroadcastExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.NestedLoopJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.UnnestPOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class SpatialJoinUtils {
+
+ private static final int DEFAULT_ROWS = 100;
+ private static final int DEFAULT_COLUMNS = 100;
+
+ protected static boolean trySpatialJoinAssignment(AbstractBinaryJoinOperator op, IOptimizationContext context,
+ ILogicalExpression joinCondition, int left, int right) throws AlgebricksException {
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) joinCondition;
+ // Check if the join condition contains spatial join
+ AbstractFunctionCallExpression spatialJoinFuncExpr = null;
+ // Maintain conditions which is not spatial_intersect in the join condition
+ List<Mutable<ILogicalExpression>> conditionExprs = new ArrayList<>();
+
+ if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.AND)) {
+ // Join condition contains multiple conditions along with spatial_intersect
+ List<Mutable<ILogicalExpression>> inputExprs = funcExpr.getArguments();
+ if (inputExprs.size() == 0) {
+ return false;
+ }
+
+ boolean spatialIntersectExists = false;
+ for (Mutable<ILogicalExpression> exp : inputExprs) {
+ AbstractFunctionCallExpression funcCallExp = (AbstractFunctionCallExpression) exp.getValue();
+ if (funcCallExp.getFunctionIdentifier().equals(BuiltinFunctions.SPATIAL_INTERSECT)) {
+ spatialJoinFuncExpr = funcCallExp;
+ spatialIntersectExists = true;
+ } else {
+ // Retain the other conditions
+ conditionExprs.add(exp);
+ }
+ }
+
+ if (!spatialIntersectExists) {
+ return false;
+ }
+ } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.SPATIAL_INTERSECT)) {
+ // Join condition is spatial_intersect only
+ spatialJoinFuncExpr = funcExpr;
+ } else {
+ return false;
+ }
+
+ // Apply the PBSM join algorithm with/without hint
+ SpatialJoinAnnotation spatialJoinAnn = spatialJoinFuncExpr.getAnnotation(SpatialJoinAnnotation.class);
+ return SpatialJoinUtils.updateJoinPlan(op, spatialJoinFuncExpr, conditionExprs, spatialJoinAnn, context, left,
+ right);
+ }
+
+ private static void setSpatialJoinOp(AbstractBinaryJoinOperator op, List<LogicalVariable> keysLeftBranch,
+ List<LogicalVariable> keysRightBranch, IOptimizationContext context) throws AlgebricksException {
+ ISpatialJoinUtilFactory isjuf = new IntersectSpatialJoinUtilFactory();
+ op.setPhysicalOperator(new SpatialJoinPOperator(op.getJoinKind(),
+ AbstractJoinPOperator.JoinPartitioningType.PAIRWISE, keysLeftBranch, keysRightBranch,
+ context.getPhysicalOptimizationConfig().getMaxFramesForJoin(), isjuf));
+ op.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ }
+
+ private static LogicalVariable injectSpatialTileUnnestOperator(IOptimizationContext context,
+ Mutable<ILogicalOperator> op, LogicalVariable unnestVar, Mutable<ILogicalExpression> unnestMBRExpr,
+ int numRows, int numColumns) throws AlgebricksException {
+ SourceLocation srcLoc = op.getValue().getSourceLocation();
+ LogicalVariable tileIdVar = context.newVar();
+ VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+ unnestVarRef.setSourceLocation(srcLoc);
+ UnnestingFunctionCallExpression spatialTileFuncExpr = new UnnestingFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SPATIAL_TILE),
+ new MutableObject<>(unnestVarRef), unnestMBRExpr,
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numRows)))),
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numColumns)))));
+ spatialTileFuncExpr.setSourceLocation(srcLoc);
+ UnnestOperator unnestOp = new UnnestOperator(tileIdVar, new MutableObject<>(spatialTileFuncExpr));
+ unnestOp.setPhysicalOperator(new UnnestPOperator());
+ unnestOp.setSourceLocation(srcLoc);
+ unnestOp.getInputs().add(new MutableObject<>(op.getValue()));
+ context.computeAndSetTypeEnvironmentForOperator(unnestOp);
+ unnestOp.recomputeSchema();
+ op.setValue(unnestOp);
+
+ return tileIdVar;
+ }
+
+ protected static boolean updateJoinPlan(AbstractBinaryJoinOperator op,
+ AbstractFunctionCallExpression spatialJoinFuncExpr, List<Mutable<ILogicalExpression>> conditionExprs,
+ SpatialJoinAnnotation spatialJoinAnn, IOptimizationContext context, int left, int right)
+ throws AlgebricksException {
+ // Extracts spatial intersect function's arguments
+ List<Mutable<ILogicalExpression>> spatialJoinArgs = spatialJoinFuncExpr.getArguments();
+ if (spatialJoinArgs.size() != 2) {
+ return false;
+ }
+
+ ILogicalExpression spatialJoinLeftArg = spatialJoinArgs.get(left).getValue();
+ ILogicalExpression spatialJoinRightArg = spatialJoinArgs.get(right).getValue();
+
+ // Left and right arguments of the spatial_intersect function should be variables
+ if (spatialJoinLeftArg.getExpressionTag() != LogicalExpressionTag.VARIABLE
+ || spatialJoinRightArg.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return false;
+ }
+
+ // We only apply this rule if the arguments of spatial_intersect are ARectangle
+ IVariableTypeEnvironment typeEnvironment = op.computeInputTypeEnvironment(context);
+ IAType leftType = (IAType) context.getExpressionTypeComputer().getType(spatialJoinLeftArg,
+ context.getMetadataProvider(), typeEnvironment);
+ IAType rightType = (IAType) context.getExpressionTypeComputer().getType(spatialJoinRightArg,
+ context.getMetadataProvider(), typeEnvironment);
+ if ((leftType.getTypeTag() != BuiltinType.ARECTANGLE.getTypeTag())
+ || (rightType.getTypeTag() != BuiltinType.ARECTANGLE.getTypeTag())) {
+ return false;
+ }
+
+ // Gets both input branches of the spatial join.
+ Mutable<ILogicalOperator> leftInputOp = op.getInputs().get(left);
+ Mutable<ILogicalOperator> rightInputOp = op.getInputs().get(right);
+
+ // Extract left and right variable of the predicate
+ LogicalVariable spatialJoinVar0 = ((VariableReferenceExpression) spatialJoinLeftArg).getVariableReference();
+ LogicalVariable spatialJoinVar1 = ((VariableReferenceExpression) spatialJoinRightArg).getVariableReference();
+
+ LogicalVariable leftInputVar;
+ LogicalVariable rightInputVar;
+ Collection<LogicalVariable> liveVars = new HashSet<>();
+ VariableUtilities.getLiveVariables(leftInputOp.getValue(), liveVars);
+ if (liveVars.contains(spatialJoinVar0)) {
+ leftInputVar = spatialJoinVar0;
+ rightInputVar = spatialJoinVar1;
+ } else {
+ leftInputVar = spatialJoinVar1;
+ rightInputVar = spatialJoinVar0;
+ }
+
+ // If the hint is not provided, the intersection MBR of two inputs will be computed on the run time
+ if (spatialJoinAnn == null) {
+ buildSpatialJoinPlanWithDynamicMbr(op, context, spatialJoinFuncExpr, conditionExprs, leftInputOp,
+ rightInputOp, leftInputVar, rightInputVar);
+ } else {
+ buildSpatialJoinPlanWithStaticMbr(op, context, spatialJoinFuncExpr, conditionExprs, leftInputOp,
+ rightInputOp, leftInputVar, rightInputVar, spatialJoinAnn);
+ }
+
+ return true;
+ }
+
+ private static void buildSpatialJoinPlanWithStaticMbr(AbstractBinaryJoinOperator op, IOptimizationContext context,
+ AbstractFunctionCallExpression spatialJoinFuncExpr, List<Mutable<ILogicalExpression>> conditionExprs,
+ Mutable<ILogicalOperator> leftInputOp, Mutable<ILogicalOperator> rightInputOp, LogicalVariable leftInputVar,
+ LogicalVariable rightInputVar, SpatialJoinAnnotation spatialJoinAnn) throws AlgebricksException {
+ Mutable<ILogicalExpression> leftIntersectionMBRExpr = createRectangleExpression(spatialJoinAnn);
+ Mutable<ILogicalExpression> rightIntersectionMBRExpr = createRectangleExpression(spatialJoinAnn);
+ Mutable<ILogicalExpression> referencePointTestMBRExpr = createRectangleExpression(spatialJoinAnn);
+ int numRows = spatialJoinAnn.getNumRows();
+ int numColumns = spatialJoinAnn.getNumColumns();
+
+ // Inject unnest operator to add tile ID to the left and right branch of the join operator
+ LogicalVariable leftTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, leftInputOp,
+ leftInputVar, leftIntersectionMBRExpr, numRows, numColumns);
+ LogicalVariable rightTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, rightInputOp,
+ rightInputVar, rightIntersectionMBRExpr, numRows, numColumns);
+
+ // The reference point test condition is considered as a part of spatial join conditions if a hint is provided.
+ ScalarFunctionCallExpression referenceIdEquiJoinCondition =
+ createReferencePointTestCondition(op, referencePointTestMBRExpr, leftTileIdVar, rightTileIdVar,
+ leftInputVar, rightInputVar, numRows, numColumns);
+ conditionExprs.add(new MutableObject<>(referenceIdEquiJoinCondition));
+
+ conditionExprs.add(new MutableObject<>(spatialJoinFuncExpr));
+
+ ScalarFunctionCallExpression updatedJoinCondition = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND), conditionExprs);
+ updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+ Mutable<ILogicalExpression> joinConditionRef = op.getCondition();
+ joinConditionRef.setValue(updatedJoinCondition);
+
+ List<LogicalVariable> keysLeftBranch = new ArrayList<>();
+ keysLeftBranch.add(leftTileIdVar);
+ keysLeftBranch.add(leftInputVar);
+
+ List<LogicalVariable> keysRightBranch = new ArrayList<>();
+ keysRightBranch.add(rightTileIdVar);
+ keysRightBranch.add(rightInputVar);
+
+ SpatialJoinUtils.setSpatialJoinOp(op, keysLeftBranch, keysRightBranch, context);
+ }
+
+ private static void buildSpatialJoinPlanWithDynamicMbr(AbstractBinaryJoinOperator op, IOptimizationContext context,
+ AbstractFunctionCallExpression spatialJoinFuncExpr, List<Mutable<ILogicalExpression>> conditionExprs,
+ Mutable<ILogicalOperator> leftInputOp, Mutable<ILogicalOperator> rightInputOp, LogicalVariable leftInputVar,
+ LogicalVariable rightInputVar) throws AlgebricksException {
+ // Add a dynamic workflow to compute MBR of the left branch
+ Triple<MutableObject<ILogicalOperator>, List<LogicalVariable>, MutableObject<ILogicalOperator>> leftMBRCalculator =
+ createDynamicMBRCalculator(op, context, leftInputOp, leftInputVar);
+ MutableObject<ILogicalOperator> leftGlobalAgg = leftMBRCalculator.first;
+ List<LogicalVariable> leftGlobalAggResultVars = leftMBRCalculator.second;
+ MutableObject<ILogicalOperator> leftExchToJoinOpRef = leftMBRCalculator.third;
+ LogicalVariable leftMBRVar = leftGlobalAggResultVars.get(0);
+
+ // Add a dynamic workflow to compute MBR of the right branch
+ Triple<MutableObject<ILogicalOperator>, List<LogicalVariable>, MutableObject<ILogicalOperator>> rightMBRCalculator =
+ createDynamicMBRCalculator(op, context, rightInputOp, rightInputVar);
+ MutableObject<ILogicalOperator> rightGlobalAgg = rightMBRCalculator.first;
+ List<LogicalVariable> rightGlobalAggResultVars = rightMBRCalculator.second;
+ MutableObject<ILogicalOperator> rightExchToJoinOpRef = rightMBRCalculator.third;
+ LogicalVariable rightMBRVar = rightGlobalAggResultVars.get(0);
+
+ // Join the left and right union MBR
+ Mutable<ILogicalExpression> trueCondition =
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE)));
+ InnerJoinOperator unionMBRJoinOp = new InnerJoinOperator(trueCondition, leftGlobalAgg, rightGlobalAgg);
+ unionMBRJoinOp.setSourceLocation(op.getSourceLocation());
+ unionMBRJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(AbstractBinaryJoinOperator.JoinKind.INNER,
+ AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+ MutableObject<ILogicalOperator> unionMBRJoinOpRef = new MutableObject<>(unionMBRJoinOp);
+ unionMBRJoinOp.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(unionMBRJoinOp);
+
+ // Compute the intersection rectangle of left MBR and right MBR
+ List<Mutable<ILogicalExpression>> getIntersectionFuncInputExprs = new ArrayList<>();
+ getIntersectionFuncInputExprs.add(new MutableObject<>(new VariableReferenceExpression(leftMBRVar)));
+ getIntersectionFuncInputExprs.add(new MutableObject<>(new VariableReferenceExpression(rightMBRVar)));
+ ScalarFunctionCallExpression getIntersectionFuncExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.GET_INTERSECTION),
+ getIntersectionFuncInputExprs);
+ getIntersectionFuncExpr.setSourceLocation(op.getSourceLocation());
+
+ Mutable<ILogicalExpression> intersectionMBRExpr = new MutableObject<>(getIntersectionFuncExpr);
+ LogicalVariable intersectionMBR = context.newVar();
+ AbstractLogicalOperator intersectionMBRAssignOperator =
+ new AssignOperator(intersectionMBR, intersectionMBRExpr);
+ intersectionMBRAssignOperator.setSourceLocation(op.getSourceLocation());
+ intersectionMBRAssignOperator.setExecutionMode(op.getExecutionMode());
+ intersectionMBRAssignOperator.setPhysicalOperator(new AssignPOperator());
+ intersectionMBRAssignOperator.getInputs().add(new MutableObject<>(unionMBRJoinOpRef.getValue()));
+ context.computeAndSetTypeEnvironmentForOperator(intersectionMBRAssignOperator);
+ intersectionMBRAssignOperator.recomputeSchema();
+ MutableObject<ILogicalOperator> intersectionMBRAssignOperatorRef =
+ new MutableObject<>(intersectionMBRAssignOperator);
+
+ // Replicate the union MBR to left and right nested loop join(NLJ) operator, and another NLJ for reference point test
+ ReplicateOperator intersectionMBRReplicateOperator =
+ createReplicateOperator(intersectionMBRAssignOperatorRef, context, op.getSourceLocation(), 3);
+
+ // Replicate union MBR to the left branch
+ ExchangeOperator exchMBRToJoinOpLeft =
+ createBroadcastExchangeOp(intersectionMBRReplicateOperator, context, op.getSourceLocation());
+ MutableObject<ILogicalOperator> exchMBRToJoinOpLeftRef = new MutableObject<>(exchMBRToJoinOpLeft);
+ Pair<LogicalVariable, Mutable<ILogicalOperator>> createLeftAssignProjectOperatorResult =
+ createAssignProjectOperator(op, intersectionMBR, intersectionMBRReplicateOperator,
+ exchMBRToJoinOpLeftRef, context);
+ LogicalVariable leftIntersectionMBRVar = createLeftAssignProjectOperatorResult.getFirst();
+ Mutable<ILogicalOperator> leftIntersectionMBRRef = createLeftAssignProjectOperatorResult.getSecond();
+
+ // Replicate union MBR to the right branch
+ ExchangeOperator exchMBRToJoinOpRight =
+ createBroadcastExchangeOp(intersectionMBRReplicateOperator, context, op.getSourceLocation());
+ MutableObject<ILogicalOperator> exchMBRToJoinOpRightRef = new MutableObject<>(exchMBRToJoinOpRight);
+ Pair<LogicalVariable, Mutable<ILogicalOperator>> createRightAssignProjectOperatorResult =
+ createAssignProjectOperator(op, intersectionMBR, intersectionMBRReplicateOperator,
+ exchMBRToJoinOpRightRef, context);
+ LogicalVariable rightIntersectionMBRVar = createRightAssignProjectOperatorResult.getFirst();
+ Mutable<ILogicalOperator> rightIntersectionMBRRef = createRightAssignProjectOperatorResult.getSecond();
+
+ // Replicate union MBR to the right branch of a later Nested Loop Join reference point test
+ ExchangeOperator exchMBRToReferencePointTestJoinOp =
+ createBroadcastExchangeOp(intersectionMBRReplicateOperator, context, op.getSourceLocation());
+ MutableObject<ILogicalOperator> exchMBRToReferencePointTestJoinOpRef =
+ new MutableObject<>(exchMBRToReferencePointTestJoinOp);
+
+ // Add left Join (TRUE)
+ Mutable<ILogicalExpression> leftTrueCondition =
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE)));
+ InnerJoinOperator leftJoinOp =
+ new InnerJoinOperator(leftTrueCondition, leftExchToJoinOpRef, leftIntersectionMBRRef);
+ leftJoinOp.setSourceLocation(op.getSourceLocation());
+ leftJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(AbstractBinaryJoinOperator.JoinKind.INNER,
+ AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+ MutableObject<ILogicalOperator> leftJoinRef = new MutableObject<>(leftJoinOp);
+ leftJoinOp.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(leftJoinOp);
+ leftInputOp.setValue(leftJoinRef.getValue());
+
+ // Add right Join (TRUE)
+ Mutable<ILogicalExpression> rightTrueCondition =
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE)));
+ InnerJoinOperator rightJoinOp =
+ new InnerJoinOperator(rightTrueCondition, rightExchToJoinOpRef, rightIntersectionMBRRef);
+ rightJoinOp.setSourceLocation(op.getSourceLocation());
+ rightJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(AbstractBinaryJoinOperator.JoinKind.INNER,
+ AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+ MutableObject<ILogicalOperator> rightJoinRef = new MutableObject<>(rightJoinOp);
+ rightJoinOp.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(rightJoinOp);
+ rightInputOp.setValue(rightJoinRef.getValue());
+
+ Mutable<ILogicalExpression> leftIntersectionMBRExpr =
+ new MutableObject<>(new VariableReferenceExpression(leftIntersectionMBRVar));
+ Mutable<ILogicalExpression> rightIntersectionMBRExpr =
+ new MutableObject<>(new VariableReferenceExpression(rightIntersectionMBRVar));
+ Mutable<ILogicalExpression> referencePointTestMBRExpr =
+ new MutableObject<>(new VariableReferenceExpression(intersectionMBR));
+
+ // Inject unnest operator to add tile ID to the left and right branch of the join operator
+ LogicalVariable leftTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, leftInputOp,
+ leftInputVar, leftIntersectionMBRExpr, DEFAULT_ROWS, DEFAULT_COLUMNS);
+ LogicalVariable rightTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, rightInputOp,
+ rightInputVar, rightIntersectionMBRExpr, DEFAULT_ROWS, DEFAULT_COLUMNS);
+
+ // Reference point test condition will be used as the condition of a Nested Loop Join operator after the
+ // spatial join operator. This design allow us to use the union MBR (or summary of the join) efficiently,
+ // instead of propagate this variable via Hyracks context or data flow.
+ ScalarFunctionCallExpression referenceIdEquiJoinCondition =
+ createReferencePointTestCondition(op, referencePointTestMBRExpr, leftTileIdVar, rightTileIdVar,
+ leftInputVar, rightInputVar, DEFAULT_ROWS, DEFAULT_COLUMNS);
+
+ conditionExprs.add(new MutableObject<>(spatialJoinFuncExpr));
+
+ ScalarFunctionCallExpression updatedJoinCondition;
+ if (conditionExprs.size() > 1) {
+ updatedJoinCondition = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND), conditionExprs);
+ updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+ } else {
+ updatedJoinCondition = (ScalarFunctionCallExpression) spatialJoinFuncExpr;
+ }
+ Mutable<ILogicalExpression> joinConditionRef = op.getCondition();
+ joinConditionRef.setValue(updatedJoinCondition);
+
+ List<LogicalVariable> keysLeftBranch = new ArrayList<>();
+ keysLeftBranch.add(leftTileIdVar);
+ keysLeftBranch.add(leftInputVar);
+
+ List<LogicalVariable> keysRightBranch = new ArrayList<>();
+ keysRightBranch.add(rightTileIdVar);
+ keysRightBranch.add(rightInputVar);
+
+ InnerJoinOperator spatialJoinOp =
+ new InnerJoinOperator(new MutableObject<>(updatedJoinCondition), leftInputOp, rightInputOp);
+ spatialJoinOp.setSourceLocation(op.getSourceLocation());
+ SpatialJoinUtils.setSpatialJoinOp(spatialJoinOp, keysLeftBranch, keysRightBranch, context);
+ spatialJoinOp.setSchema(op.getSchema());
+ context.computeAndSetTypeEnvironmentForOperator(spatialJoinOp);
+
+ Mutable<ILogicalOperator> opRef = new MutableObject<>(op);
+ Mutable<ILogicalOperator> spatialJoinOpRef = new MutableObject<>(spatialJoinOp);
+
+ InnerJoinOperator referencePointTestJoinOp =
+ new InnerJoinOperator(new MutableObject<>(referenceIdEquiJoinCondition), spatialJoinOpRef,
+ exchMBRToReferencePointTestJoinOpRef);
+ referencePointTestJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(
+ AbstractBinaryJoinOperator.JoinKind.INNER, AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+ MutableObject<ILogicalOperator> referencePointTestJoinOpRef = new MutableObject<>(referencePointTestJoinOp);
+ referencePointTestJoinOp.setSourceLocation(op.getSourceLocation());
+ context.computeAndSetTypeEnvironmentForOperator(referencePointTestJoinOp);
+ referencePointTestJoinOp.recomputeSchema();
+ opRef.setValue(referencePointTestJoinOpRef.getValue());
+ op.getInputs().clear();
+ op.getInputs().addAll(referencePointTestJoinOp.getInputs());
+ op.setPhysicalOperator(referencePointTestJoinOp.getPhysicalOperator());
+ op.getCondition().setValue(referencePointTestJoinOp.getCondition().getValue());
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ op.recomputeSchema();
+ }
+
+ private static ScalarFunctionCallExpression createReferencePointTestCondition(AbstractBinaryJoinOperator op,
+ Mutable<ILogicalExpression> referencePointTestMBRExpr, LogicalVariable leftTileIdVar,
+ LogicalVariable rightTileIdVar, LogicalVariable leftInputVar, LogicalVariable rightInputVar, int numRows,
+ int numColumns) {
+ // Compute reference tile ID
+ ScalarFunctionCallExpression referenceTileId = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.REFERENCE_TILE),
+ new MutableObject<>(new VariableReferenceExpression(leftInputVar)),
+ new MutableObject<>(new VariableReferenceExpression(rightInputVar)), referencePointTestMBRExpr,
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numRows)))),
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numColumns)))),
+ new MutableObject<>(new VariableReferenceExpression(rightTileIdVar)));
+ referenceTileId.setSourceLocation(op.getSourceLocation());
+
+ ScalarFunctionCallExpression referenceIdEquiJoinCondition =
+ new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.EQ),
+ new MutableObject<>(new VariableReferenceExpression(leftTileIdVar)),
+ new MutableObject<>(referenceTileId));
+ referenceIdEquiJoinCondition.setSourceLocation(op.getSourceLocation());
+
+ return referenceIdEquiJoinCondition;
+ }
+
+ private static Pair<LogicalVariable, Mutable<ILogicalOperator>> createAssignProjectOperator(
+ AbstractBinaryJoinOperator op, LogicalVariable inputVar, ReplicateOperator replicateOperator,
+ MutableObject<ILogicalOperator> exchMBRToForwardRef, IOptimizationContext context)
+ throws AlgebricksException {
+ LogicalVariable newFinalMbrVar = context.newVar();
+ List<LogicalVariable> finalMBRLiveVars = new ArrayList<>();
+ finalMBRLiveVars.add(newFinalMbrVar);
+ ListSet<LogicalVariable> finalMBRLiveVarsSet = new ListSet<>();
+ finalMBRLiveVarsSet.add(newFinalMbrVar);
+
+ Mutable<ILogicalExpression> finalMBRExpr = new MutableObject<>(new VariableReferenceExpression(inputVar));
+ AbstractLogicalOperator assignOperator = new AssignOperator(newFinalMbrVar, finalMBRExpr);
+ assignOperator.setSourceLocation(op.getSourceLocation());
+ assignOperator.setExecutionMode(replicateOperator.getExecutionMode());
+ assignOperator.setPhysicalOperator(new AssignPOperator());
+ AbstractLogicalOperator projectOperator = new ProjectOperator(finalMBRLiveVars);
+ projectOperator.setSourceLocation(op.getSourceLocation());
+ projectOperator.setPhysicalOperator(new StreamProjectPOperator());
+ projectOperator.setExecutionMode(replicateOperator.getExecutionMode());
+ assignOperator.getInputs().add(exchMBRToForwardRef);
+ projectOperator.getInputs().add(new MutableObject<ILogicalOperator>(assignOperator));
+
+ context.computeAndSetTypeEnvironmentForOperator(assignOperator);
+ assignOperator.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(projectOperator);
+ projectOperator.recomputeSchema();
+ Mutable<ILogicalOperator> projectOperatorRef = new MutableObject<>(projectOperator);
+
+ return new Pair<>(newFinalMbrVar, projectOperatorRef);
+ }
+
+ private static ReplicateOperator createReplicateOperator(Mutable<ILogicalOperator> inputOperator,
+ IOptimizationContext context, SourceLocation sourceLocation, int outputArity) throws AlgebricksException {
+ ReplicateOperator replicateOperator = new ReplicateOperator(outputArity);
+ replicateOperator.setPhysicalOperator(new ReplicatePOperator());
+ replicateOperator.setSourceLocation(sourceLocation);
+ replicateOperator.getInputs().add(new MutableObject<>(inputOperator.getValue()));
+ OperatorManipulationUtil.setOperatorMode(replicateOperator);
+ replicateOperator.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(replicateOperator);
+ return replicateOperator;
+ }
+
+ private static ExchangeOperator createOneToOneExchangeOp(ReplicateOperator replicateOperator,
+ IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+ ExchangeOperator exchangeOperator = new ExchangeOperator();
+ exchangeOperator.setSourceLocation(sourceLocation);
+ exchangeOperator.setPhysicalOperator(new OneToOneExchangePOperator());
+ replicateOperator.getOutputs().add(new MutableObject<>(exchangeOperator));
+ exchangeOperator.getInputs().add(new MutableObject<>(replicateOperator));
+ exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ exchangeOperator.setSchema(replicateOperator.getSchema());
+ context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+ return exchangeOperator;
+ }
+
+ private static ExchangeOperator createRandomPartitionExchangeOp(ReplicateOperator replicateOperator,
+ IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+ ExchangeOperator exchangeOperator = new ExchangeOperator();
+ exchangeOperator.setSourceLocation(sourceLocation);
+ exchangeOperator.setPhysicalOperator(new RandomPartitionExchangePOperator(context.getComputationNodeDomain()));
+ replicateOperator.getOutputs().add(new MutableObject<>(exchangeOperator));
+ exchangeOperator.getInputs().add(new MutableObject<>(replicateOperator));
+ exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ exchangeOperator.setSchema(replicateOperator.getSchema());
+ context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+ return exchangeOperator;
+ }
+
+ private static ExchangeOperator createBroadcastExchangeOp(ReplicateOperator replicateOperator,
+ IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+ ExchangeOperator exchangeOperator = new ExchangeOperator();
+ exchangeOperator.setSourceLocation(sourceLocation);
+ exchangeOperator.setPhysicalOperator(new BroadcastExchangePOperator(context.getComputationNodeDomain()));
+ replicateOperator.getOutputs().add(new MutableObject<>(exchangeOperator));
+ exchangeOperator.getInputs().add(new MutableObject<>(replicateOperator));
+ exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+ exchangeOperator.setSchema(replicateOperator.getSchema());
+ context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+ return exchangeOperator;
+ }
+
+ private static Pair<MutableObject<ILogicalOperator>, List<LogicalVariable>> createLocalAndGlobalAggregateOperators(
+ AbstractBinaryJoinOperator op, IOptimizationContext context, LogicalVariable inputVar,
+ MutableObject<ILogicalOperator> exchToLocalAggRef) throws AlgebricksException {
+ AbstractLogicalExpression inputVarRef = new VariableReferenceExpression(inputVar, op.getSourceLocation());
+ List<Mutable<ILogicalExpression>> fields = new ArrayList<>(1);
+ fields.add(new MutableObject<>(inputVarRef));
+
+ // Create local aggregate operator
+ IFunctionInfo localAggFunc = context.getMetadataProvider().lookupFunction(BuiltinFunctions.LOCAL_UNION_MBR);
+ AggregateFunctionCallExpression localAggExpr = new AggregateFunctionCallExpression(localAggFunc, false, fields);
+ localAggExpr.setSourceLocation(op.getSourceLocation());
+ localAggExpr.setOpaqueParameters(new Object[] {});
+ List<LogicalVariable> localAggResultVars = new ArrayList<>(1);
+ List<Mutable<ILogicalExpression>> localAggFuncs = new ArrayList<>(1);
+ LogicalVariable localOutVariable = context.newVar();
+ localAggResultVars.add(localOutVariable);
+ localAggFuncs.add(new MutableObject<>(localAggExpr));
+ AggregateOperator localAggOperator = createAggregate(localAggResultVars, false, localAggFuncs,
+ exchToLocalAggRef, context, op.getSourceLocation());
+ MutableObject<ILogicalOperator> localAgg = new MutableObject<>(localAggOperator);
+
+ // Output of local aggregate operator is the input of global aggregate operator
+ return createGlobalAggregateOperator(op, context, localOutVariable, localAgg);
+ }
+
+ private static Pair<MutableObject<ILogicalOperator>, List<LogicalVariable>> createGlobalAggregateOperator(
+ AbstractBinaryJoinOperator op, IOptimizationContext context, LogicalVariable inputVar,
+ MutableObject<ILogicalOperator> inputOperator) throws AlgebricksException {
+ List<Mutable<ILogicalExpression>> globalAggFuncArgs = new ArrayList<>(1);
+ AbstractLogicalExpression inputVarRef = new VariableReferenceExpression(inputVar, op.getSourceLocation());
+ globalAggFuncArgs.add(new MutableObject<>(inputVarRef));
+ IFunctionInfo globalAggFunc = context.getMetadataProvider().lookupFunction(BuiltinFunctions.GLOBAL_UNION_MBR);
+ AggregateFunctionCallExpression globalAggExpr =
+ new AggregateFunctionCallExpression(globalAggFunc, true, globalAggFuncArgs);
+ globalAggExpr.setStepOneAggregate(globalAggFunc);
+ globalAggExpr.setStepTwoAggregate(globalAggFunc);
+ globalAggExpr.setSourceLocation(op.getSourceLocation());
+ globalAggExpr.setOpaqueParameters(new Object[] {});
+ List<LogicalVariable> globalAggResultVars = new ArrayList<>(1);
+ globalAggResultVars.add(context.newVar());
+ List<Mutable<ILogicalExpression>> globalAggFuncs = new ArrayList<>(1);
+ globalAggFuncs.add(new MutableObject<>(globalAggExpr));
+ AggregateOperator globalAggOperator = createAggregate(globalAggResultVars, true, globalAggFuncs, inputOperator,
+ context, op.getSourceLocation());
+ globalAggOperator.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(globalAggOperator);
+ MutableObject<ILogicalOperator> globalAgg = new MutableObject<>(globalAggOperator);
+ return new Pair<>(globalAgg, globalAggResultVars);
+ }
+
+ private static Triple<MutableObject<ILogicalOperator>, List<LogicalVariable>, MutableObject<ILogicalOperator>> createDynamicMBRCalculator(
+ AbstractBinaryJoinOperator op, IOptimizationContext context, Mutable<ILogicalOperator> inputOp,
+ LogicalVariable inputVar) throws AlgebricksException {
+ // Add ReplicationOperator for the input branch
+ SourceLocation sourceLocation = op.getSourceLocation();
+ ReplicateOperator replicateOperator = createReplicateOperator(inputOp, context, sourceLocation, 2);
+
+ // Create one to one exchange operators for the replicator of the input branch
+ ExchangeOperator exchToForward = createRandomPartitionExchangeOp(replicateOperator, context, sourceLocation);
+ MutableObject<ILogicalOperator> exchToForwardRef = new MutableObject<>(exchToForward);
+
+ ExchangeOperator exchToLocalAgg = createOneToOneExchangeOp(replicateOperator, context, op.getSourceLocation());
+ MutableObject<ILogicalOperator> exchToLocalAggRef = new MutableObject<>(exchToLocalAgg);
+
+ // Materialize the data to be able to re-read the data again
+ replicateOperator.getOutputMaterializationFlags()[0] = true;
+
+ Pair<MutableObject<ILogicalOperator>, List<LogicalVariable>> createLocalAndGlobalAggResult =
+ createLocalAndGlobalAggregateOperators(op, context, inputVar, exchToLocalAggRef);
+ return new Triple<>(createLocalAndGlobalAggResult.first, createLocalAndGlobalAggResult.second,
+ exchToForwardRef);
+ }
+
+ /**
+ * Creates an aggregate operator. $$resultVariables = expressions()
+ * @param resultVariables the variables which stores the result of the aggregation
+ * @param isGlobal whether the aggregate operator is a global or local one
+ * @param expressions the aggregation functions desired
+ * @param inputOperator the input op that is feeding the aggregate operator
+ * @param context optimization context
+ * @param sourceLocation source location
+ * @return an aggregate operator with the specified information
+ * @throws AlgebricksException when there is error setting the type environment of the newly created aggregate op
+ */
+ private static AggregateOperator createAggregate(List<LogicalVariable> resultVariables, boolean isGlobal,
+ List<Mutable<ILogicalExpression>> expressions, MutableObject<ILogicalOperator> inputOperator,
+ IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+ AggregateOperator aggregateOperator = new AggregateOperator(resultVariables, expressions);
+ aggregateOperator.setPhysicalOperator(new AggregatePOperator());
+ aggregateOperator.setSourceLocation(sourceLocation);
+ aggregateOperator.getInputs().add(inputOperator);
+ aggregateOperator.setGlobal(isGlobal);
+ if (!isGlobal) {
+ aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+ } else {
+ aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+ }
+ aggregateOperator.recomputeSchema();
+ context.computeAndSetTypeEnvironmentForOperator(aggregateOperator);
+ return aggregateOperator;
+ }
+
+ private static Mutable<ILogicalExpression> createRectangleExpression(SpatialJoinAnnotation spatialJoinAnn) {
+ return new MutableObject<>(new ConstantExpression(
+ new AsterixConstantValue(new ARectangle(new APoint(spatialJoinAnn.getMinX(), spatialJoinAnn.getMinY()),
+ new APoint(spatialJoinAnn.getMaxX(), spatialJoinAnn.getMaxY())))));
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index 6f07718..ac7fabe 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -46,6 +46,7 @@
import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
+import org.apache.asterix.lang.common.statement.CreateViewStatement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
import org.apache.asterix.lang.common.statement.DeleteStatement;
@@ -341,6 +342,17 @@
}
break;
+ case CREATE_VIEW:
+ CreateViewStatement viewCreateStmt = (CreateViewStatement) stmt;
+ if (viewCreateStmt.getDataverseName() != null) {
+ dataverseName = viewCreateStmt.getDataverseName();
+ }
+ invalidOperation = isMetadataDataverse(dataverseName);
+ if (invalidOperation) {
+ message = String.format(BAD_DATAVERSE_OBJECT_DDL_MESSAGE, "create", "view", dataverseName);
+ }
+ break;
+
case CREATE_FEED:
CreateFeedStatement feedCreateStmt = (CreateFeedStatement) stmt;
if (feedCreateStmt.getDataverseName() != null) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index e85fec8..088676c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -193,6 +193,8 @@
DataverseName getDataverseName();
String getDatasetName();
+
+ byte getCategory();
}
public static class CompiledCreateIndexStatement extends AbstractCompiledStatement
@@ -227,6 +229,11 @@
public Statement.Kind getKind() {
return Statement.Kind.CREATE_INDEX;
}
+
+ @Override
+ public byte getCategory() {
+ return Statement.Category.DDL;
+ }
}
public static class CompiledLoadFromFileStatement extends AbstractCompiledStatement
@@ -272,6 +279,11 @@
public Statement.Kind getKind() {
return Statement.Kind.LOAD;
}
+
+ @Override
+ public byte getCategory() {
+ return Statement.Category.UPDATE;
+ }
}
public static class CompiledInsertStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
@@ -322,6 +334,11 @@
public Statement.Kind getKind() {
return Statement.Kind.INSERT;
}
+
+ @Override
+ public byte getCategory() {
+ return Statement.Category.UPDATE;
+ }
}
public static class CompiledUpsertStatement extends CompiledInsertStatement {
@@ -370,6 +387,11 @@
public Statement.Kind getKind() {
return Statement.Kind.SUBSCRIBE_FEED;
}
+
+ @Override
+ public byte getCategory() {
+ return Statement.Category.UPDATE;
+ }
}
public static class CompiledDeleteStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
@@ -415,6 +437,10 @@
return Statement.Kind.DELETE;
}
+ @Override
+ public byte getCategory() {
+ return Statement.Category.UPDATE;
+ }
}
public static class CompiledCompactStatement extends AbstractCompiledStatement {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
index 511b74c..f33bb01 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
@@ -21,13 +21,17 @@
import java.io.Serializable;
public class ExecutionPlans implements Serializable {
- private static final long serialVersionUID = 6853904213354224457L;
+ private static final long serialVersionUID = 6853904213354224458L;
private String expressionTree;
private String rewrittenExpressionTree;
private String logicalPlan;
private String optimizedLogicalPlan;
private String job;
+ private String signature;
+ private String statementCategory;
+ private String statementParameters;
+ private boolean explainOnly;
public String getExpressionTree() {
return expressionTree;
@@ -68,4 +72,36 @@
public void setJob(String job) {
this.job = job;
}
+
+ public String getSignature() {
+ return signature;
+ }
+
+ public void setSignature(String signature) {
+ this.signature = signature;
+ }
+
+ public String getStatementCategory() {
+ return statementCategory;
+ }
+
+ public void setStatementCategory(String statementCategory) {
+ this.statementCategory = statementCategory;
+ }
+
+ public String getStatementParameters() {
+ return statementParameters;
+ }
+
+ public void setStatementParameters(String statementParameters) {
+ this.statementParameters = statementParameters;
+ }
+
+ public boolean isExplainOnly() {
+ return explainOnly;
+ }
+
+ public void setExplainOnly(boolean explainOnly) {
+ this.explainOnly = explainOnly;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java
index c83e865..ad89ad6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.translator;
+import static org.apache.asterix.translator.SessionConfig.PlanFormat.JSON;
import static org.apache.asterix.translator.SessionConfig.PlanFormat.STRING;
import org.apache.hyracks.util.JSONUtil;
@@ -29,6 +30,9 @@
private static final String REWRITTEN_EXPRESSION_TREE_LBL = "rewrittenExpressionTree";
public static final String OPTIMIZED_LOGICAL_PLAN_LBL = "optimizedLogicalPlan";
private static final String JOB_LBL = "job";
+ private static final String STATEMENT_CATEGORY_LBL = "statementCategory";
+ private static final String STATEMENT_PARAMETERS_LBL = "statementParameters";
+ private static final String EXPLAIN_ONLY_LBL = "explainOnly";
private ExecutionPlansJsonPrintUtil() {
}
@@ -42,6 +46,11 @@
appendNonNull(output, LOGICAL_PLAN_LBL, plans.getLogicalPlan(), format);
appendNonNull(output, OPTIMIZED_LOGICAL_PLAN_LBL, plans.getOptimizedLogicalPlan(), format);
appendNonNull(output, JOB_LBL, plans.getJob(), format);
+ appendNonNull(output, STATEMENT_CATEGORY_LBL, plans.getStatementCategory(), STRING);
+ appendNonNull(output, STATEMENT_PARAMETERS_LBL, plans.getStatementParameters(), JSON);
+ if (plans.isExplainOnly()) {
+ appendNonNull(output, EXPLAIN_ONLY_LBL, Boolean.toString(plans.isExplainOnly()), JSON);
+ }
appendOutputPostfix(output);
return output.toString();
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
index 5c743ee..c6022eb 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
@@ -47,7 +47,7 @@
/**
* @return a reference on which to write properties of executed queries (e.g. what kind of statement was parsed
- * by the parser)
+ * by the parser)
*/
StatementProperties getStatementProperties();
@@ -63,8 +63,8 @@
/**
* @return a bitmask that restricts which statement
- * {@link org.apache.asterix.lang.common.base.Statement.Category categories} are permitted for this request,
- * {@code 0} if all categories are allowed
+ * {@link org.apache.asterix.lang.common.base.Statement.Category categories} are permitted for this request,
+ * {@code 0} if all categories are allowed
*/
int getStatementCategoryRestrictionMask();
@@ -75,4 +75,13 @@
boolean isForceDropDataset();
boolean isSkipAdmissionPolicy();
-}
+
+ boolean isPrintSignature();
+
+ boolean isSQLCompatMode();
+
+ /**
+ * @return canonical name of the default dataverse for this statement
+ */
+ String getDefaultDataverseName();
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index d60b791..8a91059 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -36,7 +36,6 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.metadata.DataverseName;
-import org.apache.asterix.lang.common.base.IStatementRewriter;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.statement.Query;
import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -272,8 +271,6 @@
* The data modification statement when the query results in a modification to a dataset
* @param statementParameters
* Statement parameters
- * @param statementRewriter
- * The statement rewriter
* @param requestParameters
* The request parameters
* @return the compiled {@code JobSpecification}
@@ -284,8 +281,7 @@
*/
JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector, MetadataProvider metadataProvider,
Query query, ICompiledDmlStatement dmlStatement, Map<String, IAObject> statementParameters,
- IStatementRewriter statementRewriter, IRequestParameters requestParameters)
- throws RemoteException, AlgebricksException, ACIDException;
+ IRequestParameters requestParameters) throws RemoteException, AlgebricksException, ACIDException;
/**
* returns the active dataverse for an entity or a statement
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 16d5878..794a0b7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -48,6 +48,7 @@
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
import org.apache.asterix.lang.common.clause.OrderbyClause;
+import org.apache.asterix.lang.common.clause.OrderbyClause.NullOrderModifier;
import org.apache.asterix.lang.common.clause.OrderbyClause.OrderModifier;
import org.apache.asterix.lang.common.clause.WhereClause;
import org.apache.asterix.lang.common.expression.CallExpr;
@@ -68,6 +69,7 @@
import org.apache.asterix.lang.common.literal.StringLiteral;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.OperatorType;
import org.apache.asterix.lang.common.struct.QuantifiedPair;
@@ -513,8 +515,8 @@
upsertOp = new InsertDeleteUpsertOperator(targetDatasource, payloadVarRef, varRefsForLoading,
Collections.singletonList(new MutableObject<>(metaVarRef)), InsertDeleteUpsertOperator.Kind.UPSERT,
false);
- upsertOp.setUpsertIndicatorVar(context.newVar());
- upsertOp.setUpsertIndicatorVarType(BuiltinType.ABOOLEAN);
+ upsertOp.setOperationVar(context.newVar());
+ upsertOp.setOperationVarType(BuiltinType.AINT8);
// Create and add a new variable used for representing the original record
upsertOp.setPrevRecordVar(context.newVar());
upsertOp.setPrevRecordType(targetDatasource.getItemType());
@@ -567,8 +569,8 @@
upsertOp.getInputs().add(new MutableObject<>(pkeyAssignOp));
upsertOp.setAdditionalFilteringExpressions(filterExprs);
upsertOp.setSourceLocation(sourceLoc);
- upsertOp.setUpsertIndicatorVar(context.newVar());
- upsertOp.setUpsertIndicatorVarType(BuiltinType.ABOOLEAN);
+ upsertOp.setOperationVar(context.newVar());
+ upsertOp.setOperationVarType(BuiltinType.AINT8);
// Create and add a new variable used for representing the original record
upsertOp.setPrevRecordVar(context.newVar());
upsertOp.setPrevRecordType(recordType);
@@ -977,8 +979,17 @@
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visit(FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
- throw new IllegalStateException("Function declarations should be inlined at AST rewriting phase.");
+ public Pair<ILogicalOperator, LogicalVariable> visit(FunctionDecl fd, Mutable<ILogicalOperator> tupSource)
+ throws CompilationException {
+ // Function declarations should be inlined at AST rewriting phase
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fd.getSourceLocation(), fd.getSignature());
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visit(ViewDecl vd, Mutable<ILogicalOperator> tupSource)
+ throws CompilationException {
+ // View declarations should be inlined at AST rewriting phase
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, vd.getSourceLocation(), vd.getViewName());
}
@Override
@@ -1256,13 +1267,17 @@
SourceLocation sourceLoc = oc.getSourceLocation();
OrderOperator ord = new OrderOperator();
ord.setSourceLocation(sourceLoc);
- Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+ List<Expression> orderbyList = oc.getOrderbyList();
+ List<OrderModifier> modifierList = oc.getModifierList();
+ List<NullOrderModifier> nullModifierList = oc.getNullModifierList();
Mutable<ILogicalOperator> topOp = tupSource;
- for (Expression e : oc.getOrderbyList()) {
+ for (int i = 0, n = orderbyList.size(); i < n; i++) {
+ Expression e = orderbyList.get(i);
Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(e, topOp);
- OrderModifier m = modifIter.next();
- OrderOperator.IOrder comp = translateOrderModifier(m);
- ord.getOrderExpressions().add(new Pair<>(comp, new MutableObject<>(p.first)));
+ ILogicalExpression obyExpr = p.first;
+ OrderModifier modifier = modifierList.get(i);
+ NullOrderModifier nullModifier = nullModifierList.get(i);
+ addOrderByExpression(ord.getOrderExpressions(), obyExpr, modifier, nullModifier);
topOp = p.second;
}
ord.getInputs().add(topOp);
@@ -1273,23 +1288,58 @@
ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
}
if (oc.getRangeMap() != null) {
- Iterator<OrderModifier> orderModifIter = oc.getModifierList().iterator();
- boolean ascending = orderModifIter.next() == OrderModifier.ASC;
+ boolean ascending = modifierList.get(0) == OrderModifier.ASC;
RangeMapBuilder.verifyRangeOrder(oc.getRangeMap(), ascending, sourceLoc);
ord.getAnnotations().put(OperatorAnnotations.USE_STATIC_RANGE, oc.getRangeMap());
}
return new Pair<>(ord, null);
}
+ protected void addOrderByExpression(List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> outOrderList,
+ ILogicalExpression obyExpr, OrderModifier modifier, NullOrderModifier nullModifier) {
+ OrderOperator.IOrder comp = translateOrderModifier(modifier);
+ ILogicalExpression nullModifierExpr = translateNullOrderModifier(obyExpr, modifier, nullModifier);
+ if (nullModifierExpr != null) {
+ outOrderList.add(new Pair<>(comp, new MutableObject<>(nullModifierExpr)));
+ }
+ outOrderList.add(new Pair<>(comp, new MutableObject<>(obyExpr)));
+ }
+
protected OrderOperator.IOrder translateOrderModifier(OrderModifier m) {
return m == OrderModifier.ASC ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
}
+ protected ILogicalExpression translateNullOrderModifier(ILogicalExpression obyExpr, OrderModifier m,
+ NullOrderModifier nm) {
+ if ((m == OrderModifier.ASC && nm == NullOrderModifier.LAST)
+ || (m == OrderModifier.DESC && nm == NullOrderModifier.FIRST)) {
+ AbstractFunctionCallExpression isUnknownExpr =
+ createFunctionCallExpression(BuiltinFunctions.IS_UNKNOWN, obyExpr.getSourceLocation());
+ isUnknownExpr.getArguments().add(new MutableObject<>(obyExpr.cloneExpression()));
+ return isUnknownExpr;
+ } else {
+ return null;
+ }
+ }
+
@Override
public Pair<ILogicalOperator, LogicalVariable> visit(QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
throws CompilationException {
SourceLocation sourceLoc = qe.getSourceLocation();
- Mutable<ILogicalOperator> topOp = tupSource;
+
+ Mutable<ILogicalOperator> topOp;
+ SubplanOperator subplanOp;
+ if (qe.getQuantifier() == Quantifier.SOME_AND_EVERY) {
+ subplanOp = new SubplanOperator();
+ subplanOp.getInputs().add(tupSource);
+ subplanOp.setSourceLocation(sourceLoc);
+ NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
+ ntsOp.setSourceLocation(sourceLoc);
+ topOp = new MutableObject<>(ntsOp);
+ } else {
+ subplanOp = null; // not used
+ topOp = tupSource;
+ }
ILogicalOperator firstOp = null;
Mutable<ILogicalOperator> lastOp = null;
@@ -1320,47 +1370,104 @@
Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = langExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
- AggregateFunctionCallExpression fAgg;
- SelectOperator s;
- if (qe.getQuantifier() == Quantifier.SOME) {
- s = new SelectOperator(new MutableObject<>(eo2.first), false, null);
- s.getInputs().add(eo2.second);
- s.setSourceLocation(sourceLoc);
- fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM,
- new ArrayList<>());
- fAgg.setSourceLocation(sourceLoc);
- } else { // EVERY
- // look for input items that do not satisfy the condition, if none found then return true
- // when inverting the condition account for NULL/MISSING by replacing them with FALSE
- // condition() -> not(if-missing-or-null(condition(), false))
+ switch (qe.getQuantifier()) {
+ case SOME:
+ SelectOperator s = new SelectOperator(new MutableObject<>(eo2.first));
+ s.getInputs().add(eo2.second);
+ s.setSourceLocation(sourceLoc);
+ AggregateFunctionCallExpression fAgg = BuiltinFunctions
+ .makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM, new ArrayList<>(0));
+ fAgg.setSourceLocation(sourceLoc);
+ LogicalVariable qeVar = context.newVar();
+ AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+ mkSingletonArrayList(new MutableObject<>(fAgg)));
+ a.getInputs().add(new MutableObject<>(s));
+ a.setSourceLocation(sourceLoc);
+ return new Pair<>(a, qeVar);
+ case EVERY:
+ // look for input items that do not satisfy the condition, if none found then return true
+ // when inverting the condition account for NULL/MISSING by replacing them with FALSE:
+ // condition() -> not(if-missing-or-null(condition(), false))
+ List<Mutable<ILogicalExpression>> ifMissingOrNullArgs = new ArrayList<>(2);
+ ifMissingOrNullArgs.add(new MutableObject<>(eo2.first));
+ ifMissingOrNullArgs.add(new MutableObject<>(ConstantExpression.FALSE));
+ List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
+ ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL),
+ ifMissingOrNullArgs);
+ ifMissinOrNullExpr.setSourceLocation(sourceLoc);
+ notArgs.add(new MutableObject<>(ifMissinOrNullExpr));
+ ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
+ notExpr.setSourceLocation(sourceLoc);
+ s = new SelectOperator(new MutableObject<>(notExpr));
+ s.getInputs().add(eo2.second);
+ s.setSourceLocation(sourceLoc);
+ fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM,
+ new ArrayList<>());
+ fAgg.setSourceLocation(sourceLoc);
+ qeVar = context.newVar();
+ a = new AggregateOperator(mkSingletonArrayList(qeVar), mkSingletonArrayList(new MutableObject<>(fAgg)));
+ a.getInputs().add(new MutableObject<>(s));
+ a.setSourceLocation(sourceLoc);
+ return new Pair<>(a, qeVar);
+ case SOME_AND_EVERY:
+ // return true if the stream was non-empty but there were no items that satisfied the condition
+ AbstractFunctionCallExpression fAgg1 = BuiltinFunctions
+ .makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM, new ArrayList<>(0));
+ fAgg1.setSourceLocation(sourceLoc);
- List<Mutable<ILogicalExpression>> ifMissingOrNullArgs = new ArrayList<>(2);
- ConstantExpression eFalse = new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
- eFalse.setSourceLocation(sourceLoc);
- ifMissingOrNullArgs.add(new MutableObject<>(eo2.first));
- ifMissingOrNullArgs.add(new MutableObject<>(eFalse));
+ List<Mutable<ILogicalExpression>> switchCaseArgs = new ArrayList<>(4);
+ switchCaseArgs.add(new MutableObject<>(eo2.first));
+ switchCaseArgs.add(new MutableObject<>(ConstantExpression.TRUE));
+ switchCaseArgs.add(new MutableObject<>(ConstantExpression.NULL));
+ switchCaseArgs.add(new MutableObject<>(ConstantExpression.TRUE));
+ ScalarFunctionCallExpression switchCaseExpr = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SWITCH_CASE), switchCaseArgs);
+ switchCaseExpr.setSourceLocation(sourceLoc);
- List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
- ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
- FunctionUtil.getFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL), ifMissingOrNullArgs);
- ifMissinOrNullExpr.setSourceLocation(sourceLoc);
- notArgs.add(new MutableObject<>(ifMissinOrNullExpr));
+ AbstractFunctionCallExpression fAgg2 = BuiltinFunctions.makeAggregateFunctionExpression(
+ BuiltinFunctions.SQL_COUNT, mkSingletonArrayList(new MutableObject<>(switchCaseExpr)));
+ fAgg2.setSourceLocation(sourceLoc);
- ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
- FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
- notExpr.setSourceLocation(sourceLoc);
- s = new SelectOperator(new MutableObject<>(notExpr), false, null);
- s.getInputs().add(eo2.second);
- s.setSourceLocation(sourceLoc);
- fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM, new ArrayList<>());
- fAgg.setSourceLocation(sourceLoc);
+ LogicalVariable qeVar1 = context.newVar();
+ LogicalVariable qeVar2 = context.newVar();
+ List<LogicalVariable> qeVarList = new ArrayList<>(2);
+ List<Mutable<ILogicalExpression>> fAggList = new ArrayList<>(2);
+ qeVarList.add(qeVar1);
+ qeVarList.add(qeVar2);
+ fAggList.add(new MutableObject<>(fAgg1));
+ fAggList.add(new MutableObject<>(fAgg2));
+
+ a = new AggregateOperator(qeVarList, fAggList);
+ a.getInputs().add(eo2.second);
+ a.setSourceLocation(sourceLoc);
+
+ subplanOp.setRootOp(new MutableObject<>(a));
+
+ VariableReferenceExpression qeVar1Ref = new VariableReferenceExpression(qeVar1);
+ qeVar1Ref.setSourceLocation(sourceLoc);
+
+ VariableReferenceExpression qeVar2Ref = new VariableReferenceExpression(qeVar2);
+ qeVar2Ref.setSourceLocation(sourceLoc);
+ ScalarFunctionCallExpression qeVar2EqZero = new ScalarFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.EQ), new MutableObject<>(qeVar2Ref),
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(0)))));
+
+ ScalarFunctionCallExpression andExpr =
+ new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND),
+ new MutableObject<>(qeVar1Ref), new MutableObject<>(qeVar2EqZero));
+
+ qeVar = context.newVar();
+ AssignOperator assignOp2 = new AssignOperator(qeVar, new MutableObject<>(andExpr));
+ assignOp2.setSourceLocation(sourceLoc);
+ assignOp2.getInputs().add(new MutableObject<>(subplanOp));
+
+ return new Pair<>(assignOp2, qeVar);
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ qe.getQuantifier().toString());
}
- LogicalVariable qeVar = context.newVar();
- AggregateOperator a =
- new AggregateOperator(mkSingletonArrayList(qeVar), mkSingletonArrayList(new MutableObject<>(fAgg)));
- a.getInputs().add(new MutableObject<>(s));
- a.setSourceLocation(sourceLoc);
- return new Pair<>(a, qeVar);
}
@Override
@@ -1463,7 +1570,7 @@
public Pair<ILogicalOperator, LogicalVariable> visit(WhereClause w, Mutable<ILogicalOperator> tupSource)
throws CompilationException {
Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(w.getWhereExpr(), tupSource);
- SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
+ SelectOperator s = new SelectOperator(new MutableObject<>(p.first));
s.getInputs().add(p.second);
s.setSourceLocation(w.getSourceLocation());
return new Pair<>(s, null);
@@ -1701,14 +1808,25 @@
}
protected boolean expressionNeedsNoNesting(Expression expr) throws CompilationException {
- Kind k = expr.getKind();
- boolean noNesting = k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
- || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION;
- noNesting = noNesting || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION
- || k == Kind.FIELD_ACCESSOR_EXPRESSION;
- noNesting = noNesting || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION
- || k == Kind.IF_EXPRESSION;
- return noNesting || k == Kind.CASE_EXPRESSION || k == Kind.WINDOW_EXPRESSION;
+ switch (expr.getKind()) {
+ case LITERAL_EXPRESSION:
+ case LIST_CONSTRUCTOR_EXPRESSION:
+ case RECORD_CONSTRUCTOR_EXPRESSION:
+ case VARIABLE_EXPRESSION:
+ case CALL_EXPRESSION:
+ case OP_EXPRESSION:
+ case FIELD_ACCESSOR_EXPRESSION:
+ case INDEX_ACCESSOR_EXPRESSION:
+ case UNARY_EXPRESSION:
+ case IF_EXPRESSION:
+ case CASE_EXPRESSION:
+ case WINDOW_EXPRESSION:
+ return true;
+ case QUANTIFIED_EXPRESSION:
+ return ((QuantifiedExpression) expr).getQuantifier() == Quantifier.SOME_AND_EVERY;
+ default:
+ return false;
+ }
}
protected <T> List<T> mkSingletonArrayList(T item) {
@@ -1906,7 +2024,7 @@
subplanOp.setSourceLocation(sourceLoc);
NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
ntsOp.setSourceLocation(sourceLoc);
- SelectOperator select = new SelectOperator(selectExpr, false, null);
+ SelectOperator select = new SelectOperator(selectExpr);
// The select operator cannot be moved up and down, otherwise it will cause
// typing issues (ASTERIXDB-1203).
OperatorPropertiesUtil.markMovable(select, false);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
index 8077172..4e294ec 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
@@ -21,6 +21,7 @@
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
+import java.util.Objects;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.Logger;
@@ -39,7 +40,7 @@
* <li>It allows you to specify output format-specific parameters.
*/
public class SessionConfig implements Serializable {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
/**
* Used to specify the output format for the primary execution.
@@ -48,7 +49,8 @@
ADM,
CSV,
CLEAN_JSON,
- LOSSLESS_JSON
+ LOSSLESS_JSON,
+ LOSSLESS_ADM_JSON
}
/**
@@ -72,6 +74,11 @@
}
}
+ public enum ClientType {
+ ASTERIX,
+ JDBC
+ }
+
/**
* Produce out-of-band output for Hyracks Job.
*/
@@ -127,12 +134,15 @@
*/
public static final String FORMAT_QUOTE_RECORD = "quote-record";
+ // Client type
+ private ClientType clientType;
+
// Output format.
private OutputFormat fmt;
private PlanFormat planFormat;
// Standard execution flags.
- private final boolean executeQuery;
+ private boolean executeQuery;
private final boolean generateJobSpec;
private final boolean optimize;
private long maxWarnings;
@@ -174,6 +184,18 @@
this.generateJobSpec = generateJobSpec;
this.flags = new HashMap<>();
this.planFormat = planFormat;
+ this.clientType = ClientType.ASTERIX;
+ }
+
+ /**
+ * Retrieve the client type for this execution.
+ */
+ public ClientType getClientType() {
+ return this.clientType;
+ }
+
+ public void setClientType(ClientType clientType) {
+ this.clientType = Objects.requireNonNull(clientType);
}
/**
@@ -216,6 +238,10 @@
return executeQuery;
}
+ public void setExecuteQuery(boolean executeQuery) {
+ this.executeQuery = executeQuery;
+ }
+
/**
* Retrieve the value of the "optimize" flag.
*/
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 95026a5..1d226dd 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -112,6 +112,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -344,10 +345,10 @@
public Pair<ILogicalOperator, LogicalVariable> visit(JoinClause joinClause, Mutable<ILogicalOperator> leftInputRef)
throws CompilationException {
SourceLocation sourceLoc = joinClause.getSourceLocation();
- if (joinClause.getJoinType() == JoinType.INNER) {
+ if (joinClause.getJoinType() == JoinType.INNER && !context.inSubplan()) {
Mutable<ILogicalOperator> rightInputRef = uncorrelatedRightBranchStack.peek();
Pair<ILogicalOperator, LogicalVariable> rightBranch =
- generateUnnestForBinaryCorrelateRightBranch(joinClause, rightInputRef, true);
+ generateUnnestForBinaryCorrelateRightBranch(joinClause, rightInputRef, false, null);
// A join operator with condition TRUE.
AbstractBinaryJoinOperator joinOperator = new InnerJoinOperator(
new MutableObject<>(ConstantExpression.TRUE), leftInputRef, new MutableObject<>(rightBranch.first));
@@ -357,11 +358,11 @@
// Add an additional filter operator.
Pair<ILogicalExpression, Mutable<ILogicalOperator>> conditionExprOpPair =
langExprToAlgExpression(joinClause.getConditionExpression(), joinOpRef);
- SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
+ SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first));
filter.getInputs().add(conditionExprOpPair.second);
filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
return new Pair<>(filter, rightBranch.second);
- } else if (joinClause.getJoinType() == JoinType.LEFTOUTER) {
+ } else if (joinClause.getJoinType() == JoinType.INNER || joinClause.getJoinType() == JoinType.LEFTOUTER) {
// Creates a subplan operator.
SubplanOperator subplanOp = new SubplanOperator();
subplanOp.getInputs().add(leftInputRef);
@@ -375,13 +376,13 @@
// Adds an unnest operator to unnest to right expression.
Pair<ILogicalOperator, LogicalVariable> rightBranch =
- generateUnnestForBinaryCorrelateRightBranch(joinClause, ntsRef, true);
+ generateUnnestForBinaryCorrelateRightBranch(joinClause, ntsRef, false, null);
AbstractUnnestNonMapOperator rightUnnestOp = (AbstractUnnestNonMapOperator) rightBranch.first;
// Adds an additional filter operator for the join condition.
Pair<ILogicalExpression, Mutable<ILogicalOperator>> conditionExprOpPair =
langExprToAlgExpression(joinClause.getConditionExpression(), new MutableObject<>(rightUnnestOp));
- SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
+ SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first));
filter.getInputs().add(conditionExprOpPair.second);
filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
@@ -441,20 +442,22 @@
ILogicalPlan subplan = new ALogicalPlanImpl(new MutableObject<>(aggOp));
subplanOp.getNestedPlans().add(subplan);
- // Outer unnest the aggregated var from the subplan.
- LogicalVariable outerUnnestVar = context.newVar();
+ // Unnest the aggregated var from the subplan.
+ LogicalVariable unnestVar = context.newVar();
VariableReferenceExpression aggVarRefExpr = new VariableReferenceExpression(aggVar);
aggVarRefExpr.setSourceLocation(aggOp.getSourceLocation());
Pair<ILogicalExpression, Mutable<ILogicalOperator>> pUnnestExpr =
makeUnnestExpression(aggVarRefExpr, new MutableObject<>(subplanOp));
- LeftOuterUnnestOperator outerUnnestOp =
- new LeftOuterUnnestOperator(outerUnnestVar, new MutableObject<>(pUnnestExpr.first));
- outerUnnestOp.getInputs().add(pUnnestExpr.second);
- outerUnnestOp.setSourceLocation(aggOp.getSourceLocation());
- currentTopOp = outerUnnestOp;
+ AbstractUnnestOperator unnestOp = joinClause.getJoinType() == JoinType.INNER
+ ? new UnnestOperator(unnestVar, new MutableObject<>(pUnnestExpr.first))
+ : new LeftOuterUnnestOperator(unnestVar, new MutableObject<>(pUnnestExpr.first),
+ translateLeftOuterMissingValue(joinClause.getOuterJoinMissingValueType()));
+ unnestOp.getInputs().add(pUnnestExpr.second);
+ unnestOp.setSourceLocation(aggOp.getSourceLocation());
+ currentTopOp = unnestOp;
if (hasRightPosVar) {
- VariableReferenceExpression outerUnnestVarRef1 = new VariableReferenceExpression(outerUnnestVar);
+ VariableReferenceExpression outerUnnestVarRef1 = new VariableReferenceExpression(unnestVar);
outerUnnestVarRef1.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
ScalarFunctionCallExpression fieldAccessForRightUnnestVar = new ScalarFunctionCallExpression(
FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
@@ -462,7 +465,7 @@
new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
fieldAccessForRightUnnestVar.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
- VariableReferenceExpression outerUnnestVarRef2 = new VariableReferenceExpression(outerUnnestVar);
+ VariableReferenceExpression outerUnnestVarRef2 = new VariableReferenceExpression(unnestVar);
outerUnnestVarRef2.setSourceLocation(joinClause.getPositionalVariable().getSourceLocation());
ScalarFunctionCallExpression fieldAccessForRightPosVar = new ScalarFunctionCallExpression(
FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX),
@@ -494,7 +497,7 @@
assignOp.setSourceLocation(joinClause.getRightVariable().getSourceLocation());
currentTopOp = assignOp;
} else {
- context.setVar(joinClause.getRightVariable(), outerUnnestVar);
+ context.setVar(joinClause.getRightVariable(), unnestVar);
}
return new Pair<>(currentTopOp, null);
} else if (joinClause.getJoinType() == JoinType.RIGHTOUTER) {
@@ -506,6 +509,18 @@
}
}
+ private static IAlgebricksConstantValue translateLeftOuterMissingValue(Literal.Type type)
+ throws CompilationException {
+ switch (type) {
+ case MISSING:
+ return ConstantExpression.MISSING.getValue();
+ case NULL:
+ return ConstantExpression.NULL.getValue();
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, String.valueOf(type));
+ }
+ }
+
@Override
public Pair<ILogicalOperator, LogicalVariable> visit(NestClause nestClause, Mutable<ILogicalOperator> arg)
throws CompilationException {
@@ -516,8 +531,9 @@
@Override
public Pair<ILogicalOperator, LogicalVariable> visit(UnnestClause unnestClause,
Mutable<ILogicalOperator> inputOpRef) throws CompilationException {
- return generateUnnestForBinaryCorrelateRightBranch(unnestClause, inputOpRef,
- unnestClause.getUnnestType() == UnnestType.INNER);
+ boolean outerUnnest = unnestClause.getUnnestType() == UnnestType.LEFTOUTER;
+ return generateUnnestForBinaryCorrelateRightBranch(unnestClause, inputOpRef, outerUnnest,
+ outerUnnest ? translateLeftOuterMissingValue(unnestClause.getOuterUnnestMissingValueType()) : null);
}
@Override
@@ -525,14 +541,14 @@
throws CompilationException {
Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
langExprToAlgExpression(havingClause.getFilterExpression(), tupSource);
- SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
+ SelectOperator s = new SelectOperator(new MutableObject<>(p.first));
s.getInputs().add(p.second);
return new Pair<>(s, null);
}
private Pair<ILogicalOperator, LogicalVariable> generateUnnestForBinaryCorrelateRightBranch(
- AbstractBinaryCorrelateClause binaryCorrelate, Mutable<ILogicalOperator> inputOpRef, boolean innerUnnest)
- throws CompilationException {
+ AbstractBinaryCorrelateClause binaryCorrelate, Mutable<ILogicalOperator> inputOpRef, boolean outerUnnest,
+ IAlgebricksConstantValue outerUnnestMissingValue) throws CompilationException {
LogicalVariable rightVar = context.newVarFromExpression(binaryCorrelate.getRightVariable());
Expression rightExpr = binaryCorrelate.getRightExpression();
Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(rightExpr, inputOpRef);
@@ -541,13 +557,15 @@
if (binaryCorrelate.hasPositionalVariable()) {
LogicalVariable pVar = context.newVarFromExpression(binaryCorrelate.getPositionalVariable());
// We set the positional variable type as BIGINT type.
- unnestOp = innerUnnest
- ? new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar, BuiltinType.AINT64)
- : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar,
- BuiltinType.AINT64);
+ unnestOp = outerUnnest
+ ? new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar,
+ BuiltinType.AINT64, outerUnnestMissingValue)
+ : new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar, BuiltinType.AINT64);
} else {
- unnestOp = innerUnnest ? new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first))
- : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first));
+ unnestOp = outerUnnest
+ ? new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first),
+ outerUnnestMissingValue)
+ : new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first));
}
unnestOp.getInputs().add(pUnnestExpr.second);
unnestOp.setSourceLocation(binaryCorrelate.getRightVariable().getSourceLocation());
@@ -778,45 +796,78 @@
Set<String> fieldNames = new HashSet<>();
for (Projection projection : selectRegular.getProjections()) {
- if (projection.varStar()) {
- if (!fieldBindings.isEmpty()) {
- RecordConstructor recordConstr = new RecordConstructor(new ArrayList<>(fieldBindings));
- recordConstr.setSourceLocation(selectRegular.getSourceLocation());
- recordExprs.add(recordConstr);
- fieldBindings.clear();
- }
- Expression projectionExpr = projection.getExpression();
- SourceLocation sourceLoc = projection.getSourceLocation();
- CallExpr toObjectExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.TO_OBJECT),
- Collections.singletonList(projectionExpr));
- toObjectExpr.setSourceLocation(sourceLoc);
- CallExpr ifMissingOrNullExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.IF_MISSING_OR_NULL),
- Arrays.asList(toObjectExpr, new RecordConstructor(Collections.emptyList())));
- ifMissingOrNullExpr.setSourceLocation(sourceLoc);
- recordExprs.add(ifMissingOrNullExpr);
- } else if (projection.star()) {
- if (selectBlock.hasGroupbyClause()) {
- getGroupBindings(selectBlock.getGroupbyClause(), fieldBindings, fieldNames,
- SqlppExpressionToPlanTranslator::includeInSelectStar);
- if (selectBlock.hasLetHavingClausesAfterGroupby()) {
- getLetBindings(selectBlock.getLetHavingListAfterGroupby(), fieldBindings, fieldNames,
+ boolean everyVarStar = false;
+ switch (projection.getKind()) {
+ case VAR_STAR:
+ if (!fieldBindings.isEmpty()) {
+ RecordConstructor recordConstr = new RecordConstructor(new ArrayList<>(fieldBindings));
+ recordConstr.setSourceLocation(selectRegular.getSourceLocation());
+ recordExprs.add(recordConstr);
+ fieldBindings.clear();
+ }
+ Expression expr =
+ translateProjectVarStar(projection.getExpression(), projection.getSourceLocation());
+ recordExprs.add(expr);
+ break;
+ case EVERY_VAR_STAR:
+ everyVarStar = true;
+ // fall thru to STAR
+ case STAR:
+ List<FieldBinding> fieldBindingsForStar;
+ Set<String> fieldNamesForStar;
+ if (everyVarStar) {
+ fieldBindingsForStar = new ArrayList<>();
+ fieldNamesForStar = new HashSet<>();
+ } else {
+ fieldBindingsForStar = fieldBindings;
+ fieldNamesForStar = fieldNames;
+ }
+ if (selectBlock.hasGroupbyClause()) {
+ getGroupBindings(selectBlock.getGroupbyClause(), fieldBindingsForStar, fieldNamesForStar,
+ SqlppExpressionToPlanTranslator::includeInSelectStar);
+ if (selectBlock.hasLetHavingClausesAfterGroupby()) {
+ getLetBindings(selectBlock.getLetHavingListAfterGroupby(), fieldBindingsForStar,
+ fieldNamesForStar, SqlppExpressionToPlanTranslator::includeInSelectStar);
+ }
+ } else if (selectBlock.hasFromClause()) {
+ getFromBindings(selectBlock.getFromClause(), fieldBindingsForStar, fieldNamesForStar,
+ SqlppExpressionToPlanTranslator::includeInSelectStar);
+ if (selectBlock.hasLetWhereClauses()) {
+ getLetBindings(selectBlock.getLetWhereList(), fieldBindingsForStar, fieldNamesForStar,
+ SqlppExpressionToPlanTranslator::includeInSelectStar);
+ }
+ } else if (selectBlock.hasLetWhereClauses()) {
+ getLetBindings(selectBlock.getLetWhereList(), fieldBindingsForStar, fieldNamesForStar,
SqlppExpressionToPlanTranslator::includeInSelectStar);
}
- } else if (selectBlock.hasFromClause()) {
- getFromBindings(selectBlock.getFromClause(), fieldBindings, fieldNames,
- SqlppExpressionToPlanTranslator::includeInSelectStar);
- if (selectBlock.hasLetWhereClauses()) {
- getLetBindings(selectBlock.getLetWhereList(), fieldBindings, fieldNames,
- SqlppExpressionToPlanTranslator::includeInSelectStar);
+ if (everyVarStar) {
+ if (!fieldBindings.isEmpty()) {
+ RecordConstructor recordConstr = new RecordConstructor(new ArrayList<>(fieldBindings));
+ recordConstr.setSourceLocation(selectRegular.getSourceLocation());
+ recordExprs.add(recordConstr);
+ fieldBindings.clear();
+ }
+ // We currently only support EVERY_VAR_STAR over a single variable
+ if (fieldBindingsForStar.size() > 1) {
+ throw new CompilationException(ErrorCode.AMBIGUOUS_PROJECTION,
+ projection.getSourceLocation());
+ }
+ for (FieldBinding fieldBinding : fieldBindingsForStar) {
+ expr = translateProjectVarStar(fieldBinding.getRightExpr(), projection.getSourceLocation());
+ recordExprs.add(expr);
+ }
}
- } else if (selectBlock.hasLetWhereClauses()) {
- getLetBindings(selectBlock.getLetWhereList(), fieldBindings, fieldNames,
- SqlppExpressionToPlanTranslator::includeInSelectStar);
- }
- } else if (projection.hasName()) {
- fieldBindings.add(getFieldBinding(projection, fieldNames));
- } else {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, projection.getSourceLocation(), "");
+ break;
+ case NAMED_EXPR:
+ if (!projection.hasName()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ projection.getSourceLocation(), "");
+ }
+ fieldBindings.add(getFieldBinding(projection, fieldNames));
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, projection.getSourceLocation(),
+ "");
}
}
if (!fieldBindings.isEmpty()) {
@@ -835,6 +886,17 @@
}
}
+ private Expression translateProjectVarStar(Expression projectionExpr, SourceLocation sourceLoc) {
+ // var.* -> if_missing_or_null(to_object(var), {})
+ CallExpr toObjectExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.TO_OBJECT),
+ Collections.singletonList(projectionExpr));
+ toObjectExpr.setSourceLocation(sourceLoc);
+ CallExpr ifMissingOrNullExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.IF_MISSING_OR_NULL),
+ Arrays.asList(toObjectExpr, new RecordConstructor(Collections.emptyList())));
+ ifMissingOrNullExpr.setSourceLocation(sourceLoc);
+ return ifMissingOrNullExpr;
+ }
+
private static boolean includeInSelectStar(VariableExpr varExpr) {
boolean excludeFromSelectStar =
varExpr.hasHints() && varExpr.getHints().contains(ExcludeFromSelectStarAnnotation.INSTANCE);
@@ -1178,16 +1240,16 @@
}
List<Expression> orderExprList = winExpr.getOrderbyList();
List<OrderbyClause.OrderModifier> orderModifierList = winExpr.getOrderbyModifierList();
+ List<OrderbyClause.NullOrderModifier> nullOrderModifierList = winExpr.getOrderbyNullModifierList();
orderExprCount = orderExprList.size();
orderExprListOut = new ArrayList<>(orderExprCount);
for (int i = 0; i < orderExprCount; i++) {
Expression orderExpr = orderExprList.get(i);
OrderbyClause.OrderModifier orderModifier = orderModifierList.get(i);
+ OrderbyClause.NullOrderModifier nullOrderModifier = nullOrderModifierList.get(i);
Pair<ILogicalOperator, LogicalVariable> orderExprResult = orderExpr.accept(this, currentOpRef);
VariableReferenceExpression orderExprOut = new VariableReferenceExpression(orderExprResult.second);
- orderExprOut.setSourceLocation(orderExpr.getSourceLocation());
- OrderOperator.IOrder orderModifierOut = translateOrderModifier(orderModifier);
- orderExprListOut.add(new Pair<>(orderModifierOut, new MutableObject<>(orderExprOut)));
+ addOrderByExpression(orderExprListOut, orderExprOut, orderModifier, nullOrderModifier);
currentOpRef = new MutableObject<>(orderExprResult.first);
}
} else if (winExpr.hasFrameDefinition()) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java
index 33653ad..91629b6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TranslationContext.java
@@ -123,6 +123,13 @@
}
}
+ /*
+ * Whether the translation is inside a subplan
+ */
+ public boolean inSubplan() {
+ return !stack.isEmpty();
+ }
+
/**
* @return the variables produced by the top operator in a subplan.
*/
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
index 6e02fc2..c482269 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
@@ -313,6 +313,9 @@
IAType[] fldTypes = new IAType[n];
int i = 0;
for (String s : names) {
+ if (names.indexOf(s) < i) {
+ throw new CompilationException(ErrorCode.DUPLICATE_FIELD_NAME, rtd.getSourceLocation(), s);
+ }
fldNames[i++] = s;
}
boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index 13bcfb6..0d6a452 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -19,12 +19,17 @@
package org.apache.asterix.translator.util;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
+import java.util.stream.Collectors;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.statement.CreateViewStatement;
+import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.IAType;
@@ -37,6 +42,9 @@
* or a list of key fields are valid in a record type.
*/
public class ValidateUtil {
+
+ private static final String PRIMARY = "primary";
+
private ValidateUtil() {
}
@@ -117,11 +125,19 @@
public static List<IAType> validatePartitioningExpressions(ARecordType recType, ARecordType metaRecType,
List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated,
SourceLocation sourceLoc) throws AlgebricksException {
+ return validatePartitioningExpressionsImpl(recType, metaRecType, partitioningExprs, keySourceIndicators,
+ autogenerated, true, sourceLoc);
+ }
+
+ private static List<IAType> validatePartitioningExpressionsImpl(ARecordType recType, ARecordType metaRecType,
+ List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated,
+ boolean forPrimaryKey, SourceLocation sourceLoc) throws AlgebricksException {
+ String keyKindDisplayName = forPrimaryKey ? PRIMARY : "";
List<IAType> partitioningExprTypes = new ArrayList<>(partitioningExprs.size());
if (autogenerated) {
if (partitioningExprs.size() > 1) {
- throw new CompilationException(ErrorCode.COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY,
- sourceLoc);
+ throw new CompilationException(ErrorCode.COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_KEY, sourceLoc,
+ keyKindDisplayName);
}
List<String> fieldName = partitioningExprs.get(0);
IAType fieldType = recType.getSubFieldType(fieldName);
@@ -133,7 +149,7 @@
ATypeTag pkTypeTag = fieldType.getTypeTag();
if (pkTypeTag != ATypeTag.UUID) {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_AUTOGENERATED_TYPE, sourceLoc,
- pkTypeTag.name(), ATypeTag.UUID.name());
+ keyKindDisplayName, pkTypeTag.name(), ATypeTag.UUID.name());
}
} else {
partitioningExprTypes =
@@ -145,12 +161,16 @@
throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc,
RecordUtil.toFullyQualifiedName(partitioningExpr));
}
- boolean nullable = KeyFieldTypeUtil.chooseSource(keySourceIndicators, i, recType, metaRecType)
- .isSubFieldNullable(partitioningExpr);
- if (nullable) {
- // key field is nullable
- throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE, sourceLoc,
- RecordUtil.toFullyQualifiedName(partitioningExpr));
+ if (forPrimaryKey) {
+ boolean nullable = KeyFieldTypeUtil.chooseSource(keySourceIndicators, i, recType, metaRecType)
+ .isSubFieldNullable(partitioningExpr);
+ if (nullable) {
+ // key field is nullable
+ throw new CompilationException(ErrorCode.COMPILATION_KEY_CANNOT_BE_NULLABLE, sourceLoc,
+ keyKindDisplayName, RecordUtil.toFullyQualifiedName(partitioningExpr));
+ }
+ } else {
+ fieldType = TypeComputeUtils.getActualType(fieldType);
}
switch (fieldType.getTypeTag()) {
case TINYINT:
@@ -169,11 +189,11 @@
case DAYTIMEDURATION:
break;
case UNION:
- throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE, sourceLoc,
- RecordUtil.toFullyQualifiedName(partitioningExpr));
+ throw new CompilationException(ErrorCode.COMPILATION_KEY_CANNOT_BE_NULLABLE, sourceLoc,
+ keyKindDisplayName, RecordUtil.toFullyQualifiedName(partitioningExpr));
default:
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE, sourceLoc,
- fieldType.getTypeTag());
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_KEY_TYPE, sourceLoc,
+ fieldType.getTypeTag(), keyKindDisplayName);
}
}
}
@@ -210,7 +230,6 @@
case DATE:
case TIME:
case DATETIME:
- case UNION:
case UUID:
case YEARMONTHDURATION:
case DAYTIMEDURATION:
@@ -229,7 +248,6 @@
case CIRCLE:
case POLYGON:
case GEOMETRY:
- case UNION:
break;
default:
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
@@ -238,14 +256,10 @@
}
break;
case LENGTH_PARTITIONED_NGRAM_INVIX:
- switch (fieldType.getTypeTag()) {
- case STRING:
- case UNION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the Length Partitioned N-Gram index.");
+ if (fieldType.getTypeTag() != ATypeTag.STRING) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the Length Partitioned N-Gram index.");
}
break;
case LENGTH_PARTITIONED_WORD_INVIX:
@@ -253,7 +267,6 @@
case STRING:
case MULTISET:
case ARRAY:
- case UNION:
break;
default:
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
@@ -262,14 +275,10 @@
}
break;
case SINGLE_PARTITION_NGRAM_INVIX:
- switch (fieldType.getTypeTag()) {
- case STRING:
- case UNION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the N-Gram index.");
+ if (fieldType.getTypeTag() != ATypeTag.STRING) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the N-Gram index.");
}
break;
case SINGLE_PARTITION_WORD_INVIX:
@@ -277,7 +286,6 @@
case STRING:
case MULTISET:
case ARRAY:
- case UNION:
break;
default:
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
@@ -290,4 +298,33 @@
String.valueOf(indexType));
}
}
+
+ /**
+ * Validates the key fields that will be used as either primary or foreign keys of a view.
+ */
+ public static List<String> validateViewKeyFields(CreateViewStatement.KeyDecl keyDecl, ARecordType itemType,
+ boolean isForeignKey, SourceLocation sourceLoc) throws AlgebricksException {
+ List<Integer> sourceIndicators = keyDecl.getSourceIndicators();
+ List<List<String>> fields = keyDecl.getFields();
+ int n = fields.size();
+ List<String> keyFields = new ArrayList<>(n);
+ for (int i = 0; i < n; i++) {
+ if (sourceIndicators.get(i) != Index.RECORD_INDICATOR) {
+ throw new CompilationException(isForeignKey ? ErrorCode.INVALID_FOREIGN_KEY_DEFINITION
+ : ErrorCode.INVALID_PRIMARY_KEY_DEFINITION, sourceLoc);
+ }
+ List<String> nestedField = fields.get(i);
+ if (nestedField.size() != 1) {
+ throw new CompilationException(isForeignKey ? ErrorCode.INVALID_FOREIGN_KEY_DEFINITION
+ : ErrorCode.INVALID_PRIMARY_KEY_DEFINITION, sourceLoc);
+ }
+ keyFields.add(nestedField.get(0));
+ }
+
+ validatePartitioningExpressionsImpl(itemType, null,
+ keyFields.stream().map(Collections::singletonList).collect(Collectors.toList()),
+ Collections.nCopies(keyFields.size(), Index.RECORD_INDICATOR), false, !isForeignKey, sourceLoc);
+
+ return keyFields;
+ }
}
diff --git a/asterixdb/asterix-app/data/array-index.adm b/asterixdb/asterix-app/data/array-index.adm
new file mode 100644
index 0000000..fbe2221
--- /dev/null
+++ b/asterixdb/asterix-app/data/array-index.adm
@@ -0,0 +1,500 @@
+{"grouping_1000":1, "grouping_500":1, "grouping_250":1, "grouping_100":1, "integer_rand_2000":null, "integer_seq_2000":1, "integer_rand_2":0, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":0, "string_rand_26_c":0}
+{"grouping_1000":2, "grouping_500":2, "grouping_250":2, "grouping_100":2, "integer_rand_2000":null, "integer_seq_2000":2, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_b":0, "string_rand_26_c":0}
+{"grouping_1000":3, "grouping_500":3, "grouping_250":3, "grouping_100":3, "integer_rand_2000":78, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":18, "string_rand_26_a":0, "string_rand_26_b":0, "string_rand_26_c":null}
+{"grouping_1000":4, "grouping_500":4, "grouping_250":4, "grouping_100":4, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_20":null, "string_rand_26_a":0, "string_rand_26_b":null, "string_rand_26_c":0}
+{"grouping_1000":5, "grouping_500":5, "grouping_250":5, "grouping_100":5, "integer_rand_2000":null, "integer_seq_2000":5, "integer_rand_2":null, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":0, "string_rand_26_b":0, "string_rand_26_c":null}
+{"grouping_1000":6, "grouping_500":6, "grouping_250":6, "grouping_100":6, "integer_seq_2000":null, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":0}
+{"grouping_1000":7, "grouping_500":7, "grouping_250":7, "grouping_100":7, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":null, "string_rand_26_b":0, "string_rand_26_c":0}
+{"grouping_1000":8, "grouping_500":8, "grouping_250":8, "grouping_100":8, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":9, "grouping_500":9, "grouping_250":9, "grouping_100":9, "integer_rand_2000":null, "integer_seq_2000":9, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":14, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":0}
+{"grouping_1000":10, "grouping_500":10, "grouping_250":10, "grouping_100":10, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_c":0}
+{"grouping_1000":11, "grouping_500":11, "grouping_250":11, "grouping_100":11, "integer_rand_2000":null, "integer_seq_2000":11, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_b":0, "string_rand_26_c":null}
+{"grouping_1000":12, "grouping_500":12, "grouping_250":12, "grouping_100":12, "integer_rand_2000":312, "integer_seq_2000":12, "integer_rand_2":null, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":0, "string_rand_26_b":0, "string_rand_26_c":0}
+{"grouping_1000":13, "grouping_500":13, "grouping_250":13, "grouping_100":13, "integer_rand_2000":338, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":0, "string_rand_26_b":0, "string_rand_26_c":0}
+{"grouping_1000":14, "grouping_500":14, "grouping_250":14, "grouping_100":14, "integer_rand_2000":364, "integer_seq_2000":14, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":4, "string_rand_26_a":0, "string_rand_26_b":null, "string_rand_26_c":0}
+{"grouping_1000":15, "grouping_500":15, "grouping_250":15, "grouping_100":15, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":null, "string_rand_26_a":0, "string_rand_26_b":0, "string_rand_26_c":0}
+{"grouping_1000":16, "grouping_500":16, "grouping_250":16, "grouping_100":16, "integer_rand_2000":416, "integer_seq_2000":16, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_20":16, "string_rand_26_c":null}
+{"grouping_1000":17, "grouping_500":17, "grouping_250":17, "grouping_100":17, "integer_rand_2000":442, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":0, "string_rand_26_b":0, "string_rand_26_c":0}
+{"grouping_1000":18, "grouping_500":18, "grouping_250":18, "grouping_100":18, "integer_rand_2000":468, "integer_rand_2":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":0, "string_rand_26_b":null, "string_rand_26_c":0}
+{"grouping_1000":19, "grouping_500":19, "grouping_250":19, "grouping_100":19, "integer_rand_2000":494, "integer_seq_2000":19, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":14, "string_rand_26_a":0, "string_rand_26_c":0}
+{"grouping_1000":20, "grouping_500":20, "grouping_250":20, "grouping_100":20, "integer_rand_2000":403, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null}
+{"grouping_1000":21, "grouping_500":21, "grouping_250":21, "grouping_100":21, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":18, "string_rand_26_b":18, "string_rand_26_c":null}
+{"grouping_1000":22, "grouping_500":22, "grouping_250":22, "grouping_100":22, "integer_rand_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":5, "integer_rand_20":5, "string_rand_26_a":null, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":23, "grouping_500":23, "grouping_250":23, "grouping_100":23, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":null}
+{"grouping_1000":24, "grouping_500":24, "grouping_250":24, "grouping_100":24, "integer_rand_2000":80, "integer_seq_2000":24, "integer_rand_2":null, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_a":null, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":25, "grouping_500":25, "grouping_250":25, "grouping_100":25, "integer_rand_2000":485, "integer_seq_2000":25, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":5, "string_rand_26_a":null, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":26, "grouping_500":26, "grouping_250":26, "grouping_100":26, "integer_rand_2000":423, "integer_seq_2000":26, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "string_rand_26_a":7, "string_rand_26_b":7, "string_rand_26_c":null}
+{"grouping_1000":27, "grouping_500":27, "grouping_250":27, "grouping_100":27, "integer_rand_2000":null, "integer_seq_2000":27, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":10, "string_rand_26_a":12, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":28, "grouping_500":28, "grouping_250":28, "grouping_100":28, "integer_rand_2000":null, "integer_seq_2000":28, "integer_rand_4":3, "integer_rand_10":9, "integer_rand_20":19, "string_rand_26_a":17, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":29, "grouping_500":29, "grouping_250":29, "grouping_100":29, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":13, "string_rand_26_a":17, "string_rand_26_b":null, "string_rand_26_c":17}
+{"grouping_1000":30, "grouping_500":30, "grouping_250":30, "grouping_100":30, "integer_rand_2000":null, "integer_seq_2000":30, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_20":null, "string_rand_26_b":22}
+{"grouping_1000":31, "grouping_500":31, "grouping_250":31, "grouping_100":31, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":11, "string_rand_26_b":11, "string_rand_26_c":null}
+{"grouping_1000":32, "grouping_500":32, "grouping_250":32, "grouping_100":32, "integer_seq_2000":32, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":3, "string_rand_26_a":null, "string_rand_26_b":1, "string_rand_26_c":1}
+{"grouping_1000":33, "grouping_500":33, "grouping_250":33, "grouping_100":33, "integer_rand_2000":110, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":null, "string_rand_26_a":6, "string_rand_26_b":null, "string_rand_26_c":6}
+{"grouping_1000":34, "grouping_500":34, "grouping_250":34, "grouping_100":34, "integer_rand_2000":null, "integer_seq_2000":34, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":22, "string_rand_26_c":22}
+{"grouping_1000":35, "grouping_500":35, "grouping_250":35, "grouping_100":35, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_b":null, "string_rand_26_c":11}
+{"grouping_1000":36, "grouping_500":36, "grouping_250":36, "grouping_100":36, "integer_rand_2000":null, "integer_seq_2000":36, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_a":16, "string_rand_26_c":null}
+{"grouping_1000":37, "grouping_500":37, "grouping_250":37, "grouping_100":37, "integer_rand_2000":299, "integer_seq_2000":37, "integer_rand_4":3, "integer_rand_10":9, "integer_rand_20":null, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":null}
+{"grouping_1000":38, "grouping_500":38, "grouping_250":38, "grouping_100":38, "integer_rand_2000":null, "integer_seq_2000":38, "integer_rand_2":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":21, "string_rand_26_c":21}
+{"grouping_1000":39, "grouping_500":39, "grouping_250":39, "grouping_100":39, "integer_rand_2000":5, "integer_seq_2000":39, "integer_rand_2":1, "integer_rand_4":1, "string_rand_26_a":5, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":40, "grouping_500":40, "grouping_250":40, "grouping_100":40, "integer_rand_2000":31, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_20":11, "string_rand_26_a":null, "string_rand_26_b":5, "string_rand_26_c":null}
+{"grouping_1000":41, "grouping_500":41, "grouping_250":41, "grouping_100":41, "integer_rand_2000":57, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":null, "string_rand_26_b":5, "string_rand_26_c":5}
+{"grouping_1000":42, "grouping_500":42, "grouping_250":42, "grouping_100":42, "integer_rand_2000":83, "integer_seq_2000":42, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":5, "string_rand_26_c":5}
+{"grouping_1000":43, "grouping_500":43, "grouping_250":43, "grouping_100":43, "integer_rand_2000":109, "integer_seq_2000":43, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":9, "string_rand_26_a":5, "string_rand_26_c":5}
+{"grouping_1000":44, "grouping_500":44, "grouping_250":44, "grouping_100":44, "integer_rand_2000":135, "integer_seq_2000":44, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_20":null, "string_rand_26_a":5, "string_rand_26_b":5, "string_rand_26_c":5}
+{"grouping_1000":45, "grouping_500":45, "grouping_250":45, "grouping_100":45, "integer_rand_2000":null, "integer_seq_2000":45, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":5, "string_rand_26_b":5, "string_rand_26_c":null}
+{"grouping_1000":46, "grouping_500":46, "grouping_250":46, "grouping_100":46, "integer_rand_2000":187, "integer_seq_2000":46, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":5, "string_rand_26_b":5, "string_rand_26_c":null}
+{"grouping_1000":47, "grouping_500":47, "grouping_250":47, "grouping_100":47, "integer_rand_2000":213, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":5, "string_rand_26_b":null}
+{"grouping_1000":48, "grouping_500":48, "grouping_250":48, "grouping_100":48, "integer_rand_2000":239, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":9, "integer_rand_20":19, "string_rand_26_a":5, "string_rand_26_b":5, "string_rand_26_c":null}
+{"grouping_1000":49, "grouping_500":49, "grouping_250":49, "grouping_100":49, "integer_rand_2000":null, "integer_seq_2000":49, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":5, "string_rand_26_a":5, "string_rand_26_b":5, "string_rand_26_c":null}
+{"grouping_1000":50, "grouping_500":50, "grouping_250":50, "grouping_100":50, "integer_rand_2000":null, "integer_seq_2000":50, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":1, "string_rand_26_a":null, "string_rand_26_b":5, "string_rand_26_c":5}
+{"grouping_1000":51, "grouping_500":51, "grouping_250":51, "grouping_100":51, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":5, "string_rand_26_b":null, "string_rand_26_c":5}
+{"grouping_1000":52, "grouping_500":52, "grouping_250":52, "grouping_100":52, "integer_seq_2000":52, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":null, "string_rand_26_b":null}
+{"grouping_1000":53, "grouping_500":53, "grouping_250":53, "grouping_100":53, "integer_seq_2000":53, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "string_rand_26_a":5, "string_rand_26_b":null, "string_rand_26_c":5}
+{"grouping_1000":54, "grouping_500":54, "grouping_250":54, "grouping_100":54, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_a":5, "string_rand_26_b":null, "string_rand_26_c":5}
+{"grouping_1000":55, "grouping_500":55, "grouping_250":55, "grouping_100":55, "integer_rand_2000":null, "integer_seq_2000":55, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":1, "string_rand_26_b":null, "string_rand_26_c":5}
+{"grouping_1000":56, "grouping_500":56, "grouping_250":56, "grouping_100":56, "integer_rand_2000":447, "integer_seq_2000":56, "integer_rand_4":3, "integer_rand_20":7, "string_rand_26_a":5, "string_rand_26_b":null, "string_rand_26_c":5}
+{"grouping_1000":57, "grouping_500":57, "grouping_250":57, "grouping_100":57, "integer_rand_2000":473, "integer_seq_2000":57, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":5, "string_rand_26_b":5, "string_rand_26_c":null}
+{"grouping_1000":58, "grouping_500":58, "grouping_250":58, "grouping_100":58, "integer_rand_2000":499, "integer_seq_2000":58, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_a":null, "string_rand_26_c":5}
+{"grouping_1000":59, "grouping_500":59, "grouping_250":59, "grouping_100":59, "integer_rand_2000":95, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_20":15, "string_rand_26_a":17, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":60, "grouping_500":60, "grouping_250":60, "grouping_100":60, "integer_rand_2000":200, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_b":18, "string_rand_26_c":null}
+{"grouping_1000":61, "grouping_500":61, "grouping_250":61, "grouping_100":61, "integer_rand_2000":245, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":5, "string_rand_26_a":11, "string_rand_26_b":null, "string_rand_26_c":11}
+{"grouping_1000":62, "grouping_500":62, "grouping_250":62, "grouping_100":62, "integer_rand_2000":339, "integer_seq_2000":62, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":1}
+{"grouping_1000":63, "grouping_500":63, "grouping_250":63, "grouping_100":63, "integer_rand_2000":null, "integer_seq_2000":63, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":64, "grouping_500":64, "grouping_250":64, "grouping_100":64, "integer_rand_2000":439, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "string_rand_26_a":null, "string_rand_26_b":23, "string_rand_26_c":23}
+{"grouping_1000":65, "grouping_500":65, "grouping_250":65, "grouping_100":65, "integer_rand_2000":252, "integer_seq_2000":65, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":2, "string_rand_26_a":18, "string_rand_26_b":18, "string_rand_26_c":18}
+{"grouping_1000":66, "grouping_500":66, "grouping_250":66, "grouping_100":66, "integer_rand_2000":220, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_20":0, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":67, "grouping_500":67, "grouping_250":67, "grouping_100":67, "integer_rand_2000":89, "integer_seq_2000":67, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_b":null, "string_rand_26_c":11}
+{"grouping_1000":68, "grouping_500":68, "grouping_250":68, "grouping_100":68, "integer_rand_2000":null, "integer_seq_2000":68, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":69, "grouping_500":69, "grouping_250":69, "grouping_100":69, "integer_rand_2000":230, "integer_seq_2000":69, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":70, "grouping_500":70, "grouping_250":70, "grouping_100":70, "integer_rand_2000":349, "integer_seq_2000":70, "integer_rand_4":1, "integer_rand_10":9, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":11}
+{"grouping_1000":71, "grouping_500":71, "grouping_250":71, "grouping_100":71, "integer_seq_2000":71, "integer_rand_4":3, "integer_rand_20":19, "string_rand_26_a":null, "string_rand_26_b":11, "string_rand_26_c":null}
+{"grouping_1000":72, "grouping_500":72, "grouping_250":72, "grouping_100":72, "integer_rand_2000":240, "integer_seq_2000":72, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_b":6, "string_rand_26_c":6}
+{"grouping_1000":73, "grouping_500":73, "grouping_250":73, "grouping_100":73, "integer_rand_2000":12, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":12, "string_rand_26_b":12, "string_rand_26_c":12}
+{"grouping_1000":74, "grouping_500":74, "grouping_250":74, "grouping_100":74, "integer_rand_2000":23, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":23}
+{"grouping_1000":75, "grouping_500":75, "grouping_250":75, "grouping_100":75, "integer_rand_2000":null, "integer_seq_2000":75, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "string_rand_26_a":null, "string_rand_26_b":16, "string_rand_26_c":16}
+{"grouping_1000":76, "grouping_500":76, "grouping_250":76, "grouping_100":76, "integer_rand_2000":396, "integer_seq_2000":76, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":16, "string_rand_26_a":null, "string_rand_26_b":null}
+{"grouping_1000":77, "grouping_500":77, "grouping_250":77, "grouping_100":77, "integer_seq_2000":77, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_20":3, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":23}
+{"grouping_1000":78, "grouping_500":78, "grouping_250":78, "grouping_100":78, "integer_rand_2000":null, "integer_seq_2000":78, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "string_rand_26_a":10, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":79, "grouping_500":79, "grouping_250":79, "grouping_100":79, "integer_rand_2000":36, "integer_seq_2000":79, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":16, "string_rand_26_a":10, "string_rand_26_c":10}
+{"grouping_1000":80, "grouping_500":80, "grouping_250":80, "grouping_100":80, "integer_rand_2000":62, "integer_seq_2000":80, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":81, "grouping_500":81, "grouping_250":81, "grouping_100":81, "integer_rand_2000":null, "integer_seq_2000":81, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":10, "string_rand_26_b":10, "string_rand_26_c":null}
+{"grouping_1000":82, "grouping_500":82, "grouping_250":82, "grouping_100":82, "integer_rand_2000":114, "integer_seq_2000":82, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":10, "string_rand_26_b":null, "string_rand_26_c":10}
+{"grouping_1000":83, "grouping_500":83, "grouping_250":83, "grouping_100":83, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":10}
+{"grouping_1000":84, "grouping_500":84, "grouping_250":84, "grouping_100":84, "integer_rand_2000":null, "integer_seq_2000":84, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":10, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":85, "grouping_500":85, "grouping_250":85, "grouping_100":85, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":null, "string_rand_26_b":10}
+{"grouping_1000":86, "grouping_500":86, "grouping_250":86, "grouping_100":86, "integer_rand_2000":218, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":10, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":87, "grouping_500":87, "grouping_250":87, "grouping_100":87, "integer_rand_2000":null, "integer_seq_2000":87, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":88, "grouping_500":88, "grouping_250":88, "grouping_100":88, "integer_rand_2000":270, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":10, "string_rand_26_a":null, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":89, "grouping_500":89, "grouping_250":89, "grouping_100":89, "integer_rand_2000":296, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":10}
+{"grouping_1000":90, "grouping_500":90, "grouping_250":90, "grouping_100":90, "integer_rand_2000":null, "integer_seq_2000":90, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "string_rand_26_a":10, "string_rand_26_b":10}
+{"grouping_1000":91, "grouping_500":91, "grouping_250":91, "grouping_100":91, "integer_rand_2000":348, "integer_seq_2000":91, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":10, "string_rand_26_b":null, "string_rand_26_c":10}
+{"grouping_1000":92, "grouping_500":92, "grouping_250":92, "grouping_100":92, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":14, "string_rand_26_a":null, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":93, "grouping_500":93, "grouping_250":93, "grouping_100":93, "integer_seq_2000":93, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_20":0, "string_rand_26_a":10, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":94, "grouping_500":94, "grouping_250":94, "grouping_100":94, "integer_rand_2000":426, "integer_seq_2000":94, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_b":10}
+{"grouping_1000":95, "grouping_500":95, "grouping_250":95, "grouping_100":95, "integer_rand_2000":452, "integer_seq_2000":95, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":96, "grouping_500":96, "grouping_250":96, "grouping_100":96, "integer_rand_2000":478, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":10, "string_rand_26_b":10, "string_rand_26_c":10}
+{"grouping_1000":97, "grouping_500":97, "grouping_250":97, "grouping_100":97, "integer_rand_2000":null, "integer_seq_2000":97, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":7, "string_rand_26_b":7, "string_rand_26_c":7}
+{"grouping_1000":98, "grouping_500":98, "grouping_250":98, "grouping_100":98, "integer_rand_2000":85, "integer_seq_2000":98, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_20":5, "string_rand_26_a":7, "string_rand_26_b":7, "string_rand_26_c":7}
+{"grouping_1000":99, "grouping_500":99, "grouping_250":99, "grouping_100":99, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_10":null, "integer_rand_20":10, "string_rand_26_b":18, "string_rand_26_c":null}
+{"grouping_1000":100, "grouping_500":100, "grouping_250":100, "grouping_100":0, "integer_rand_2000":303, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":17}
+{"grouping_1000":101, "grouping_500":101, "grouping_250":101, "grouping_100":1, "integer_rand_2000":345, "integer_seq_2000":null, "integer_rand_4":1, "integer_rand_20":5, "string_rand_26_a":null, "string_rand_26_b":7, "string_rand_26_c":null}
+{"grouping_1000":102, "grouping_500":102, "grouping_250":102, "grouping_100":2, "integer_rand_2000":340, "integer_seq_2000":102, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_a":2, "string_rand_26_b":null, "string_rand_26_c":2}
+{"grouping_1000":103, "grouping_500":103, "grouping_250":103, "grouping_100":3, "integer_rand_2000":7, "integer_seq_2000":103, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_20":7, "string_rand_26_a":null, "string_rand_26_b":7}
+{"grouping_1000":104, "grouping_500":104, "grouping_250":104, "grouping_100":4, "integer_rand_2000":335, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":null}
+{"grouping_1000":105, "grouping_500":105, "grouping_250":105, "grouping_100":5, "integer_rand_2000":350, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":null, "string_rand_26_b":12}
+{"grouping_1000":106, "grouping_500":106, "grouping_250":106, "grouping_100":6, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":17, "string_rand_26_b":null, "string_rand_26_c":17}
+{"grouping_1000":107, "grouping_500":107, "grouping_250":107, "grouping_100":7, "integer_rand_2000":292, "integer_seq_2000":107, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":12, "string_rand_26_a":6, "string_rand_26_b":null, "string_rand_26_c":6}
+{"grouping_1000":108, "grouping_500":108, "grouping_250":108, "grouping_100":8, "integer_rand_2000":360, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_a":null, "string_rand_26_b":22, "string_rand_26_c":22}
+{"grouping_1000":109, "grouping_500":109, "grouping_250":109, "grouping_100":9, "integer_seq_2000":109, "integer_rand_2":null, "integer_rand_20":7, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":null}
+{"grouping_1000":110, "grouping_500":110, "grouping_250":110, "grouping_100":10, "integer_rand_2000":116, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":12, "string_rand_26_b":12, "string_rand_26_c":12}
+{"grouping_1000":111, "grouping_500":111, "grouping_250":111, "grouping_100":11, "integer_seq_2000":111, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":6, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":112, "grouping_500":112, "grouping_250":112, "grouping_100":12, "integer_rand_2000":37, "integer_seq_2000":112, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":11, "string_rand_26_b":11, "string_rand_26_c":null}
+{"grouping_1000":113, "grouping_500":113, "grouping_250":113, "grouping_100":13, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_a":12, "string_rand_26_c":12}
+{"grouping_1000":114, "grouping_500":114, "grouping_250":114, "grouping_100":14, "integer_rand_2000":null, "integer_seq_2000":114, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_a":16, "string_rand_26_c":16}
+{"grouping_1000":115, "grouping_500":115, "grouping_250":115, "grouping_100":15, "integer_rand_2000":47, "integer_seq_2000":115, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":21, "string_rand_26_b":21, "string_rand_26_c":null}
+{"grouping_1000":116, "grouping_500":116, "grouping_250":116, "grouping_100":16, "integer_rand_2000":392, "integer_seq_2000":116, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":117, "grouping_500":117, "grouping_250":117, "grouping_100":17, "integer_rand_2000":15, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":null, "string_rand_26_a":15, "string_rand_26_b":15, "string_rand_26_c":15}
+{"grouping_1000":118, "grouping_500":118, "grouping_250":118, "grouping_100":18, "integer_rand_2000":41, "integer_seq_2000":118, "integer_rand_2":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":15, "string_rand_26_b":15}
+{"grouping_1000":119, "grouping_500":119, "grouping_250":119, "grouping_100":19, "integer_rand_2000":67, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":7, "string_rand_26_a":15, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":120, "grouping_500":120, "grouping_250":120, "grouping_100":20, "integer_rand_2000":93, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":15, "string_rand_26_b":15, "string_rand_26_c":15}
+{"grouping_1000":121, "grouping_500":121, "grouping_250":121, "grouping_100":21, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":9, "integer_rand_20":19, "string_rand_26_a":15, "string_rand_26_b":15, "string_rand_26_c":15}
+{"grouping_1000":122, "grouping_500":122, "grouping_250":122, "grouping_100":22, "integer_rand_2000":145, "integer_seq_2000":122, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":5, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":15}
+{"grouping_1000":123, "grouping_500":123, "grouping_250":123, "grouping_100":23, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_20":11, "string_rand_26_a":null, "string_rand_26_b":15}
+{"grouping_1000":124, "grouping_500":124, "grouping_250":124, "grouping_100":24, "integer_rand_2000":197, "integer_seq_2000":124, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":125, "grouping_500":125, "grouping_250":125, "grouping_100":25, "integer_rand_2000":223, "integer_seq_2000":125, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "string_rand_26_a":15, "string_rand_26_b":15, "string_rand_26_c":15}
+{"grouping_1000":126, "grouping_500":126, "grouping_250":126, "grouping_100":26, "integer_rand_2000":null, "integer_seq_2000":126, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":15, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":127, "grouping_500":127, "grouping_250":127, "grouping_100":27, "integer_rand_2000":275, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_b":null, "string_rand_26_c":15}
+{"grouping_1000":128, "grouping_500":128, "grouping_250":128, "grouping_100":28, "integer_rand_2000":301, "integer_seq_2000":128, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":15, "string_rand_26_b":null}
+{"grouping_1000":129, "grouping_500":129, "grouping_250":129, "grouping_100":29, "integer_rand_2000":327, "integer_seq_2000":129, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":7, "string_rand_26_a":15, "string_rand_26_b":null, "string_rand_26_c":15}
+{"grouping_1000":130, "grouping_500":130, "grouping_250":130, "grouping_100":30, "integer_rand_2000":353, "integer_seq_2000":130, "integer_rand_2":1, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_b":15, "string_rand_26_c":null}
+{"grouping_1000":131, "grouping_500":131, "grouping_250":131, "grouping_100":31, "integer_rand_2000":379, "integer_seq_2000":131, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_20":null, "string_rand_26_a":15, "string_rand_26_b":15, "string_rand_26_c":null}
+{"grouping_1000":132, "grouping_500":132, "grouping_250":132, "grouping_100":32, "integer_rand_2000":405, "integer_seq_2000":132, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":5, "integer_rand_20":5, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":133, "grouping_500":133, "grouping_250":133, "grouping_100":33, "integer_rand_2000":431, "integer_seq_2000":133, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":11, "string_rand_26_a":15, "string_rand_26_b":null, "string_rand_26_c":15}
+{"grouping_1000":134, "grouping_500":134, "grouping_250":134, "grouping_100":34, "integer_rand_2000":null, "integer_seq_2000":134, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":17, "string_rand_26_a":15, "string_rand_26_c":15}
+{"grouping_1000":135, "grouping_500":135, "grouping_250":135, "grouping_100":35, "integer_rand_2000":483, "integer_seq_2000":135, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":15, "string_rand_26_b":null, "string_rand_26_c":15}
+{"grouping_1000":136, "grouping_500":136, "grouping_250":136, "grouping_100":36, "integer_rand_2000":117, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":13}
+{"grouping_1000":137, "grouping_500":137, "grouping_250":137, "grouping_100":37, "integer_rand_2000":438, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_20":18, "string_rand_26_a":null, "string_rand_26_b":22}
+{"grouping_1000":138, "grouping_500":138, "grouping_250":138, "grouping_100":38, "integer_rand_2000":460, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "string_rand_26_a":18, "string_rand_26_b":18, "string_rand_26_c":18}
+{"grouping_1000":139, "grouping_500":139, "grouping_250":139, "grouping_100":39, "integer_seq_2000":139, "integer_rand_2":null, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":23}
+{"grouping_1000":140, "grouping_500":140, "grouping_250":140, "grouping_100":40, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":2, "string_rand_26_c":null}
+{"grouping_1000":141, "grouping_500":141, "grouping_250":141, "grouping_100":41, "integer_rand_2000":null, "integer_seq_2000":141, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":null}
+{"grouping_1000":142, "grouping_500":142, "grouping_250":142, "grouping_100":42, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":7, "string_rand_26_c":7}
+{"grouping_1000":143, "grouping_500":143, "grouping_250":143, "grouping_100":43, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":2, "string_rand_26_c":null}
+{"grouping_1000":144, "grouping_500":144, "grouping_250":144, "grouping_100":44, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_a":12, "string_rand_26_b":12, "string_rand_26_c":12}
+{"grouping_1000":145, "grouping_500":145, "grouping_250":145, "grouping_100":45, "integer_rand_2000":147, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":17}
+{"grouping_1000":146, "grouping_500":146, "grouping_250":146, "grouping_100":46, "integer_rand_2000":null, "integer_seq_2000":146, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":147, "grouping_500":147, "grouping_250":147, "grouping_100":47, "integer_rand_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":null, "string_rand_26_b":22, "string_rand_26_c":null}
+{"grouping_1000":148, "grouping_500":148, "grouping_250":148, "grouping_100":48, "integer_rand_2000":157, "integer_seq_2000":148, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_b":1}
+{"grouping_1000":149, "grouping_500":149, "grouping_250":149, "grouping_100":49, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_10":null, "integer_rand_20":9, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":1}
+{"grouping_1000":150, "grouping_500":150, "grouping_250":150, "grouping_100":50, "integer_rand_2000":null, "integer_seq_2000":150, "integer_rand_2":0, "integer_rand_20":null, "string_rand_26_a":6, "string_rand_26_b":null}
+{"grouping_1000":151, "grouping_500":151, "grouping_250":151, "grouping_100":51, "integer_rand_2000":167, "integer_seq_2000":151, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":11, "string_rand_26_b":null}
+{"grouping_1000":152, "grouping_500":152, "grouping_250":152, "grouping_100":52, "integer_rand_2000":412, "integer_seq_2000":152, "integer_rand_2":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":22}
+{"grouping_1000":153, "grouping_500":153, "grouping_250":153, "grouping_100":53, "integer_rand_2000":null, "integer_seq_2000":153, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":null, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":154, "grouping_500":154, "grouping_250":154, "grouping_100":54, "integer_rand_2000":null, "integer_seq_2000":154, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":null, "string_rand_26_b":21, "string_rand_26_c":21}
+{"grouping_1000":155, "grouping_500":155, "grouping_250":155, "grouping_100":55, "integer_rand_2000":489, "integer_seq_2000":155, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "string_rand_26_a":21, "string_rand_26_b":null, "string_rand_26_c":21}
+{"grouping_1000":156, "grouping_500":156, "grouping_250":156, "grouping_100":56, "integer_rand_2000":20, "integer_rand_2":0, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_b":20, "string_rand_26_c":20}
+{"grouping_1000":157, "grouping_500":157, "grouping_250":157, "grouping_100":57, "integer_rand_2000":null, "integer_seq_2000":157, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":20, "string_rand_26_c":20}
+{"grouping_1000":158, "grouping_500":158, "grouping_250":158, "grouping_100":58, "integer_rand_2000":72, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":20}
+{"grouping_1000":159, "grouping_500":159, "grouping_250":159, "grouping_100":59, "integer_rand_2000":98, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":null, "string_rand_26_b":20}
+{"grouping_1000":160, "grouping_500":160, "grouping_250":160, "grouping_100":60, "integer_rand_2000":null, "integer_seq_2000":160, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_20":4, "string_rand_26_a":20, "string_rand_26_b":20, "string_rand_26_c":20}
+{"grouping_1000":161, "grouping_500":161, "grouping_250":161, "grouping_100":61, "integer_rand_2000":150, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_b":null, "string_rand_26_c":20}
+{"grouping_1000":162, "grouping_500":162, "grouping_250":162, "grouping_100":62, "integer_rand_2000":176, "integer_seq_2000":162, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_a":20, "string_rand_26_b":20, "string_rand_26_c":20}
+{"grouping_1000":163, "grouping_500":163, "grouping_250":163, "grouping_100":63, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":2, "string_rand_26_a":20, "string_rand_26_b":20, "string_rand_26_c":null}
+{"grouping_1000":164, "grouping_500":164, "grouping_250":164, "grouping_100":64, "integer_rand_2000":228, "integer_seq_2000":164, "integer_rand_2":0, "integer_rand_10":null, "integer_rand_20":8, "string_rand_26_a":null, "string_rand_26_b":null}
+{"grouping_1000":165, "grouping_500":165, "grouping_250":165, "grouping_100":65, "integer_rand_2000":254, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":14, "string_rand_26_a":20, "string_rand_26_b":null, "string_rand_26_c":20}
+{"grouping_1000":166, "grouping_500":166, "grouping_250":166, "grouping_100":66, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":20}
+{"grouping_1000":167, "grouping_500":167, "grouping_250":167, "grouping_100":67, "integer_rand_2000":306, "integer_seq_2000":167, "integer_rand_2":null, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_a":20, "string_rand_26_b":null, "string_rand_26_c":20}
+{"grouping_1000":168, "grouping_500":168, "grouping_250":168, "grouping_100":68, "integer_rand_2000":332, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":20, "string_rand_26_b":null, "string_rand_26_c":20}
+{"grouping_1000":169, "grouping_500":169, "grouping_250":169, "grouping_100":69, "integer_seq_2000":169, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":20, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":170, "grouping_500":170, "grouping_250":170, "grouping_100":70, "integer_rand_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":171, "grouping_500":171, "grouping_250":171, "grouping_100":71, "integer_rand_2000":410, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":20, "string_rand_26_c":20}
+{"grouping_1000":172, "grouping_500":172, "grouping_250":172, "grouping_100":72, "integer_rand_2000":436, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":20, "string_rand_26_c":20}
+{"grouping_1000":173, "grouping_500":173, "grouping_250":173, "grouping_100":73, "integer_rand_2000":462, "integer_seq_2000":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":2, "string_rand_26_a":20, "string_rand_26_b":20, "string_rand_26_c":20}
+{"grouping_1000":174, "grouping_500":174, "grouping_250":174, "grouping_100":74, "integer_rand_2000":null, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":20, "string_rand_26_b":20, "string_rand_26_c":null}
+{"grouping_1000":175, "grouping_500":175, "grouping_250":175, "grouping_100":75, "integer_rand_2000":null, "integer_seq_2000":175, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":176, "grouping_500":176, "grouping_250":176, "grouping_100":76, "integer_rand_2000":386, "integer_seq_2000":176, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":22, "string_rand_26_b":null, "string_rand_26_c":22}
+{"grouping_1000":177, "grouping_500":177, "grouping_250":177, "grouping_100":77, "integer_seq_2000":177, "integer_rand_2":1, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":23}
+{"grouping_1000":178, "grouping_500":178, "grouping_250":178, "grouping_100":78, "integer_rand_2000":257, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":23}
+{"grouping_1000":179, "grouping_500":179, "grouping_250":179, "grouping_100":79, "integer_rand_2000":42, "integer_seq_2000":179, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":16, "string_rand_26_b":16, "string_rand_26_c":null}
+{"grouping_1000":180, "grouping_500":180, "grouping_250":180, "grouping_100":80, "integer_rand_2000":465, "integer_seq_2000":180, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":23, "string_rand_26_c":23}
+{"grouping_1000":181, "grouping_500":181, "grouping_250":181, "grouping_100":81, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":7, "string_rand_26_a":7, "string_rand_26_b":7, "string_rand_26_c":7}
+{"grouping_1000":182, "grouping_500":182, "grouping_250":182, "grouping_100":82, "integer_rand_2000":302, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":16, "string_rand_26_b":16, "string_rand_26_c":16}
+{"grouping_1000":183, "grouping_500":183, "grouping_250":183, "grouping_100":83, "integer_rand_2000":432, "integer_seq_2000":183, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":2, "string_rand_26_a":16, "string_rand_26_b":16, "string_rand_26_c":null}
+{"grouping_1000":184, "grouping_500":184, "grouping_250":184, "grouping_100":84, "integer_rand_2000":null, "integer_seq_2000":184, "integer_rand_2":1, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":17, "string_rand_26_c":null}
+{"grouping_1000":185, "grouping_500":185, "grouping_250":185, "grouping_100":85, "integer_rand_2000":486, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":18, "string_rand_26_c":null}
+{"grouping_1000":186, "grouping_500":186, "grouping_250":186, "grouping_100":86, "integer_rand_2000":null, "integer_seq_2000":186, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":187, "grouping_500":187, "grouping_250":187, "grouping_100":87, "integer_seq_2000":187, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_20":null, "string_rand_26_b":1, "string_rand_26_c":1}
+{"grouping_1000":188, "grouping_500":188, "grouping_250":188, "grouping_100":88, "integer_rand_2000":183, "integer_seq_2000":188, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":1, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":189, "grouping_500":189, "grouping_250":189, "grouping_100":89, "integer_rand_2000":null, "integer_seq_2000":189, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":16, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":190, "grouping_500":190, "grouping_250":190, "grouping_100":90, "integer_rand_2000":297, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":17, "string_rand_26_a":11, "string_rand_26_b":11, "string_rand_26_c":11}
+{"grouping_1000":191, "grouping_500":191, "grouping_250":191, "grouping_100":91, "integer_rand_2000":73, "integer_seq_2000":191, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":21, "string_rand_26_c":21}
+{"grouping_1000":192, "grouping_500":192, "grouping_250":192, "grouping_100":92, "integer_rand_2000":496, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_a":null, "string_rand_26_c":2}
+{"grouping_1000":193, "grouping_500":193, "grouping_250":193, "grouping_100":93, "integer_rand_2000":307, "integer_seq_2000":193, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":7, "string_rand_26_a":21, "string_rand_26_c":null}
+{"grouping_1000":194, "grouping_500":194, "grouping_250":194, "grouping_100":94, "integer_seq_2000":194, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":21, "string_rand_26_c":21}
+{"grouping_1000":195, "grouping_500":195, "grouping_250":195, "grouping_100":95, "integer_rand_2000":25, "integer_seq_2000":195, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":null, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":196, "grouping_500":196, "grouping_250":196, "grouping_100":96, "integer_seq_2000":196, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "string_rand_26_a":null, "string_rand_26_b":25}
+{"grouping_1000":197, "grouping_500":197, "grouping_250":197, "grouping_100":97, "integer_rand_2000":77, "integer_seq_2000":197, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":198, "grouping_500":198, "grouping_250":198, "grouping_100":98, "integer_rand_2000":null, "integer_seq_2000":198, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":3, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":199, "grouping_500":199, "grouping_250":199, "grouping_100":99, "integer_rand_2000":129, "integer_seq_2000":null, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":200, "grouping_500":200, "grouping_250":200, "grouping_100":0, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":201, "grouping_500":201, "grouping_250":201, "grouping_100":1, "integer_rand_2000":null, "integer_seq_2000":201, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":null, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":202, "grouping_500":202, "grouping_250":202, "grouping_100":2, "integer_rand_2000":null, "integer_seq_2000":202, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":203, "grouping_500":203, "grouping_250":203, "grouping_100":3, "integer_rand_2000":233, "integer_seq_2000":203, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":13, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":null}
+{"grouping_1000":204, "grouping_500":204, "grouping_250":204, "grouping_100":4, "integer_rand_2000":null, "integer_seq_2000":204, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":9, "integer_rand_20":null, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":205, "grouping_500":205, "grouping_250":205, "grouping_100":5, "integer_rand_2000":285, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":5, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":206, "grouping_500":206, "grouping_250":206, "grouping_100":6, "integer_rand_2000":311, "integer_seq_2000":null, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":207, "grouping_500":207, "grouping_250":207, "grouping_100":7, "integer_rand_2000":null, "integer_seq_2000":207, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":null, "string_rand_26_b":25, "string_rand_26_c":null}
+{"grouping_1000":208, "grouping_500":208, "grouping_250":208, "grouping_100":8, "integer_rand_2000":363, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":25, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":209, "grouping_500":209, "grouping_250":209, "grouping_100":9, "integer_rand_2000":389, "integer_seq_2000":209, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":210, "grouping_500":210, "grouping_250":210, "grouping_100":10, "integer_seq_2000":210, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":211, "grouping_500":211, "grouping_250":211, "grouping_100":11, "integer_rand_2000":441, "integer_seq_2000":211, "integer_rand_2":null, "integer_rand_4":1, "string_rand_26_a":25, "string_rand_26_b":25}
+{"grouping_1000":212, "grouping_500":212, "grouping_250":212, "grouping_100":12, "integer_rand_2000":null, "integer_seq_2000":212, "integer_rand_4":3, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":213, "grouping_500":213, "grouping_250":213, "grouping_100":13, "integer_rand_2000":null, "integer_seq_2000":213, "integer_rand_2":1, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":25, "string_rand_26_b":25, "string_rand_26_c":25}
+{"grouping_1000":214, "grouping_500":214, "grouping_250":214, "grouping_100":14, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":13, "string_rand_26_b":null, "string_rand_26_c":13}
+{"grouping_1000":215, "grouping_500":215, "grouping_250":215, "grouping_100":15, "integer_rand_2000":44, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":18, "string_rand_26_b":18, "string_rand_26_c":18}
+{"grouping_1000":216, "grouping_500":216, "grouping_250":216, "grouping_100":16, "integer_seq_2000":216, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":217, "grouping_500":217, "grouping_250":217, "grouping_100":17, "integer_rand_2000":null, "integer_seq_2000":217, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":23}
+{"grouping_1000":218, "grouping_500":218, "grouping_250":218, "grouping_100":18, "integer_rand_2000":54, "integer_seq_2000":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":219, "grouping_500":219, "grouping_250":219, "grouping_100":19, "integer_rand_2000":79, "integer_seq_2000":219, "integer_rand_2":1, "integer_rand_10":9, "integer_rand_20":19, "string_rand_26_a":1, "string_rand_26_b":null, "string_rand_26_c":1}
+{"grouping_1000":220, "grouping_500":220, "grouping_250":220, "grouping_100":20, "integer_rand_2000":null, "integer_seq_2000":220, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":7, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":221, "grouping_500":221, "grouping_250":221, "grouping_100":21, "integer_rand_2000":64, "integer_seq_2000":221, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":12, "string_rand_26_b":12, "string_rand_26_c":12}
+{"grouping_1000":222, "grouping_500":222, "grouping_250":222, "grouping_100":22, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":17}
+{"grouping_1000":223, "grouping_500":223, "grouping_250":223, "grouping_100":23, "integer_rand_2000":407, "integer_seq_2000":223, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":17, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":224, "grouping_500":224, "grouping_250":224, "grouping_100":24, "integer_rand_2000":74, "integer_rand_2":0, "integer_rand_10":null, "string_rand_26_a":22, "string_rand_26_b":22, "string_rand_26_c":22}
+{"grouping_1000":225, "grouping_500":225, "grouping_250":225, "grouping_100":25, "integer_rand_2000":329, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":null, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":226, "grouping_500":226, "grouping_250":226, "grouping_100":26, "integer_rand_2000":417, "integer_seq_2000":226, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":1}
+{"grouping_1000":227, "grouping_500":227, "grouping_250":227, "grouping_100":27, "integer_rand_2000":null, "integer_seq_2000":227, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":null, "string_rand_26_b":6, "string_rand_26_c":null}
+{"grouping_1000":228, "grouping_500":228, "grouping_250":228, "grouping_100":28, "integer_rand_2000":179, "integer_seq_2000":228, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_b":23, "string_rand_26_c":null}
+{"grouping_1000":229, "grouping_500":229, "grouping_250":229, "grouping_100":29, "integer_rand_2000":427, "integer_seq_2000":229, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":11, "string_rand_26_b":null, "string_rand_26_c":11}
+{"grouping_1000":230, "grouping_500":230, "grouping_250":230, "grouping_100":30, "integer_rand_2000":null, "integer_seq_2000":230, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":4, "integer_rand_20":14, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":16}
+{"grouping_1000":231, "grouping_500":231, "grouping_250":231, "grouping_100":31, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_b":16, "string_rand_26_c":16}
+{"grouping_1000":232, "grouping_500":232, "grouping_250":232, "grouping_100":32, "integer_rand_2000":437, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":null, "string_rand_26_a":21, "string_rand_26_c":21}
+{"grouping_1000":233, "grouping_500":233, "grouping_250":233, "grouping_100":33, "integer_rand_2000":null, "integer_seq_2000":233, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":234, "grouping_500":234, "grouping_250":234, "grouping_100":34, "integer_rand_2000":30, "integer_seq_2000":234, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":4, "string_rand_26_b":null}
+{"grouping_1000":235, "grouping_500":235, "grouping_250":235, "grouping_100":35, "integer_rand_2000":56, "integer_seq_2000":null, "integer_rand_4":0, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":4}
+{"grouping_1000":236, "grouping_500":236, "grouping_250":236, "grouping_100":36, "integer_rand_2000":82, "integer_seq_2000":236, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "string_rand_26_a":null, "string_rand_26_b":4, "string_rand_26_c":null}
+{"grouping_1000":237, "grouping_500":237, "grouping_250":237, "grouping_100":37, "integer_rand_2000":null, "integer_rand_2":0, "integer_rand_20":8, "string_rand_26_a":4, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":238, "grouping_500":238, "grouping_250":238, "grouping_100":38, "integer_rand_2000":null, "integer_seq_2000":238, "integer_rand_2":0, "integer_rand_4":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":239, "grouping_500":239, "grouping_250":239, "grouping_100":39, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":0, "string_rand_26_a":4, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":240, "grouping_500":240, "grouping_250":240, "grouping_100":40, "integer_rand_2000":186, "integer_seq_2000":240, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":241, "grouping_500":241, "grouping_250":241, "grouping_100":41, "integer_seq_2000":241, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":4, "string_rand_26_b":null}
+{"grouping_1000":242, "grouping_500":242, "grouping_250":242, "grouping_100":42, "integer_rand_2000":238, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":4, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":243, "grouping_500":243, "grouping_250":243, "grouping_100":43, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":null, "string_rand_26_c":4}
+{"grouping_1000":244, "grouping_500":244, "grouping_250":244, "grouping_100":44, "integer_rand_2000":null, "integer_seq_2000":244, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "string_rand_26_a":4, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":245, "grouping_500":245, "grouping_250":245, "grouping_100":45, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_a":4, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":246, "grouping_500":246, "grouping_250":246, "grouping_100":46, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":4, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":247, "grouping_500":247, "grouping_250":247, "grouping_100":47, "integer_rand_2000":null, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":4}
+{"grouping_1000":248, "grouping_500":248, "grouping_250":248, "grouping_100":48, "integer_rand_2000":394, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":14, "string_rand_26_a":null, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":249, "grouping_500":249, "grouping_250":249, "grouping_100":49, "integer_rand_2000":420, "integer_seq_2000":249, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_a":null, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":250, "grouping_500":250, "grouping_250":0, "grouping_100":50, "integer_rand_2000":null, "integer_seq_2000":250, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":4, "string_rand_26_c":4}
+{"grouping_1000":251, "grouping_500":251, "grouping_250":1, "grouping_100":51, "integer_rand_2000":472, "integer_seq_2000":251, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":4, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":252, "grouping_500":252, "grouping_250":2, "grouping_100":52, "integer_rand_2000":498, "integer_seq_2000":252, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":8, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":4}
+{"grouping_1000":253, "grouping_500":253, "grouping_250":3, "grouping_100":53, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":5, "string_rand_26_a":null, "string_rand_26_b":13, "string_rand_26_c":null}
+{"grouping_1000":254, "grouping_500":254, "grouping_250":4, "grouping_100":54, "integer_rand_2000":174, "integer_seq_2000":254, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":4, "integer_rand_20":14, "string_rand_26_a":18, "string_rand_26_b":18}
+{"grouping_1000":255, "grouping_500":255, "grouping_250":5, "grouping_100":55, "integer_rand_2000":null, "integer_seq_2000":255, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":9, "integer_rand_20":19, "string_rand_26_b":null, "string_rand_26_c":11}
+{"grouping_1000":256, "grouping_500":256, "grouping_250":6, "grouping_100":56, "integer_rand_2000":325, "integer_seq_2000":256, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "integer_rand_20":null, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":257, "grouping_500":257, "grouping_250":7, "grouping_100":57, "integer_rand_2000":184, "integer_seq_2000":257, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":2, "string_rand_26_b":null, "string_rand_26_c":2}
+{"grouping_1000":258, "grouping_500":258, "grouping_250":8, "grouping_100":58, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":6, "string_rand_26_c":null}
+{"grouping_1000":259, "grouping_500":259, "grouping_250":9, "grouping_100":59, "integer_rand_2000":75, "integer_seq_2000":null, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_b":23, "string_rand_26_c":null}
+{"grouping_1000":260, "grouping_500":260, "grouping_250":10, "grouping_100":60, "integer_rand_2000":194, "integer_seq_2000":260, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_c":12}
+{"grouping_1000":261, "grouping_500":261, "grouping_250":11, "grouping_100":61, "integer_rand_2000":422, "integer_seq_2000":261, "integer_rand_2":null, "integer_rand_10":2, "string_rand_26_a":null, "string_rand_26_b":6, "string_rand_26_c":6}
+{"grouping_1000":262, "grouping_500":262, "grouping_250":12, "grouping_100":62, "integer_rand_2000":null, "integer_seq_2000":262, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":22, "string_rand_26_c":22}
+{"grouping_1000":263, "grouping_500":263, "grouping_250":13, "grouping_100":63, "integer_rand_2000":204, "integer_seq_2000":263, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":4, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":264, "grouping_500":264, "grouping_250":14, "grouping_100":64, "integer_rand_2000":null, "integer_seq_2000":264, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":17, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":265, "grouping_500":265, "grouping_250":15, "grouping_100":65, "integer_rand_2000":96, "integer_seq_2000":265, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":16, "string_rand_26_a":18, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":266, "grouping_500":266, "grouping_250":16, "grouping_100":66, "integer_rand_2000":214, "integer_seq_2000":266, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":6, "string_rand_26_b":6, "string_rand_26_c":null}
+{"grouping_1000":267, "grouping_500":267, "grouping_250":17, "grouping_100":67, "integer_rand_2000":276, "integer_seq_2000":267, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_b":16, "string_rand_26_c":16}
+{"grouping_1000":268, "grouping_500":268, "grouping_250":18, "grouping_100":68, "integer_rand_2000":356, "integer_seq_2000":268, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_a":18, "string_rand_26_b":null, "string_rand_26_c":18}
+{"grouping_1000":269, "grouping_500":269, "grouping_250":19, "grouping_100":69, "integer_rand_2000":224, "integer_seq_2000":269, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":4, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":270, "grouping_500":270, "grouping_250":20, "grouping_100":70, "integer_rand_2000":193, "integer_seq_2000":270, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":null, "string_rand_26_b":null}
+{"grouping_1000":271, "grouping_500":271, "grouping_250":21, "grouping_100":71, "integer_rand_2000":246, "integer_seq_2000":271, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":12, "string_rand_26_c":12}
+{"grouping_1000":272, "grouping_500":272, "grouping_250":22, "grouping_100":72, "integer_rand_2000":9, "integer_seq_2000":272, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_b":9}
+{"grouping_1000":273, "grouping_500":273, "grouping_250":23, "grouping_100":73, "integer_rand_2000":35, "integer_seq_2000":273, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":9, "string_rand_26_b":9, "string_rand_26_c":9}
+{"grouping_1000":274, "grouping_500":274, "grouping_250":24, "grouping_100":74, "integer_rand_2000":61, "integer_seq_2000":274, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":275, "grouping_500":275, "grouping_250":25, "grouping_100":75, "integer_seq_2000":275, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":7, "string_rand_26_a":9, "string_rand_26_b":9, "string_rand_26_c":9}
+{"grouping_1000":276, "grouping_500":276, "grouping_250":26, "grouping_100":76, "integer_rand_2000":null, "integer_seq_2000":276, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":9, "string_rand_26_b":null, "string_rand_26_c":9}
+{"grouping_1000":277, "grouping_500":277, "grouping_250":27, "grouping_100":77, "integer_rand_2000":139, "integer_seq_2000":277, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_20":19, "string_rand_26_a":9, "string_rand_26_b":9}
+{"grouping_1000":278, "grouping_500":278, "grouping_250":28, "grouping_100":78, "integer_rand_2000":165, "integer_seq_2000":278, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "string_rand_26_a":9, "string_rand_26_b":9, "string_rand_26_c":9}
+{"grouping_1000":279, "grouping_500":279, "grouping_250":29, "grouping_100":79, "integer_rand_2000":191, "integer_seq_2000":279, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_b":null}
+{"grouping_1000":280, "grouping_500":280, "grouping_250":30, "grouping_100":80, "integer_rand_2000":217, "integer_seq_2000":280, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":null, "string_rand_26_b":9, "string_rand_26_c":9}
+{"grouping_1000":281, "grouping_500":281, "grouping_250":31, "grouping_100":81, "integer_rand_2000":243, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":3, "string_rand_26_a":9, "string_rand_26_c":9}
+{"grouping_1000":282, "grouping_500":282, "grouping_250":32, "grouping_100":82, "integer_rand_2000":269, "integer_seq_2000":282, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":null, "string_rand_26_b":9, "string_rand_26_c":9}
+{"grouping_1000":283, "grouping_500":283, "grouping_250":33, "grouping_100":83, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_20":null, "string_rand_26_a":9, "string_rand_26_b":9}
+{"grouping_1000":284, "grouping_500":284, "grouping_250":34, "grouping_100":84, "integer_rand_2000":null, "integer_seq_2000":284, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":9, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":285, "grouping_500":285, "grouping_250":35, "grouping_100":85, "integer_rand_2000":347, "integer_seq_2000":285, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":9, "string_rand_26_b":null}
+{"grouping_1000":286, "grouping_500":286, "grouping_250":36, "grouping_100":86, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":9, "string_rand_26_c":null}
+{"grouping_1000":287, "grouping_500":287, "grouping_250":37, "grouping_100":87, "integer_rand_2000":null, "integer_seq_2000":287, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_a":9, "string_rand_26_b":9, "string_rand_26_c":9}
+{"grouping_1000":288, "grouping_500":288, "grouping_250":38, "grouping_100":88, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "integer_rand_20":null, "string_rand_26_a":9, "string_rand_26_b":null, "string_rand_26_c":9}
+{"grouping_1000":289, "grouping_500":289, "grouping_250":39, "grouping_100":89, "integer_rand_2000":451, "integer_seq_2000":289, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":290, "grouping_500":290, "grouping_250":40, "grouping_100":90, "integer_rand_2000":477, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":9, "string_rand_26_b":9, "string_rand_26_c":null}
+{"grouping_1000":291, "grouping_500":291, "grouping_250":41, "grouping_100":91, "integer_rand_2000":13, "integer_seq_2000":291, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":13, "string_rand_26_a":null, "string_rand_26_c":13}
+{"grouping_1000":292, "grouping_500":292, "grouping_250":42, "grouping_100":92, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":293, "grouping_500":293, "grouping_250":43, "grouping_100":93, "integer_rand_2000":304, "integer_seq_2000":293, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":18, "string_rand_26_b":18, "string_rand_26_c":18}
+{"grouping_1000":294, "grouping_500":294, "grouping_250":44, "grouping_100":94, "integer_seq_2000":294, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_a":21, "string_rand_26_b":null, "string_rand_26_c":21}
+{"grouping_1000":295, "grouping_500":295, "grouping_250":45, "grouping_100":95, "integer_rand_2000":475, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_a":null, "string_rand_26_b":7, "string_rand_26_c":7}
+{"grouping_1000":296, "grouping_500":296, "grouping_250":46, "grouping_100":96, "integer_rand_2000":314, "integer_seq_2000":296, "integer_rand_2":0, "integer_rand_10":null, "integer_rand_20":14, "string_rand_26_a":2, "string_rand_26_b":null, "string_rand_26_c":2}
+{"grouping_1000":297, "grouping_500":297, "grouping_250":47, "grouping_100":97, "integer_rand_2000":null, "integer_seq_2000":297, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":null}
+{"grouping_1000":298, "grouping_500":298, "grouping_250":48, "grouping_100":98, "integer_rand_2000":172, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":16, "string_rand_26_b":16, "string_rand_26_c":16}
+{"grouping_1000":299, "grouping_500":299, "grouping_250":49, "grouping_100":99, "integer_rand_2000":324, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":12}
+{"grouping_1000":300, "grouping_500":300, "grouping_250":50, "grouping_100":0, "integer_seq_2000":300, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":21, "string_rand_26_b":21, "string_rand_26_c":21}
+{"grouping_1000":301, "grouping_500":301, "grouping_250":51, "grouping_100":1, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":7, "string_rand_26_b":null, "string_rand_26_c":7}
+{"grouping_1000":302, "grouping_500":302, "grouping_250":52, "grouping_100":2, "integer_rand_2000":334, "integer_seq_2000":302, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":14, "string_rand_26_a":22, "string_rand_26_b":22, "string_rand_26_c":22}
+{"grouping_1000":303, "grouping_500":303, "grouping_250":53, "grouping_100":3, "integer_rand_2000":1, "integer_seq_2000":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":1, "string_rand_26_b":null, "string_rand_26_c":1}
+{"grouping_1000":304, "grouping_500":304, "grouping_250":54, "grouping_100":4, "integer_rand_2000":449, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":7, "string_rand_26_b":null, "string_rand_26_c":7}
+{"grouping_1000":305, "grouping_500":305, "grouping_250":55, "grouping_100":5, "integer_rand_2000":null, "integer_seq_2000":305, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":6, "string_rand_26_b":6}
+{"grouping_1000":306, "grouping_500":306, "grouping_250":56, "grouping_100":6, "integer_rand_2000":null, "integer_seq_2000":306, "integer_rand_2":1, "integer_rand_4":null, "string_rand_26_a":11, "string_rand_26_b":11, "string_rand_26_c":11}
+{"grouping_1000":307, "grouping_500":307, "grouping_250":57, "grouping_100":7, "integer_rand_2000":272, "integer_seq_2000":307, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":12, "string_rand_26_b":null, "string_rand_26_c":12}
+{"grouping_1000":308, "grouping_500":308, "grouping_250":58, "grouping_100":8, "integer_rand_2000":354, "integer_seq_2000":null, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_b":16, "string_rand_26_c":16}
+{"grouping_1000":309, "grouping_500":309, "grouping_250":59, "grouping_100":9, "integer_rand_2000":null, "integer_seq_2000":309, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":null, "string_rand_26_b":21, "string_rand_26_c":21}
+{"grouping_1000":310, "grouping_500":310, "grouping_250":60, "grouping_100":10, "integer_rand_2000":203, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null}
+{"grouping_1000":311, "grouping_500":311, "grouping_250":61, "grouping_100":11, "integer_rand_2000":14, "integer_seq_2000":311, "integer_rand_2":0, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":14, "string_rand_26_b":null, "string_rand_26_c":14}
+{"grouping_1000":312, "grouping_500":312, "grouping_250":62, "grouping_100":12, "integer_rand_2000":40, "integer_seq_2000":null, "integer_rand_4":0, "integer_rand_10":0, "string_rand_26_a":null, "string_rand_26_b":14, "string_rand_26_c":null}
+{"grouping_1000":313, "grouping_500":313, "grouping_250":63, "grouping_100":13, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":314, "grouping_500":314, "grouping_250":64, "grouping_100":14, "integer_rand_2000":92, "integer_seq_2000":314, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":null, "string_rand_26_b":14, "string_rand_26_c":null}
+{"grouping_1000":315, "grouping_500":315, "grouping_250":65, "grouping_100":15, "integer_rand_2000":null, "integer_seq_2000":315, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":316, "grouping_500":316, "grouping_250":66, "grouping_100":16, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":14, "string_rand_26_b":null, "string_rand_26_c":14}
+{"grouping_1000":317, "grouping_500":317, "grouping_250":67, "grouping_100":17, "integer_rand_2000":170, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":10, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":318, "grouping_500":318, "grouping_250":68, "grouping_100":18, "integer_rand_2000":null, "integer_seq_2000":318, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":16, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":319, "grouping_500":319, "grouping_250":69, "grouping_100":19, "integer_rand_2000":null, "integer_seq_2000":319, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":null}
+{"grouping_1000":320, "grouping_500":320, "grouping_250":70, "grouping_100":20, "integer_rand_2000":248, "integer_seq_2000":320, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":null, "string_rand_26_c":14}
+{"grouping_1000":321, "grouping_500":321, "grouping_250":71, "grouping_100":21, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":322, "grouping_500":322, "grouping_250":72, "grouping_100":22, "integer_rand_2000":300, "integer_seq_2000":322, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_a":14, "string_rand_26_c":14}
+{"grouping_1000":323, "grouping_500":323, "grouping_250":73, "grouping_100":23, "integer_rand_2000":326, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":6, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":324, "grouping_500":324, "grouping_250":74, "grouping_100":24, "integer_rand_2000":352, "integer_seq_2000":324, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":12, "string_rand_26_a":14, "string_rand_26_b":null, "string_rand_26_c":14}
+{"grouping_1000":325, "grouping_500":325, "grouping_250":75, "grouping_100":25, "integer_rand_2000":null, "integer_seq_2000":325, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":18, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":326, "grouping_500":326, "grouping_250":76, "grouping_100":26, "integer_rand_2000":404, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":327, "grouping_500":327, "grouping_250":77, "grouping_100":27, "integer_rand_2000":430, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":14}
+{"grouping_1000":328, "grouping_500":328, "grouping_250":78, "grouping_100":28, "integer_rand_2000":456, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":329, "grouping_500":329, "grouping_250":79, "grouping_100":29, "integer_rand_2000":482, "integer_seq_2000":329, "integer_rand_4":2, "integer_rand_20":null, "string_rand_26_a":14, "string_rand_26_b":14, "string_rand_26_c":14}
+{"grouping_1000":330, "grouping_500":330, "grouping_250":80, "grouping_100":30, "integer_rand_2000":null, "integer_seq_2000":330, "integer_rand_2":1, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":13, "string_rand_26_b":13}
+{"grouping_1000":331, "grouping_500":331, "grouping_250":81, "grouping_100":31, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":18, "string_rand_26_b":18, "string_rand_26_c":18}
+{"grouping_1000":332, "grouping_500":332, "grouping_250":82, "grouping_100":32, "integer_rand_2000":null, "integer_seq_2000":332, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":18, "string_rand_26_c":18}
+{"grouping_1000":333, "grouping_500":333, "grouping_250":83, "grouping_100":33, "integer_rand_2000":101, "integer_seq_2000":333, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":null}
+{"grouping_1000":334, "grouping_500":334, "grouping_250":84, "grouping_100":34, "integer_rand_2000":22, "integer_rand_2":0, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":22, "string_rand_26_b":null, "string_rand_26_c":22}
+{"grouping_1000":335, "grouping_500":335, "grouping_250":85, "grouping_100":35, "integer_rand_2000":444, "integer_seq_2000":335, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":null, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":336, "grouping_500":336, "grouping_250":86, "grouping_100":36, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":7, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":337, "grouping_500":337, "grouping_250":87, "grouping_100":37, "integer_rand_2000":282, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":22}
+{"grouping_1000":338, "grouping_500":338, "grouping_250":88, "grouping_100":38, "integer_rand_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":14, "string_rand_26_a":12, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":339, "grouping_500":339, "grouping_250":89, "grouping_100":39, "integer_rand_2000":null, "integer_seq_2000":339, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_a":17, "string_rand_26_b":17, "string_rand_26_c":null}
+{"grouping_1000":340, "grouping_500":340, "grouping_250":90, "grouping_100":40, "integer_rand_2000":null, "integer_seq_2000":340, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "integer_rand_20":5, "string_rand_26_a":21, "string_rand_26_b":21, "string_rand_26_c":21}
+{"grouping_1000":341, "grouping_500":341, "grouping_250":91, "grouping_100":41, "integer_rand_2000":null, "integer_seq_2000":341, "integer_rand_2":null, "integer_rand_10":null, "string_rand_26_a":22, "string_rand_26_b":22, "string_rand_26_c":null}
+{"grouping_1000":342, "grouping_500":342, "grouping_250":92, "grouping_100":42, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":1}
+{"grouping_1000":343, "grouping_500":343, "grouping_250":93, "grouping_100":43, "integer_rand_2000":319, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":7}
+{"grouping_1000":344, "grouping_500":344, "grouping_250":94, "grouping_100":44, "integer_rand_2000":474, "integer_seq_2000":344, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":6, "string_rand_26_b":6, "string_rand_26_c":6}
+{"grouping_1000":345, "grouping_500":345, "grouping_250":95, "grouping_100":45, "integer_rand_2000":null, "integer_seq_2000":345, "integer_rand_2":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":11, "string_rand_26_b":11}
+{"grouping_1000":346, "grouping_500":346, "grouping_250":96, "grouping_100":46, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":1}
+{"grouping_1000":347, "grouping_500":347, "grouping_250":97, "grouping_100":47, "integer_rand_2000":484, "integer_seq_2000":347, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":16, "string_rand_26_b":null}
+{"grouping_1000":348, "grouping_500":348, "grouping_250":98, "grouping_100":48, "integer_rand_2000":151, "integer_seq_2000":348, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":11, "string_rand_26_c":21}
+{"grouping_1000":349, "grouping_500":349, "grouping_250":99, "grouping_100":49, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":13, "string_rand_26_a":1, "string_rand_26_c":1}
+{"grouping_1000":350, "grouping_500":350, "grouping_250":100, "grouping_100":50, "integer_seq_2000":350, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_a":19, "string_rand_26_b":19, "string_rand_26_c":19}
+{"grouping_1000":351, "grouping_500":351, "grouping_250":101, "grouping_100":51, "integer_rand_2000":null, "integer_seq_2000":351, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "string_rand_26_a":19, "string_rand_26_c":19}
+{"grouping_1000":352, "grouping_500":352, "grouping_250":102, "grouping_100":52, "integer_rand_2000":null, "integer_seq_2000":352, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":19, "string_rand_26_b":19}
+{"grouping_1000":353, "grouping_500":353, "grouping_250":103, "grouping_100":53, "integer_seq_2000":353, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":17, "string_rand_26_a":19, "string_rand_26_b":19, "string_rand_26_c":19}
+{"grouping_1000":354, "grouping_500":354, "grouping_250":104, "grouping_100":54, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":null, "string_rand_26_b":19, "string_rand_26_c":null}
+{"grouping_1000":355, "grouping_500":355, "grouping_250":105, "grouping_100":55, "integer_rand_2000":null, "integer_seq_2000":355, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":9, "string_rand_26_b":19, "string_rand_26_c":19}
+{"grouping_1000":356, "grouping_500":356, "grouping_250":106, "grouping_100":56, "integer_rand_2000":null, "integer_seq_2000":356, "integer_rand_4":3, "string_rand_26_a":null, "string_rand_26_b":19, "string_rand_26_c":19}
+{"grouping_1000":357, "grouping_500":357, "grouping_250":107, "grouping_100":57, "integer_rand_2000":201, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":19}
+{"grouping_1000":358, "grouping_500":358, "grouping_250":108, "grouping_100":58, "integer_rand_2000":227, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":359, "grouping_500":359, "grouping_250":109, "grouping_100":59, "integer_rand_2000":253, "integer_seq_2000":359, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":13, "string_rand_26_a":null, "string_rand_26_b":19, "string_rand_26_c":null}
+{"grouping_1000":360, "grouping_500":360, "grouping_250":110, "grouping_100":60, "integer_rand_2000":null, "integer_seq_2000":360, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_a":null, "string_rand_26_b":19, "string_rand_26_c":19}
+{"grouping_1000":361, "grouping_500":361, "grouping_250":111, "grouping_100":61, "integer_rand_2000":305, "integer_seq_2000":361, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":5, "integer_rand_20":5, "string_rand_26_a":null, "string_rand_26_b":19, "string_rand_26_c":19}
+{"grouping_1000":362, "grouping_500":362, "grouping_250":112, "grouping_100":62, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":19, "string_rand_26_b":19, "string_rand_26_c":null}
+{"grouping_1000":363, "grouping_500":363, "grouping_250":113, "grouping_100":63, "integer_seq_2000":363, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":17, "string_rand_26_a":19, "string_rand_26_b":null, "string_rand_26_c":19}
+{"grouping_1000":364, "grouping_500":364, "grouping_250":114, "grouping_100":64, "integer_seq_2000":null, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":19, "string_rand_26_c":19}
+{"grouping_1000":365, "grouping_500":365, "grouping_250":115, "grouping_100":65, "integer_rand_2000":null, "integer_seq_2000":365, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":9, "string_rand_26_a":19, "string_rand_26_b":null, "string_rand_26_c":19}
+{"grouping_1000":366, "grouping_500":366, "grouping_250":116, "grouping_100":66, "integer_rand_2000":435, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_a":19, "string_rand_26_b":null, "string_rand_26_c":19}
+{"grouping_1000":367, "grouping_500":367, "grouping_250":117, "grouping_100":67, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":368, "grouping_500":368, "grouping_250":118, "grouping_100":68, "integer_rand_2000":487, "integer_seq_2000":368, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":null, "string_rand_26_c":19}
+{"grouping_1000":369, "grouping_500":369, "grouping_250":119, "grouping_100":69, "integer_rand_2000":221, "integer_seq_2000":369, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":null, "string_rand_26_b":13, "string_rand_26_c":null}
+{"grouping_1000":370, "grouping_500":370, "grouping_250":120, "grouping_100":70, "integer_rand_2000":115, "integer_seq_2000":370, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_a":11, "string_rand_26_b":11}
+{"grouping_1000":371, "grouping_500":371, "grouping_250":121, "grouping_100":71, "integer_rand_2000":null, "integer_seq_2000":371, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":6, "string_rand_26_c":6}
+{"grouping_1000":372, "grouping_500":372, "grouping_250":122, "grouping_100":72, "integer_rand_2000":null, "integer_seq_2000":372, "integer_rand_2":1, "integer_rand_10":1, "string_rand_26_a":23, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":373, "grouping_500":373, "grouping_250":123, "grouping_100":73, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":5, "integer_rand_20":null, "string_rand_26_a":11, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":374, "grouping_500":374, "grouping_250":124, "grouping_100":74, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":18, "string_rand_26_b":null}
+{"grouping_1000":375, "grouping_500":375, "grouping_250":125, "grouping_100":75, "integer_rand_2000":241, "integer_seq_2000":375, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":7, "string_rand_26_b":null, "string_rand_26_c":7}
+{"grouping_1000":376, "grouping_500":376, "grouping_250":126, "grouping_100":76, "integer_seq_2000":376, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":6, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":377, "grouping_500":377, "grouping_250":127, "grouping_100":77, "integer_rand_2000":49, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":9, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":null}
+{"grouping_1000":378, "grouping_500":378, "grouping_250":128, "grouping_100":78, "integer_rand_2000":null, "integer_seq_2000":378, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":17}
+{"grouping_1000":379, "grouping_500":379, "grouping_250":129, "grouping_100":79, "integer_rand_2000":63, "integer_seq_2000":379, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":11, "string_rand_26_c":null}
+{"grouping_1000":380, "grouping_500":380, "grouping_250":130, "grouping_100":80, "integer_rand_2000":309, "integer_seq_2000":380, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":23}
+{"grouping_1000":381, "grouping_500":381, "grouping_250":131, "grouping_100":81, "integer_rand_2000":261, "integer_seq_2000":381, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_20":null, "string_rand_26_a":1, "string_rand_26_c":1}
+{"grouping_1000":382, "grouping_500":382, "grouping_250":132, "grouping_100":82, "integer_rand_2000":146, "integer_seq_2000":382, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":16, "string_rand_26_b":16}
+{"grouping_1000":383, "grouping_500":383, "grouping_250":133, "grouping_100":83, "integer_rand_2000":null, "integer_seq_2000":383, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":7, "string_rand_26_c":null}
+{"grouping_1000":384, "grouping_500":384, "grouping_250":134, "grouping_100":84, "integer_rand_2000":271, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":null, "string_rand_26_c":11}
+{"grouping_1000":385, "grouping_500":385, "grouping_250":135, "grouping_100":85, "integer_rand_2000":null, "integer_seq_2000":385, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":16}
+{"grouping_1000":386, "grouping_500":386, "grouping_250":136, "grouping_100":86, "integer_rand_2000":365, "integer_seq_2000":386, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "integer_rand_20":5, "string_rand_26_a":1, "string_rand_26_b":null, "string_rand_26_c":1}
+{"grouping_1000":387, "grouping_500":387, "grouping_250":137, "grouping_100":87, "integer_rand_2000":281, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":21, "string_rand_26_b":21, "string_rand_26_c":null}
+{"grouping_1000":388, "grouping_500":388, "grouping_250":138, "grouping_100":88, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":3, "integer_rand_20":3, "string_rand_26_a":7, "string_rand_26_b":7, "string_rand_26_c":null}
+{"grouping_1000":389, "grouping_500":389, "grouping_250":139, "grouping_100":89, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":24, "string_rand_26_c":24}
+{"grouping_1000":390, "grouping_500":390, "grouping_250":140, "grouping_100":90, "integer_rand_2000":50, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":24, "string_rand_26_b":24, "string_rand_26_c":24}
+{"grouping_1000":391, "grouping_500":391, "grouping_250":141, "grouping_100":91, "integer_rand_2000":76, "integer_rand_2":0, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_a":24, "string_rand_26_c":24}
+{"grouping_1000":392, "grouping_500":392, "grouping_250":142, "grouping_100":92, "integer_rand_2000":102, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":24, "string_rand_26_c":null}
+{"grouping_1000":393, "grouping_500":393, "grouping_250":143, "grouping_100":93, "integer_rand_2000":128, "integer_seq_2000":393, "integer_rand_2":0, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":24, "string_rand_26_b":24, "string_rand_26_c":24}
+{"grouping_1000":394, "grouping_500":394, "grouping_250":144, "grouping_100":94, "integer_rand_2000":154, "integer_seq_2000":394, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":395, "grouping_500":395, "grouping_250":145, "grouping_100":95, "integer_rand_2000":180, "integer_seq_2000":null, "integer_rand_10":0, "integer_rand_20":null, "string_rand_26_b":24, "string_rand_26_c":null}
+{"grouping_1000":396, "grouping_500":396, "grouping_250":146, "grouping_100":96, "integer_rand_2000":206, "integer_seq_2000":396, "integer_rand_2":0, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":24, "string_rand_26_b":24, "string_rand_26_c":null}
+{"grouping_1000":397, "grouping_500":397, "grouping_250":147, "grouping_100":97, "integer_rand_2000":null, "integer_seq_2000":397, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":2, "string_rand_26_a":24, "string_rand_26_c":24}
+{"grouping_1000":398, "grouping_500":398, "grouping_250":148, "grouping_100":98, "integer_rand_2000":null, "integer_seq_2000":398, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":24, "string_rand_26_c":null}
+{"grouping_1000":399, "grouping_500":399, "grouping_250":149, "grouping_100":99, "integer_seq_2000":399, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_20":null, "string_rand_26_a":24, "string_rand_26_b":24, "string_rand_26_c":24}
+{"grouping_1000":400, "grouping_500":400, "grouping_250":150, "grouping_100":0, "integer_rand_2000":null, "integer_seq_2000":400, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":24, "string_rand_26_c":24}
+{"grouping_1000":401, "grouping_500":401, "grouping_250":151, "grouping_100":1, "integer_rand_2000":336, "integer_seq_2000":401, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":24}
+{"grouping_1000":402, "grouping_500":402, "grouping_250":152, "grouping_100":2, "integer_rand_2000":362, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":2, "string_rand_26_a":null, "string_rand_26_b":24, "string_rand_26_c":24}
+{"grouping_1000":403, "grouping_500":403, "grouping_250":153, "grouping_100":3, "integer_seq_2000":403, "integer_rand_2":null, "integer_rand_10":null, "integer_rand_20":8, "string_rand_26_a":24}
+{"grouping_1000":404, "grouping_500":404, "grouping_250":154, "grouping_100":4, "integer_rand_2000":null, "integer_seq_2000":404, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":24, "string_rand_26_b":24}
+{"grouping_1000":405, "grouping_500":405, "grouping_250":155, "grouping_100":5, "integer_rand_2000":440, "integer_seq_2000":405, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":0, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":24}
+{"grouping_1000":406, "grouping_500":406, "grouping_250":156, "grouping_100":6, "integer_rand_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":24, "string_rand_26_c":24}
+{"grouping_1000":407, "grouping_500":407, "grouping_250":157, "grouping_100":7, "integer_rand_2000":null, "integer_rand_2":0, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":24, "string_rand_26_b":24, "string_rand_26_c":24}
+{"grouping_1000":408, "grouping_500":408, "grouping_250":158, "grouping_100":8, "integer_rand_2000":351, "integer_seq_2000":408, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":409, "grouping_500":409, "grouping_250":159, "grouping_100":9, "integer_seq_2000":409, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "string_rand_26_a":null, "string_rand_26_b":18, "string_rand_26_c":null}
+{"grouping_1000":410, "grouping_500":410, "grouping_250":160, "grouping_100":10, "integer_rand_2000":105, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":5, "integer_rand_20":5, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":null}
+{"grouping_1000":411, "grouping_500":411, "grouping_250":161, "grouping_100":11, "integer_rand_2000":361, "integer_seq_2000":411, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":1, "integer_rand_20":1, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":23}
+{"grouping_1000":412, "grouping_500":412, "grouping_250":162, "grouping_100":12, "integer_seq_2000":412, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_b":null, "string_rand_26_c":2}
+{"grouping_1000":413, "grouping_500":413, "grouping_250":163, "grouping_100":13, "integer_rand_2000":142, "integer_seq_2000":413, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":12, "string_rand_26_b":null, "string_rand_26_c":12}
+{"grouping_1000":414, "grouping_500":414, "grouping_250":164, "grouping_100":14, "integer_rand_2000":371, "integer_seq_2000":414, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_20":null, "string_rand_26_a":7, "string_rand_26_b":7, "string_rand_26_c":null}
+{"grouping_1000":415, "grouping_500":415, "grouping_250":165, "grouping_100":15, "integer_rand_2000":38, "integer_seq_2000":415, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":12, "string_rand_26_b":12, "string_rand_26_c":12}
+{"grouping_1000":416, "grouping_500":416, "grouping_250":166, "grouping_100":16, "integer_rand_2000":402, "integer_seq_2000":416, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":12, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":417, "grouping_500":417, "grouping_250":167, "grouping_100":17, "integer_rand_2000":381, "integer_seq_2000":417, "integer_rand_2":1, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":17, "string_rand_26_c":null}
+{"grouping_1000":418, "grouping_500":418, "grouping_250":168, "grouping_100":18, "integer_rand_2000":48, "integer_seq_2000":418, "integer_rand_2":0, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":22, "string_rand_26_b":null, "string_rand_26_c":22}
+{"grouping_1000":419, "grouping_500":419, "grouping_250":169, "grouping_100":19, "integer_rand_2000":null, "integer_seq_2000":419, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":9, "integer_rand_20":19, "string_rand_26_a":7, "string_rand_26_c":null}
+{"grouping_1000":420, "grouping_500":420, "grouping_250":170, "grouping_100":20, "integer_rand_2000":391, "integer_seq_2000":420, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":1}
+{"grouping_1000":421, "grouping_500":421, "grouping_250":171, "grouping_100":21, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":6, "string_rand_26_b":null, "string_rand_26_c":6}
+{"grouping_1000":422, "grouping_500":422, "grouping_250":172, "grouping_100":22, "integer_rand_2000":null, "integer_seq_2000":422, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":8, "string_rand_26_a":12, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":423, "grouping_500":423, "grouping_250":173, "grouping_100":23, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":1, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":11}
+{"grouping_1000":424, "grouping_500":424, "grouping_250":174, "grouping_100":24, "integer_rand_2000":null, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":8, "string_rand_26_a":16}
+{"grouping_1000":425, "grouping_500":425, "grouping_250":175, "grouping_100":25, "integer_rand_2000":173, "integer_seq_2000":425, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":13, "string_rand_26_a":17, "string_rand_26_b":17, "string_rand_26_c":17}
+{"grouping_1000":426, "grouping_500":426, "grouping_250":176, "grouping_100":26, "integer_rand_2000":411, "integer_seq_2000":426, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":11, "string_rand_26_b":21, "string_rand_26_c":null}
+{"grouping_1000":427, "grouping_500":427, "grouping_250":177, "grouping_100":27, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":3, "string_rand_26_a":3, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":428, "grouping_500":428, "grouping_250":178, "grouping_100":28, "integer_rand_2000":29, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":9, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":3}
+{"grouping_1000":429, "grouping_500":429, "grouping_250":179, "grouping_100":29, "integer_rand_2000":55, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_a":3, "string_rand_26_b":3, "string_rand_26_c":3}
+{"grouping_1000":430, "grouping_500":430, "grouping_250":180, "grouping_100":30, "integer_rand_2000":81, "integer_seq_2000":430, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":1, "string_rand_26_a":3, "string_rand_26_c":3}
+{"grouping_1000":431, "grouping_500":431, "grouping_250":181, "grouping_100":31, "integer_rand_2000":null, "integer_seq_2000":431, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":3, "string_rand_26_b":3, "string_rand_26_c":3}
+{"grouping_1000":432, "grouping_500":432, "grouping_250":182, "grouping_100":32, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":null, "string_rand_26_b":3}
+{"grouping_1000":433, "grouping_500":433, "grouping_250":183, "grouping_100":33, "integer_rand_2000":159, "integer_seq_2000":null, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_a":3, "string_rand_26_b":null}
+{"grouping_1000":434, "grouping_500":434, "grouping_250":184, "grouping_100":34, "integer_rand_2000":185, "integer_seq_2000":434, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":null, "integer_rand_20":5, "string_rand_26_a":3, "string_rand_26_b":null, "string_rand_26_c":3}
+{"grouping_1000":435, "grouping_500":435, "grouping_250":185, "grouping_100":35, "integer_rand_2000":211, "integer_seq_2000":435, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":3}
+{"grouping_1000":436, "grouping_500":436, "grouping_250":186, "grouping_100":36, "integer_rand_2000":237, "integer_seq_2000":436, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_20":17, "string_rand_26_a":3, "string_rand_26_b":3, "string_rand_26_c":3}
+{"grouping_1000":437, "grouping_500":437, "grouping_250":187, "grouping_100":37, "integer_rand_2000":263, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":3, "string_rand_26_b":3, "string_rand_26_c":null}
+{"grouping_1000":438, "grouping_500":438, "grouping_250":188, "grouping_100":38, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":null, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":439, "grouping_500":439, "grouping_250":189, "grouping_100":39, "integer_rand_2000":null, "integer_seq_2000":439, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_a":null, "string_rand_26_b":3, "string_rand_26_c":3}
+{"grouping_1000":440, "grouping_500":440, "grouping_250":190, "grouping_100":40, "integer_rand_2000":null, "integer_seq_2000":440, "integer_rand_2":1, "integer_rand_10":null, "string_rand_26_a":3, "string_rand_26_b":null, "string_rand_26_c":3}
+{"grouping_1000":441, "grouping_500":441, "grouping_250":191, "grouping_100":41, "integer_rand_2000":367, "integer_rand_2":null, "integer_rand_10":7, "integer_rand_20":7, "string_rand_26_a":null, "string_rand_26_b":3, "string_rand_26_c":3}
+{"grouping_1000":442, "grouping_500":442, "grouping_250":192, "grouping_100":42, "integer_rand_2000":null, "integer_seq_2000":442, "integer_rand_2":null, "integer_rand_10":3, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":3, "string_rand_26_c":null}
+{"grouping_1000":443, "grouping_500":443, "grouping_250":193, "grouping_100":43, "integer_rand_2000":419, "integer_seq_2000":443, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":9, "integer_rand_20":null, "string_rand_26_a":3, "string_rand_26_b":null}
+{"grouping_1000":444, "grouping_500":444, "grouping_250":194, "grouping_100":44, "integer_seq_2000":444, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_20":5, "string_rand_26_a":3, "string_rand_26_b":null}
+{"grouping_1000":445, "grouping_500":445, "grouping_250":195, "grouping_100":45, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":3, "string_rand_26_b":3, "string_rand_26_c":null}
+{"grouping_1000":446, "grouping_500":446, "grouping_250":196, "grouping_100":46, "integer_rand_2000":null, "integer_seq_2000":446, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":7, "string_rand_26_a":null, "string_rand_26_c":3}
+{"grouping_1000":447, "grouping_500":447, "grouping_250":197, "grouping_100":47, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_20":1, "string_rand_26_a":null, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":448, "grouping_500":448, "grouping_250":198, "grouping_100":48, "integer_rand_2000":null, "integer_seq_2000":448, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":449, "grouping_500":449, "grouping_250":199, "grouping_100":49, "integer_rand_2000":null, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":5, "integer_rand_20":null, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":null}
+{"grouping_1000":450, "grouping_500":450, "grouping_250":200, "grouping_100":50, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_10":1, "integer_rand_20":null, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":23}
+{"grouping_1000":451, "grouping_500":451, "grouping_250":201, "grouping_100":51, "integer_rand_2000":null, "integer_rand_2":0, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_b":null}
+{"grouping_1000":452, "grouping_500":452, "grouping_250":202, "grouping_100":52, "integer_rand_2000":495, "integer_seq_2000":452, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_a":1, "string_rand_26_b":1, "string_rand_26_c":1}
+{"grouping_1000":453, "grouping_500":453, "grouping_250":203, "grouping_100":53, "integer_rand_2000":33, "integer_seq_2000":453, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":null, "string_rand_26_b":7, "string_rand_26_c":null}
+{"grouping_1000":454, "grouping_500":454, "grouping_250":204, "grouping_100":54, "integer_rand_2000":168, "integer_seq_2000":454, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":12, "string_rand_26_b":null, "string_rand_26_c":12}
+{"grouping_1000":455, "grouping_500":455, "grouping_250":205, "grouping_100":55, "integer_rand_2000":459, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":9, "integer_rand_20":null, "string_rand_26_a":17, "string_rand_26_b":null, "string_rand_26_c":17}
+{"grouping_1000":456, "grouping_500":456, "grouping_250":206, "grouping_100":56, "integer_rand_2000":169, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":457, "grouping_500":457, "grouping_250":207, "grouping_100":57, "integer_rand_2000":178, "integer_seq_2000":457, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":22, "string_rand_26_b":null, "string_rand_26_c":22}
+{"grouping_1000":458, "grouping_500":458, "grouping_250":208, "grouping_100":58, "integer_rand_2000":6, "integer_seq_2000":458, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":6, "string_rand_26_a":null, "string_rand_26_b":6, "string_rand_26_c":null}
+{"grouping_1000":459, "grouping_500":459, "grouping_250":209, "grouping_100":59, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":null, "integer_rand_4":1, "integer_rand_10":9, "integer_rand_20":9, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":460, "grouping_500":460, "grouping_250":210, "grouping_100":60, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_4":0, "integer_rand_20":8, "string_rand_26_b":null, "string_rand_26_c":6}
+{"grouping_1000":461, "grouping_500":461, "grouping_250":211, "grouping_100":61, "integer_rand_2000":266, "integer_seq_2000":461, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":6, "string_rand_26_b":6, "string_rand_26_c":null}
+{"grouping_1000":462, "grouping_500":462, "grouping_250":212, "grouping_100":62, "integer_rand_2000":32, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_20":12, "string_rand_26_a":6, "string_rand_26_b":6, "string_rand_26_c":6}
+{"grouping_1000":463, "grouping_500":463, "grouping_250":213, "grouping_100":63, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":16}
+{"grouping_1000":464, "grouping_500":464, "grouping_250":214, "grouping_100":64, "integer_rand_2000":408, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":null, "string_rand_26_a":18, "string_rand_26_c":18}
+{"grouping_1000":465, "grouping_500":465, "grouping_250":215, "grouping_100":65, "integer_rand_2000":458, "integer_seq_2000":465, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":16, "string_rand_26_b":16, "string_rand_26_c":null}
+{"grouping_1000":466, "grouping_500":466, "grouping_250":216, "grouping_100":66, "integer_rand_2000":8, "integer_seq_2000":466, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":8}
+{"grouping_1000":467, "grouping_500":467, "grouping_250":217, "grouping_100":67, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":4, "integer_rand_20":null, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":468, "grouping_500":468, "grouping_250":218, "grouping_100":68, "integer_seq_2000":468, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_a":null, "string_rand_26_b":8, "string_rand_26_c":8}
+{"grouping_1000":469, "grouping_500":469, "grouping_250":219, "grouping_100":69, "integer_rand_2000":86, "integer_seq_2000":469, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":6, "integer_rand_20":6, "string_rand_26_a":null, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":470, "grouping_500":470, "grouping_250":220, "grouping_100":70, "integer_rand_2000":112, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_a":8, "string_rand_26_b":8}
+{"grouping_1000":471, "grouping_500":471, "grouping_250":221, "grouping_100":71, "integer_rand_2000":138, "integer_seq_2000":471, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":18, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":472, "grouping_500":472, "grouping_250":222, "grouping_100":72, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_20":null, "string_rand_26_a":8, "string_rand_26_b":null, "string_rand_26_c":8}
+{"grouping_1000":473, "grouping_500":473, "grouping_250":223, "grouping_100":73, "integer_seq_2000":473, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":10, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":474, "grouping_500":474, "grouping_250":224, "grouping_100":74, "integer_rand_2000":216, "integer_seq_2000":474, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":16, "string_rand_26_a":null, "string_rand_26_c":null}
+{"grouping_1000":475, "grouping_500":475, "grouping_250":225, "grouping_100":75, "integer_rand_2000":242, "integer_seq_2000":475, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_10":2, "integer_rand_20":null, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":476, "grouping_500":476, "grouping_250":226, "grouping_100":76, "integer_rand_2000":268, "integer_seq_2000":476, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":null, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":477, "grouping_500":477, "grouping_250":227, "grouping_100":77, "integer_rand_4":2, "integer_rand_10":4, "integer_rand_20":14, "string_rand_26_a":8, "string_rand_26_b":null, "string_rand_26_c":8}
+{"grouping_1000":478, "grouping_500":478, "grouping_250":228, "grouping_100":78, "integer_rand_2000":null, "integer_seq_2000":478, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":0, "string_rand_26_a":null, "string_rand_26_b":8, "string_rand_26_c":8}
+{"grouping_1000":479, "grouping_500":479, "grouping_250":229, "grouping_100":79, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_10":6, "integer_rand_20":null, "string_rand_26_b":8, "string_rand_26_c":8}
+{"grouping_1000":480, "grouping_500":480, "grouping_250":230, "grouping_100":80, "integer_rand_2000":null, "integer_seq_2000":480, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":2, "integer_rand_20":12, "string_rand_26_b":8, "string_rand_26_c":8}
+{"grouping_1000":481, "grouping_500":481, "grouping_250":231, "grouping_100":81, "integer_rand_2000":398, "integer_seq_2000":481, "integer_rand_2":null, "integer_rand_4":null, "integer_rand_20":18, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":8}
+{"grouping_1000":482, "grouping_500":482, "grouping_250":232, "grouping_100":82, "integer_rand_2000":424, "integer_seq_2000":482, "integer_rand_2":null, "integer_rand_4":0, "integer_rand_10":4, "integer_rand_20":4, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":483, "grouping_500":483, "grouping_250":233, "grouping_100":83, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":0, "integer_rand_20":null, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":null}
+{"grouping_1000":484, "grouping_500":484, "grouping_250":234, "grouping_100":84, "integer_rand_2000":476, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":16, "string_rand_26_a":8, "string_rand_26_b":8, "string_rand_26_c":8}
+{"grouping_1000":485, "grouping_500":485, "grouping_250":235, "grouping_100":85, "integer_rand_2000":328, "integer_rand_4":0, "integer_rand_10":8, "integer_rand_20":8, "string_rand_26_a":16, "string_rand_26_b":16, "string_rand_26_c":16}
+{"grouping_1000":486, "grouping_500":486, "grouping_250":236, "grouping_100":86, "integer_rand_2000":355, "integer_seq_2000":486, "integer_rand_2":1, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":15, "string_rand_26_a":17, "string_rand_26_b":null, "string_rand_26_c":17}
+{"grouping_1000":487, "grouping_500":487, "grouping_250":237, "grouping_100":87, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":18, "string_rand_26_b":null, "string_rand_26_c":18}
+{"grouping_1000":488, "grouping_500":488, "grouping_250":238, "grouping_100":88, "integer_rand_2000":null, "integer_seq_2000":null, "integer_rand_2":1, "integer_rand_4":1, "integer_rand_10":3, "integer_rand_20":13, "string_rand_26_a":23, "string_rand_26_b":23, "string_rand_26_c":null}
+{"grouping_1000":489, "grouping_500":489, "grouping_250":239, "grouping_100":89, "integer_rand_2000":2, "integer_seq_2000":489, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":null, "integer_rand_20":2, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":490, "grouping_500":490, "grouping_250":240, "grouping_100":90, "integer_rand_2000":null, "integer_seq_2000":490, "integer_rand_2":0, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":2, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":491, "grouping_500":491, "grouping_250":241, "grouping_100":91, "integer_rand_2000":136, "integer_rand_2":0, "integer_rand_10":6, "integer_rand_20":16, "string_rand_26_a":6, "string_rand_26_b":null, "string_rand_26_c":null}
+{"grouping_1000":492, "grouping_500":492, "grouping_250":242, "grouping_100":92, "integer_rand_2000":262, "integer_seq_2000":492, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":2, "integer_rand_20":2, "string_rand_26_a":null, "string_rand_26_b":2, "string_rand_26_c":2}
+{"grouping_1000":493, "grouping_500":493, "grouping_250":243, "grouping_100":93, "integer_rand_2000":298, "integer_seq_2000":493, "integer_rand_2":0, "integer_rand_4":2, "integer_rand_10":8, "integer_rand_20":18, "string_rand_26_a":null, "string_rand_26_b":12, "string_rand_26_c":12}
+{"grouping_1000":494, "grouping_500":494, "grouping_250":244, "grouping_100":94, "integer_rand_2000":99, "integer_seq_2000":494, "integer_rand_4":null, "string_rand_26_a":21, "string_rand_26_b":21, "string_rand_26_c":null}
+{"grouping_1000":495, "grouping_500":495, "grouping_250":245, "grouping_100":95, "integer_rand_2000":455, "integer_seq_2000":495, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":5, "integer_rand_20":15, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":13}
+{"grouping_1000":496, "grouping_500":496, "grouping_250":246, "grouping_100":96, "integer_rand_2000":null, "integer_seq_2000":496, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_b":null, "string_rand_26_c":22}
+{"grouping_1000":497, "grouping_500":497, "grouping_250":247, "grouping_100":97, "integer_seq_2000":497, "integer_rand_2":1, "integer_rand_4":null, "integer_rand_10":null, "integer_rand_20":19, "string_rand_26_a":null, "string_rand_26_b":21, "string_rand_26_c":null}
+{"grouping_1000":498, "grouping_500":498, "grouping_250":248, "grouping_100":98, "integer_rand_2000":152, "integer_seq_2000":498, "integer_rand_2":0, "integer_rand_4":0, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":null, "string_rand_26_b":22, "string_rand_26_c":22}
+{"grouping_1000":499, "grouping_500":499, "grouping_250":249, "grouping_100":99, "integer_rand_2000":null, "integer_seq_2000":499, "integer_rand_2":null, "integer_rand_4":2, "integer_rand_10":8, "string_rand_26_a":6, "string_rand_26_b":null}
+{"grouping_1000":500, "grouping_500":0, "grouping_250":0, "grouping_100":0, "integer_rand_2000":39, "integer_seq_2000":500, "integer_rand_2":null, "integer_rand_4":3, "integer_rand_10":null, "integer_rand_20":null, "string_rand_26_a":13, "string_rand_26_b":13, "string_rand_26_c":13}
diff --git a/asterixdb/asterix-app/data/hdfs/parquet/heterogeneous_1.json b/asterixdb/asterix-app/data/hdfs/parquet/heterogeneous_1.json
new file mode 100644
index 0000000..69cefc0
--- /dev/null
+++ b/asterixdb/asterix-app/data/hdfs/parquet/heterogeneous_1.json
@@ -0,0 +1,3 @@
+{"id": 0, "arrayOrObject" : [{"text" : "1"}, {"text" : "2"}]}
+{"id": 1, "arrayOrObject" : [{"text" : "3"}, {"text" : "4"}]}
+{"id": 2, "arrayOrObject" : [{"text" : "5"}, {"text" : "6"}]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/hdfs/parquet/heterogeneous_2.json b/asterixdb/asterix-app/data/hdfs/parquet/heterogeneous_2.json
new file mode 100644
index 0000000..4826af59
--- /dev/null
+++ b/asterixdb/asterix-app/data/hdfs/parquet/heterogeneous_2.json
@@ -0,0 +1,3 @@
+{"id": 3, "arrayOrObject" : {"text" : "7"}}
+{"id": 4, "arrayOrObject" : {"text" : "8"}}
+{"id": 5, "arrayOrObject" : {"text" : "9"}}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/hdfs/parquet/id_age-string.json b/asterixdb/asterix-app/data/hdfs/parquet/id_age-string.json
new file mode 100644
index 0000000..36680ac
--- /dev/null
+++ b/asterixdb/asterix-app/data/hdfs/parquet/id_age-string.json
@@ -0,0 +1,7 @@
+{"id": 15, "age": "10"}
+{"id": 16, "age": "20"}
+{"id": 17, "age": "30"}
+{"id": 18, "age": "40"}
+{"id": 19, "age": "50"}
+{"id": 20, "age": "60"}
+{"id": 21, "age": "70"}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/misc/1.adm b/asterixdb/asterix-app/data/misc/1.adm
new file mode 100644
index 0000000..93c166a
--- /dev/null
+++ b/asterixdb/asterix-app/data/misc/1.adm
@@ -0,0 +1,9 @@
+{"id": 1, "int_f": 1 , "str_f": "1"}
+{"id": 2, "int_f": null, "str_f": "2"}
+{"id": 3 , "str_f": "3"}
+{"id": 4, "int_f": 4 , "str_f": null}
+{"id": 5, "int_f": 5 }
+{"id": 6, "int_f": null }
+{"id": 7 , "str_f": null}
+{"id": 8, "int_f": null, "str_f": null}
+{"id": 9 ,"other_f": "other" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/spatial/lakes.json b/asterixdb/asterix-app/data/spatial/lakes.json
new file mode 100644
index 0000000..0e7e4e2
--- /dev/null
+++ b/asterixdb/asterix-app/data/spatial/lakes.json
@@ -0,0 +1,100 @@
+{"id":0,"geom":rectangle("-27.9323481511,9.15518656723 -24.8445930928,11.9309972473")}
+{"id":1,"geom":rectangle("0.0621153624124,18.2734730769 4.83281963665,19.3336733634")}
+{"id":2,"geom":rectangle("-37.873318599,50.4137494117 -33.4774896319,55.3172948124")}
+{"id":3,"geom":rectangle("47.1325883602,-19.4791980415 50.9598502084,-18.481896624")}
+{"id":4,"geom":rectangle("51.3016231532,-5.60195086869 55.9768771092,-4.44676667106")}
+{"id":5,"geom":rectangle("9.27570071324,-1.68430260794 14.3618786871,0.0989231691509")}
+{"id":6,"geom":rectangle("-40.8426642141,-12.3828594111 -36.2560812836,-8.69636838084")}
+{"id":7,"geom":rectangle("42.0000180351,-6.38480480539 46.3487445182,-2.08245163105")}
+{"id":8,"geom":rectangle("-49.1105328926,1.23797277776 -48.6903344422,2.26860151234")}
+{"id":9,"geom":rectangle("8.41156431616,23.7284234749 12.3270757028,25.9675336727")}
+{"id":10,"geom":rectangle("24.2568615633,-20.3953636207 27.8001525267,-16.0145481861")}
+{"id":11,"geom":rectangle("-16.3094782394,8.91257038501 -12.3410467932,13.799767852")}
+{"id":12,"geom":rectangle("-5.47983206871,10.2243601629 -5.10262146508,13.9002669231")}
+{"id":13,"geom":rectangle("-60.9461639943,10.4323876808 -56.7511118231,14.8796053416")}
+{"id":14,"geom":rectangle("-11.972629563,15.8894382199 -8.06809484334,18.6230609701")}
+{"id":15,"geom":rectangle("-55.4148350065,-11.2390940595 -51.7126250696,-7.70494356838")}
+{"id":16,"geom":rectangle("-39.6238055795,5.95626957878 -37.3601098467,6.89396261843")}
+{"id":17,"geom":rectangle("-9.31124052127,-11.7939905005 -5.81616506883,-10.7313254896")}
+{"id":18,"geom":rectangle("55.8575494942,-34.8735358882 59.052052825,-32.1878431332")}
+{"id":19,"geom":rectangle("6.78644433431,-15.1103391491 11.022852731,-9.97257905438")}
+{"id":20,"geom":rectangle("-46.9172489187,8.34059859105 -44.8843063877,10.2219856068")}
+{"id":21,"geom":rectangle("18.8133717489,7.58225769546 20.7327951646,10.400196251")}
+{"id":22,"geom":rectangle("39.6991218048,-19.5111127309 43.4813236282,-17.2751694545")}
+{"id":23,"geom":rectangle("-53.0916999826,15.6224380883 -51.3276055144,18.4131633566")}
+{"id":24,"geom":rectangle("15.4403413661,-16.717177507 20.5659895946,-14.4905598285")}
+{"id":25,"geom":rectangle("18.0403882995,-16.942547031 18.4674639813,-12.0137417407")}
+{"id":26,"geom":rectangle("-10.1680547254,-60.7865500782 -8.00879668415,-56.7438699052")}
+{"id":27,"geom":rectangle("68.6004972482,8.11792475815 71.5635512937,11.1498717613")}
+{"id":28,"geom":rectangle("-42.941592897,-12.1682044962 -37.7633402924,-11.336014273")}
+{"id":29,"geom":rectangle("72.6533873046,2.07227812745 76.6403854525,7.09220721646")}
+{"id":30,"geom":rectangle("51.8540534407,-17.0442271016 54.8389508618,-12.6231227706")}
+{"id":31,"geom":rectangle("6.60620427018,15.9937496153 8.0217849552,16.8439308667")}
+{"id":32,"geom":rectangle("-54.4890021697,-23.8103029028 -50.031768851,-22.2413347517")}
+{"id":33,"geom":rectangle("-34.95511358,2.02954068386 -33.703431683,4.19422929144")}
+{"id":34,"geom":rectangle("-20.7743277107,-0.496422564668 -19.777600253,-0.0970785280508")}
+{"id":35,"geom":rectangle("-39.3142027252,-3.48265518567 -38.0781722188,-1.12776007008")}
+{"id":36,"geom":rectangle("6.77600057461,26.8759104261 11.7404611175,32.0174621982")}
+{"id":37,"geom":rectangle("24.4099866125,3.57651968637 26.7640691754,4.88288221349")}
+{"id":38,"geom":rectangle("15.2975621765,-0.569018270344 17.3906547082,1.74652310434")}
+{"id":39,"geom":rectangle("56.9872441628,-2.94723993425 61.3324446299,0.926328661561")}
+{"id":40,"geom":rectangle("41.047461317,10.1632058932 44.7623919157,13.6693470556")}
+{"id":41,"geom":rectangle("10.1079158628,-23.8702911151 12.7244666464,-22.8542384108")}
+{"id":42,"geom":rectangle("-22.4974254873,11.8509722536 -21.8574961025,15.2160233895")}
+{"id":43,"geom":rectangle("-43.9972059177,25.2595980469 -43.2811239229,28.5826167355")}
+{"id":44,"geom":rectangle("-7.23037396361,-7.11084459626 -2.27629349239,-3.61114277177")}
+{"id":45,"geom":rectangle("41.7404274398,16.4641333114 42.3134948646,17.21273776")}
+{"id":46,"geom":rectangle("-78.8108223124,33.0735094389 -73.6471184423,35.1192424639")}
+{"id":47,"geom":rectangle("69.3770819027,-8.4844290646 72.8377016978,-6.7522689002")}
+{"id":48,"geom":rectangle("-12.3843945904,4.42691376249 -9.57989036317,8.73880139764")}
+{"id":49,"geom":rectangle("4.26177443767,23.7431103696 8.16629007069,27.5738476242")}
+{"id":50,"geom":rectangle("-32.8317754069,-34.9288132423 -32.5521111063,-30.0667506035")}
+{"id":51,"geom":rectangle("38.3564015651,3.76035299468 42.9897451059,6.28666914167")}
+{"id":52,"geom":rectangle("9.66830715974,-13.2164544981 13.4317818497,-9.61970001253")}
+{"id":53,"geom":rectangle("-41.8619178712,0.833642096163 -37.7009933113,4.95720796208")}
+{"id":54,"geom":rectangle("36.9003622721,-16.1326998758 38.2630992969,-14.1602245262")}
+{"id":55,"geom":rectangle("-39.2043477954,-8.37064618527 -37.5700101189,-3.7861350448")}
+{"id":56,"geom":rectangle("-73.1306562341,-28.4389790443 -72.7740930226,-25.4986564116")}
+{"id":57,"geom":rectangle("-33.9976660158,5.22879142129 -32.8492047898,10.3133401489")}
+{"id":58,"geom":rectangle("-77.1746910469,0.922084338132 -75.6504210426,1.2034103681")}
+{"id":59,"geom":rectangle("22.2027355036,24.8837157674 23.9232110479,28.4103275964")}
+{"id":60,"geom":rectangle("9.88465219327,19.6537576964 10.2765884889,20.3193773926")}
+{"id":61,"geom":rectangle("-26.8112564518,2.7690970691 -22.5926465585,6.93867080924")}
+{"id":62,"geom":rectangle("-43.7073938674,-6.60657607222 -41.4240040146,-2.71455237271")}
+{"id":63,"geom":rectangle("24.4048294711,-4.62891935142 27.906994004,-4.29135734323")}
+{"id":64,"geom":rectangle("51.6531727755,21.8910939472 52.3394468954,23.2826790089")}
+{"id":65,"geom":rectangle("25.8914076927,6.92224912861 27.3432267461,9.47019756737")}
+{"id":66,"geom":rectangle("13.3696451516,5.615604925 16.9054074708,8.52144979175")}
+{"id":67,"geom":rectangle("-13.6800980239,-18.9444336886 -9.95347951639,-16.7345361171")}
+{"id":68,"geom":rectangle("-17.7510748238,-11.1136308762 -15.412795239,-7.34469680748")}
+{"id":69,"geom":rectangle("33.2947868986,8.2834932121 34.7613269962,9.03569363267")}
+{"id":70,"geom":rectangle("-33.8874431333,-23.8338234095 -30.7254105611,-20.359752895")}
+{"id":71,"geom":rectangle("22.0907475181,2.29036800869 23.7011151494,3.2765296671")}
+{"id":72,"geom":rectangle("14.5755950289,52.8417124837 19.5671060363,55.1073390372")}
+{"id":73,"geom":rectangle("-18.8077581597,16.9934859515 -15.9861161902,17.8343474049")}
+{"id":74,"geom":rectangle("10.9084864464,29.5391931137 15.3037198201,33.1052109059")}
+{"id":75,"geom":rectangle("19.4718498204,7.79868609475 22.1664168013,12.1717440014")}
+{"id":76,"geom":rectangle("-3.62863167929,11.4665296807 -3.30540699392,14.1121656494")}
+{"id":77,"geom":rectangle("-41.64372999,-23.0555507792 -36.4832305461,-21.3068023443")}
+{"id":78,"geom":rectangle("26.5213107831,46.2022585409 30.6432878791,47.0129737813")}
+{"id":79,"geom":rectangle("10.789264065,21.5625965712 14.840984666,22.3260628176")}
+{"id":80,"geom":rectangle("10.9202625272,-19.0658990104 14.1023247437,-15.3016631473")}
+{"id":81,"geom":rectangle("-15.7551653757,-2.25551304155 -12.4037949556,2.0012711873")}
+{"id":82,"geom":rectangle("37.5942150963,10.997741867 41.372075762,16.1596730965")}
+{"id":83,"geom":rectangle("-13.7359186757,29.958869656 -11.4409182302,33.4511779239")}
+{"id":84,"geom":rectangle("-13.0283243952,-11.3762227927 -12.0713603337,-8.89907873732")}
+{"id":85,"geom":rectangle("77.5973017056,-22.6402092272 78.7497490033,-21.2308882262")}
+{"id":86,"geom":rectangle("-22.4088952378,-21.7807762792 -21.497054438,-17.9602452153")}
+{"id":87,"geom":rectangle("21.2973448736,26.9027621694 22.622096107,28.9878687289")}
+{"id":88,"geom":rectangle("2.51672920699,0.0240546639022 7.26260016522,1.81047192527")}
+{"id":89,"geom":rectangle("7.51495774499,-3.85627105055 8.35285901504,-0.886614413758")}
+{"id":90,"geom":rectangle("-9.42837494505,-3.98577929823 -4.62420944265,-2.23733766023")}
+{"id":91,"geom":rectangle("-40.7216302836,3.90553033275 -37.0193557402,5.3260889493")}
+{"id":92,"geom":rectangle("20.1823194132,-26.8902590996 23.5753973148,-24.3474326885")}
+{"id":93,"geom":rectangle("37.4808096204,-26.1731289166 39.5942876414,-24.2173822745")}
+{"id":94,"geom":rectangle("-1.0730021895,-26.6770190359 -0.0802012128098,-22.3629468705")}
+{"id":95,"geom":rectangle("-16.4351749786,-17.9132294166 -11.378109551,-17.0573428992")}
+{"id":96,"geom":rectangle("-1.88355736652,8.26818869244 -0.978125324269,9.48445568")}
+{"id":97,"geom":rectangle("-33.1347880034,11.6973366548 -31.6968872823,12.0041059694")}
+{"id":98,"geom":rectangle("-25.2878269988,10.2165628058 -23.4713285433,14.3321956802")}
+{"id":99,"geom":rectangle("56.4069666051,25.5819994228 59.1720253242,30.0069600333")}
diff --git a/asterixdb/asterix-app/data/spatial/parks.json b/asterixdb/asterix-app/data/spatial/parks.json
new file mode 100644
index 0000000..6a3db62
--- /dev/null
+++ b/asterixdb/asterix-app/data/spatial/parks.json
@@ -0,0 +1,100 @@
+{"id":0,"geom":rectangle("33.0815213198,-9.81639132361 37.8016305166,-7.84406572273")}
+{"id":1,"geom":rectangle("-124.421948157,1.93767883788 -124.154251809,2.28744269497")}
+{"id":2,"geom":rectangle("57.9915252427,-2.36135910497 62.6034311404,-1.82486897086")}
+{"id":3,"geom":rectangle("-29.0688935271,-26.5487124393 -25.497240315,-23.6620099901")}
+{"id":4,"geom":rectangle("44.5633661766,-10.0985164886 45.7269764454,-5.25971946827")}
+{"id":5,"geom":rectangle("15.305286444,-12.2753304202 18.9827351797,-9.36713711588")}
+{"id":6,"geom":rectangle("-29.911107762,22.5440353307 -27.4139067872,23.0896507211")}
+{"id":7,"geom":rectangle("-57.6425342931,-12.8580151738 -52.8507052534,-12.4825801116")}
+{"id":8,"geom":rectangle("25.8735318882,-9.38051778287 27.4904674652,-6.75725388702")}
+{"id":9,"geom":rectangle("9.84546779304,34.7908792934 11.3423158265,37.6163294751")}
+{"id":10,"geom":rectangle("-51.9726341759,10.6323648191 -50.5305531532,15.7945290881")}
+{"id":11,"geom":rectangle("-23.4907513017,20.662944916 -19.4319569217,23.4464479085")}
+{"id":12,"geom":rectangle("43.4201149814,12.3343076751 47.4451061727,12.9883840827")}
+{"id":13,"geom":rectangle("29.4059288562,5.16907601248 34.5850093403,6.2687077698")}
+{"id":14,"geom":rectangle("32.5674476303,-16.9214726806 36.3961900958,-16.2800680951")}
+{"id":15,"geom":rectangle("-0.1834488207,6.13920491381 4.72384569903,9.56598789645")}
+{"id":16,"geom":rectangle("38.7756881143,5.39841667745 42.7750262309,10.0289384536")}
+{"id":17,"geom":rectangle("-14.44056663,18.4307856816 -9.36677067147,19.7583766879")}
+{"id":18,"geom":rectangle("11.570745802,-6.80699899747 13.9937592089,-4.86081653246")}
+{"id":19,"geom":rectangle("17.5420798083,18.958729748 21.7364051367,23.8559784582")}
+{"id":20,"geom":rectangle("50.3626687343,-5.75624879038 50.7160445447,-1.90079629364")}
+{"id":21,"geom":rectangle("59.9180321974,-0.209278657599 60.3779738543,1.67329307706")}
+{"id":22,"geom":rectangle("-13.4143437329,-5.4128583532 -8.51004730147,-2.97939716072")}
+{"id":23,"geom":rectangle("-67.7749282406,34.4099425167 -62.7649400663,36.0128931629")}
+{"id":24,"geom":rectangle("-19.6954394465,-9.38097306619 -14.5717338989,-5.62158895288")}
+{"id":25,"geom":rectangle("-4.62283914627,14.1191109493 -0.582166990327,16.8665197135")}
+{"id":26,"geom":rectangle("11.9450583857,-18.3252119827 15.1481317185,-13.305454101")}
+{"id":27,"geom":rectangle("45.3300959836,15.1502443345 46.1914046026,16.5505569007")}
+{"id":28,"geom":rectangle("-38.1544150849,13.2427359229 -37.7223058052,13.9289037156")}
+{"id":29,"geom":rectangle("25.9117553379,-24.2063944594 27.2556664105,-20.0878926667")}
+{"id":30,"geom":rectangle("13.6057700863,-6.29749212816 14.2712566373,-3.79005795277")}
+{"id":31,"geom":rectangle("-26.9662410313,19.1425547919 -21.9190087014,22.6025571835")}
+{"id":32,"geom":rectangle("39.9580362433,6.31802974783 40.6550631538,7.78428466256")}
+{"id":33,"geom":rectangle("-22.1168360436,-20.6231838474 -21.0373812122,-17.4818461759")}
+{"id":34,"geom":rectangle("-13.7706153852,30.000930354 -10.2285702156,30.4633714206")}
+{"id":35,"geom":rectangle("10.7513899208,-46.332627949 15.1729042238,-41.7087522917")}
+{"id":36,"geom":rectangle("54.398317257,11.2893054229 55.5126158122,11.7291194213")}
+{"id":37,"geom":rectangle("63.5968193492,-15.1966382455 65.352375016,-11.0480131127")}
+{"id":38,"geom":rectangle("47.1514141104,-4.68389071981 50.0106551109,0.398482234144")}
+{"id":39,"geom":rectangle("-17.4590864829,-0.237667446208 -15.5534126253,0.984234392487")}
+{"id":40,"geom":rectangle("13.1160139515,-19.784760558 14.4541800302,-17.909695768")}
+{"id":41,"geom":rectangle("53.3179838434,15.3179563991 54.2805821576,17.6281387012")}
+{"id":42,"geom":rectangle("31.369457427,13.165056978 35.1955254051,15.0403301856")}
+{"id":43,"geom":rectangle("17.1741906327,22.8030636353 17.4889623468,26.5730209982")}
+{"id":44,"geom":rectangle("-10.3878457081,-0.815065604011 -9.62601719521,2.09908908899")}
+{"id":45,"geom":rectangle("-16.3825969516,-1.12018500993 -15.9860095134,2.45473855859")}
+{"id":46,"geom":rectangle("-33.1253688726,7.46964360071 -30.5439240628,9.95558524072")}
+{"id":47,"geom":rectangle("-29.368011305,11.6519597155 -28.3048883686,12.9818049726")}
+{"id":48,"geom":rectangle("35.0611437776,7.03034670814 39.8962767006,8.40495515366")}
+{"id":49,"geom":rectangle("-41.2271620731,1.73252488725 -38.6763728352,2.5739855887")}
+{"id":50,"geom":rectangle("-58.3254878558,-1.93159998918 -54.4272611956,2.80164821601")}
+{"id":51,"geom":rectangle("13.8023199965,37.9991713987 16.5154301058,41.2520828093")}
+{"id":52,"geom":rectangle("-0.00851486511255,-14.2470351009 3.64419833081,-9.47603639972")}
+{"id":53,"geom":rectangle("26.7054431235,-5.16438137653 31.3237631214,-2.54420971647")}
+{"id":54,"geom":rectangle("27.8866028806,7.75111403626 31.7478444739,12.478395325")}
+{"id":55,"geom":rectangle("8.21394812371,-2.29056972429 11.4573593773,-0.764304448463")}
+{"id":56,"geom":rectangle("8.46285788596,1.64018059958 12.2673723294,4.75102475847")}
+{"id":57,"geom":rectangle("1.55454244344,-12.6324441794 1.9419040003,-8.32756051385")}
+{"id":58,"geom":rectangle("81.9243274831,15.3399706853 82.9893838755,20.2484476347")}
+{"id":59,"geom":rectangle("-3.80521441571,5.59544653202 1.00217973611,6.45642148213")}
+{"id":60,"geom":rectangle("7.3027607876,41.7183918554 9.61818587568,42.6701949556")}
+{"id":61,"geom":rectangle("1.50687245994,-0.463553775161 1.71988336805,0.5148202732")}
+{"id":62,"geom":rectangle("8.1114994271,-16.9975459513 10.181118141,-12.4214451465")}
+{"id":63,"geom":rectangle("8.47517844885,3.94583778361 9.75608493291,6.45740496653")}
+{"id":64,"geom":rectangle("32.8142697785,1.7347153599 33.5075936393,4.03213277156")}
+{"id":65,"geom":rectangle("-13.9851977532,10.2201556852 -8.94180139582,10.8359476684")}
+{"id":66,"geom":rectangle("-5.81864220867,5.72831389433 -3.96515661333,9.40311082423")}
+{"id":67,"geom":rectangle("-5.94114472317,22.0425945223 -4.75208419142,25.1480304519")}
+{"id":68,"geom":rectangle("3.32138012085,-2.54014463571 4.09010361369,2.22061165506")}
+{"id":69,"geom":rectangle("24.8845851876,-3.56812813721 26.1424974309,-1.93911469198")}
+{"id":70,"geom":rectangle("8.86845184432,6.96043446979 10.6719210071,10.5089327467")}
+{"id":71,"geom":rectangle("-10.2928503721,13.7414792395 -6.33804173624,18.799298066")}
+{"id":72,"geom":rectangle("-39.2694219453,30.5862681062 -36.709885472,32.6290866592")}
+{"id":73,"geom":rectangle("-0.680131563271,27.8806141259 4.25320010443,28.960840374")}
+{"id":74,"geom":rectangle("-4.18305807732,20.931302603 -1.58595809374,24.800187646")}
+{"id":75,"geom":rectangle("21.8984327715,-1.33650659675 22.9297670191,0.553946005393")}
+{"id":76,"geom":rectangle("-24.1977676209,4.21509449843 -19.3574031974,5.40102847909")}
+{"id":77,"geom":rectangle("-15.4154311616,24.7153248018 -13.8319530786,29.4011183457")}
+{"id":78,"geom":rectangle("1.94643236293,50.3738409701 5.89810279711,54.5201490674")}
+{"id":79,"geom":rectangle("-7.96849116411,20.5720060343 -3.12254526065,21.5654341637")}
+{"id":80,"geom":rectangle("-21.7043763659,10.1241268849 -20.8416042867,13.1904255813")}
+{"id":81,"geom":rectangle("-23.3888414298,-7.16854549483 -19.3844982673,-4.41149238515")}
+{"id":82,"geom":rectangle("-33.4895210367,17.3253537643 -28.824401802,21.3670812131")}
+{"id":83,"geom":rectangle("-42.3546712937,9.76999085801 -37.8559413646,12.6720101498")}
+{"id":84,"geom":rectangle("38.8013417317,1.10992679636 40.3179710832,6.1153997663")}
+{"id":85,"geom":rectangle("4.96535858672,23.1457530824 10.1207790147,27.3333356925")}
+{"id":86,"geom":rectangle("-90.4574845046,-19.5212053021 -85.9504562794,-15.0951106414")}
+{"id":87,"geom":rectangle("-57.8849765268,6.34375036955 -54.4546233959,7.12447857349")}
+{"id":88,"geom":rectangle("1.00138643712,7.96484069816 2.42708795033,12.8608965948")}
+{"id":89,"geom":rectangle("14.3045039032,7.89032274379 16.1025253052,9.76043987428")}
+{"id":90,"geom":rectangle("-39.3422390328,29.6303954006 -35.0044772013,33.7255680238")}
+{"id":91,"geom":rectangle("-34.5615691092,15.0808712376 -32.7510701778,16.2024568771")}
+{"id":92,"geom":rectangle("-89.4105780877,-18.6334926022 -85.1904254639,-17.9571587502")}
+{"id":93,"geom":rectangle("-1.31609859863,-4.2409753097 3.68297237575,-1.77236714054")}
+{"id":94,"geom":rectangle("62.0933242625,16.4294408774 63.5309980156,17.6884479663")}
+{"id":95,"geom":rectangle("-12.3749081173,-14.5326618303 -12.1631207021,-9.77210703996")}
+{"id":96,"geom":rectangle("80.7260631215,12.5546640031 85.2928232587,17.4875933461")}
+{"id":97,"geom":rectangle("-44.180993763,-16.0807020012 -39.6192096547,-13.9933902095")}
+{"id":98,"geom":rectangle("26.0438041781,-5.57090292895 27.0334370304,-2.87316728612")}
+{"id":99,"geom":rectangle("81.8095539196,11.9399712319 83.7438057534,13.072433318")}
diff --git a/asterixdb/asterix-app/data/yelp-checkin/feeds.csv b/asterixdb/asterix-app/data/yelp-checkin/feeds.csv
new file mode 100644
index 0000000..b7bd79c
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/feeds.csv
@@ -0,0 +1,3 @@
+1,"{ ""business_id"": ""--1UhMGODdWsrMastO9DZw"", ""dates"": [ ""2016-04-26 19:49:16"", ""2016-08-30 18:36:57"", ""2016-10-15 02:45:18"", ""2016-11-18 01:54:50"", ""2017-04-20 18:39:06"", ""2017-05-03 17:58:02"", ""2019-03-19 22:04:48"" ] }"
+2,"{ ""business_id"": ""--EF5N7P70J_UYBTPypYlA"", ""dates"": [ ""2018-05-25 19:52:07"", ""2018-09-18 16:09:44"", ""2019-10-18 21:29:09"" ] }"
+3,"{ ""business_id"": ""--Ni3oJ4VOqfOEu7Sj2Vzg"", ""dates"": [ ""2019-06-07 17:54:58"" ] }"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 3eff19a..51ede69 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -150,11 +150,11 @@
<configuration>
<ignoredUsedUndeclaredDependencies combine.children="append">
<ignoredUsedUndeclaredDependency>commons-logging:commons-logging-api:*</ignoredUsedUndeclaredDependency>
- <ignoredUsedUndeclaredDependency>org.apache.hive:hive-exec:*</ignoredUsedUndeclaredDependency>
</ignoredUsedUndeclaredDependencies>
<usedDependencies combine.children="append">
<usedDependency>org.apache.hadoop:hadoop-common</usedDependency>
<usedDependency>org.apache.asterix:asterix-geo</usedDependency>
+ <usedDependency>org.apache.tomcat:tomcat-annotations-api</usedDependency>
</usedDependencies>
<ignoredUnusedDeclaredDependencies>
<ignoredUnusedDeclaredDependency>org.apache.asterix:asterix-external-data:zip:*</ignoredUnusedDeclaredDependency>
@@ -730,10 +730,6 @@
<scope>test</scope>
</dependency>
<dependency>
- <groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
- </dependency>
- <dependency>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-storage-am-lsm-btree-test</artifactId>
<type>test-jar</type>
@@ -759,6 +755,12 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-fuzzyjoin</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-storage-am-rtree</artifactId>
<version>${hyracks.version}</version>
@@ -806,6 +808,18 @@
</exclusions>
</dependency>
<dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-buffer</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-common</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.mindrot</groupId>
<artifactId>jbcrypt</artifactId>
</dependency>
@@ -860,6 +874,15 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-1.2-api</artifactId>
</dependency>
+ <!-- Google Cloud Storage SDK -->
+ <dependency>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-storage</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tomcat</groupId>
+ <artifactId>tomcat-annotations-api</artifactId>
+ </dependency>
<!-- Needed for inferring the schema for writing Parquet files -->
<dependency>
<groupId>org.kitesdk</groupId>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 326ddcc..3a9d54d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -33,7 +33,9 @@
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
+import org.apache.asterix.api.http.server.ResultUtil;
import org.apache.asterix.app.result.fields.ExplainOnlyResultsPrinter;
+import org.apache.asterix.app.result.fields.SignaturePrinter;
import org.apache.asterix.common.api.INodeJobTracker;
import org.apache.asterix.common.api.IResponsePrinter;
import org.apache.asterix.common.config.CompilerProperties;
@@ -64,6 +66,7 @@
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
import org.apache.asterix.lang.common.statement.StartFeedStatement;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
@@ -147,7 +150,7 @@
SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION,
SetAsterixPhysicalOperatorsRule.REWRITE_ATTEMPT_BATCH_ASSIGN,
- EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK);
+ EquivalenceClassUtils.REWRITE_INTERNAL_QUERYUID_PK, SqlppQueryRewriter.SQL_COMPAT_OPTION);
private final IRewriterFactory rewriterFactory;
private final IAstPrintVisitorFactory astPrintVisitorFactory;
@@ -186,9 +189,9 @@
}
public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
- MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output,
- boolean allowNonStoredUdfCalls, boolean inlineUdfs, Collection<VarIdentifier> externalVars,
- IWarningCollector warningCollector) throws CompilationException {
+ List<ViewDecl> declaredViews, MetadataProvider metadataProvider, IReturningStatement q,
+ SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
+ Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
if (q == null) {
return null;
}
@@ -197,9 +200,9 @@
generateExpressionTree(q);
}
IQueryRewriter rw = rewriterFactory.createQueryRewriter();
- LangRewritingContext rwCtx =
- new LangRewritingContext(metadataProvider, declaredFunctions, warningCollector, q.getVarCounter());
- rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfs, externalVars);
+ LangRewritingContext rwCtx = new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews,
+ warningCollector, q.getVarCounter());
+ rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
return new Pair<>(q, q.getVarCounter());
}
@@ -277,14 +280,34 @@
}
}
}
+
+ if (conf.getClientType() == SessionConfig.ClientType.JDBC) {
+ executionPlans.setStatementCategory(Statement.Category.toString(getStatementCategory(query, statement)));
+ if (!conf.isExecuteQuery()) {
+ String stmtParams = ResultUtil.ParseOnlyResult.printStatementParameters(externalVars.keySet(), v -> v);
+ executionPlans.setStatementParameters(stmtParams);
+ }
+ if (isExplainOnly) {
+ executionPlans.setExplainOnly(true);
+ } else if (isQuery) {
+ executionPlans.setSignature(SignaturePrinter.generateFlatSignature(resultMetadata));
+ }
+ }
+
+ boolean printSignature = isQuery && requestParameters != null && requestParameters.isPrintSignature();
+
if (isExplainOnly) {
- printPlanAsResult(metadataProvider, output, printer);
+ printPlanAsResult(metadataProvider, output, printer, printSignature);
if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
executionPlans.setOptimizedLogicalPlan(null);
}
return null;
}
+ if (printSignature) {
+ printer.addResultPrinter(SignaturePrinter.newInstance(executionPlans));
+ }
+
if (!conf.isGenerateJobSpec()) {
return null;
}
@@ -297,6 +320,7 @@
builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
builder.setMissingWriterFactory(format.getMissingWriterFactory());
+ builder.setNullWriterFactory(format.getNullWriterFactory());
builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
@@ -327,9 +351,12 @@
return spec;
}
- private void printPlanAsResult(MetadataProvider metadataProvider, SessionOutput output, IResponsePrinter printer)
- throws AlgebricksException {
+ private void printPlanAsResult(MetadataProvider metadataProvider, SessionOutput output, IResponsePrinter printer,
+ boolean printSignature) throws AlgebricksException {
try {
+ if (printSignature) {
+ printer.addResultPrinter(SignaturePrinter.INSTANCE);
+ }
printer.addResultPrinter(new ExplainOnlyResultsPrinter(metadataProvider.getApplicationContext(),
executionPlans.getOptimizedLogicalPlan(), output));
printer.printResults();
@@ -343,6 +370,8 @@
switch (outputFormat) {
case LOSSLESS_JSON:
return format.getLosslessJSONPrinterFactoryProvider();
+ case LOSSLESS_ADM_JSON:
+ return format.getLosslessADMJSONPrinterFactoryProvider();
case CSV:
return format.getCSVPrinterFactoryProvider();
case ADM:
@@ -359,6 +388,11 @@
: PlanPrettyPrinter.createStringPlanPrettyPrinter();
}
+ private byte getStatementCategory(Query query, ICompiledDmlStatement statement) {
+ return statement != null ? statement.getCategory()
+ : query != null ? Statement.Category.QUERY : Statement.Category.DDL;
+ }
+
public void executeJobArray(IHyracksClientConnection hcc, JobSpecification[] specs, PrintWriter out)
throws Exception {
for (JobSpecification spec : specs) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
index 35e8daf..94360a1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/ResultMetadata.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.api.common;
+import java.util.List;
import java.util.Set;
import org.apache.asterix.translator.SessionConfig;
@@ -36,6 +37,7 @@
private long diskIoCount;
private Set<Warning> warnings;
private long totalWarningsCount;
+ private transient List<Object> outputTypes;
public ResultMetadata(SessionConfig.OutputFormat format) {
this.format = format;
@@ -95,6 +97,15 @@
}
@Override
+ public void setOutputTypes(List<Object> typeList) {
+ this.outputTypes = typeList;
+ }
+
+ public List<Object> getOutputTypes() {
+ return outputTypes;
+ }
+
+ @Override
public String toString() {
return "ResultMetadata{" + "format=" + format + ", jobDuration=" + jobDuration + ", processedObjects="
+ processedObjects + ", diskIoCount=" + diskIoCount + '}';
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
index 5a3cde6..9c8cc5e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
@@ -18,7 +18,6 @@
*/
package org.apache.asterix.api.http.server;
-import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNECTION_ATTR;
import static org.apache.asterix.common.functions.ExternalFunctionLanguage.JAVA;
import static org.apache.asterix.common.functions.ExternalFunctionLanguage.PYTHON;
@@ -41,8 +40,8 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.exceptions.IFormattedException;
+import org.apache.hyracks.control.common.controllers.NCConfig;
import org.apache.hyracks.control.common.work.SynchronizableWork;
import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.http.api.IServletResponse;
@@ -134,19 +133,11 @@
}
URI createDownloadURI(Path file) throws Exception {
+ String host = appCtx.getServiceContext().getAppConfig().getString(NCConfig.Option.PUBLIC_ADDRESS);
String path = paths[0].substring(0, servletPathLengths[0]) + GET_UDF_DIST_ENDPOINT + '/' + file.getFileName();
- String host = getHyracksClientConnection().getHost();
return new URI(httpServerProtocol.toString(), null, host, httpServerPort, path, null, null);
}
- IHyracksClientConnection getHyracksClientConnection() throws Exception { // NOSONAR
- IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
- if (hcc == null) {
- throw new RuntimeDataException(ErrorCode.PROPERTY_NOT_SET, HYRACKS_CONNECTION_ATTR);
- }
- return hcc;
- }
-
private boolean isNotAttribute(InterfaceHttpData field) {
return field == null || !field.getHttpDataType().equals(InterfaceHttpData.HttpDataType.Attribute);
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index c5eb69d..081c69a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -151,7 +151,7 @@
long startTime = System.currentTimeMillis();
final IRequestParameters requestParameters = new RequestParameters(requestReference, query, resultSet,
new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE), new IStatementExecutor.Stats(),
- new IStatementExecutor.StatementProperties(), null, null, null, null, true);
+ new IStatementExecutor.StatementProperties(), null, null, null, null, null, true);
translator.compileAndExecute(hcc, requestParameters);
long endTime = System.currentTimeMillis();
duration = (endTime - startTime) / 1000.00;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
index 9dc971a..061ceb9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
@@ -168,4 +168,9 @@
private static String generateRandomString(int size) {
return RandomStringUtils.randomAlphanumeric(size);
}
+
+ @Override
+ public boolean ignoresQueryParameters(HttpMethod method) {
+ return delegate.ignoresQueryParameters(method);
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index d6ebdad..5bcde3d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -32,6 +32,7 @@
import org.apache.asterix.app.message.ExecuteStatementResponseMessage;
import org.apache.asterix.app.result.ResponsePrinter;
import org.apache.asterix.app.result.fields.NcResultPrinter;
+import org.apache.asterix.app.result.fields.SignaturePrinter;
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.common.api.IRequestReference;
import org.apache.asterix.common.config.GlobalConfig;
@@ -48,6 +49,7 @@
import org.apache.hyracks.api.util.ExceptionUtils;
import org.apache.hyracks.http.api.IChannelClosedHandler;
import org.apache.hyracks.http.api.IServletRequest;
+import org.apache.hyracks.http.api.IServletResponse;
import org.apache.hyracks.http.server.HttpServer;
import org.apache.hyracks.http.server.InterruptOnCloseHandler;
import org.apache.hyracks.ipc.exceptions.IPCException;
@@ -121,6 +123,9 @@
// if the was no error, we can set the result status to success
executionState.setStatus(ResultStatus.SUCCESS, HttpResponseStatus.OK);
updateStatsFromCC(stats, responseMsg);
+ if (param.isSignature() && delivery != IStatementExecutor.ResultDelivery.ASYNC && !param.isParseOnly()) {
+ responsePrinter.addResultPrinter(SignaturePrinter.newInstance(responseMsg.getExecutionPlans()));
+ }
if (hasResult(responseMsg)) {
responsePrinter.addResultPrinter(
new NcResultPrinter(appCtx, responseMsg, getResultSet(), delivery, sessionOutput, stats));
@@ -135,11 +140,13 @@
Map<String, String> optionalParameters, Map<String, byte[]> statementParameters, INCServiceContext ncCtx,
MessageFuture responseFuture, ILangExtension.Language queryLanguage, String handleUrl,
int stmtCategoryRestrictionMask, boolean forceDropDataset) {
- return new ExecuteStatementRequestMessage(ncCtx.getNodeId(), responseFuture.getFutureId(), queryLanguage,
- statementsText, sessionOutput.config(), resultProperties.getNcToCcResultProperties(),
- param.getClientContextID(), handleUrl, optionalParameters, statementParameters,
- param.isMultiStatement(), param.getProfileType(), stmtCategoryRestrictionMask, requestReference,
- forceDropDataset);
+ ExecuteStatementRequestMessage requestMessage = new ExecuteStatementRequestMessage(ncCtx.getNodeId(),
+ responseFuture.getFutureId(), queryLanguage, statementsText, sessionOutput.config(),
+ resultProperties.getNcToCcResultProperties(), param.getClientContextID(), param.getDataverse(),
+ handleUrl, optionalParameters, statementParameters, param.isMultiStatement(), param.getProfileType(),
+ stmtCategoryRestrictionMask, requestReference, forceDropDataset);
+ requestMessage.setSQLCompatMode(param.isSQLCompatMode());
+ return requestMessage;
}
private void cancelQuery(INCMessageBroker messageBroker, String nodeId, String uuid, String clientContextID,
@@ -167,13 +174,13 @@
@Override
protected void handleExecuteStatementException(Throwable t, RequestExecutionState executionState,
- QueryServiceRequestParameters param) {
+ QueryServiceRequestParameters param, IServletResponse response) {
if (t instanceof TimeoutException // TODO(mblow): I don't think t can ever been an instance of TimeoutException
|| ExceptionUtils.matchingCause(t, candidate -> candidate instanceof IPCException)) {
GlobalConfig.ASTERIX_LOGGER.log(Level.WARN, t.toString(), t);
executionState.setStatus(ResultStatus.FAILED, HttpResponseStatus.SERVICE_UNAVAILABLE);
} else {
- super.handleExecuteStatementException(t, executionState, param);
+ super.handleExecuteStatementException(t, executionState, param, response);
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index f705b89..981cdc9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -94,7 +94,8 @@
SessionOutput sessionOutput = initResponse(request, response, metadata.getFormat());
ResponsePrinter printer = new ResponsePrinter(sessionOutput);
if (metadata.getFormat() == SessionConfig.OutputFormat.CLEAN_JSON
- || metadata.getFormat() == SessionConfig.OutputFormat.LOSSLESS_JSON) {
+ || metadata.getFormat() == SessionConfig.OutputFormat.LOSSLESS_JSON
+ || metadata.getFormat() == SessionConfig.OutputFormat.LOSSLESS_ADM_JSON) {
final Stats stats = new Stats();
printer.begin();
printer.addResultPrinter(new ResultsPrinter(appCtx, resultReader, null, stats, sessionOutput));
@@ -150,7 +151,8 @@
// If it's JSON or ADM, check for the "wrapper-array" flag. Default is
// "true" for JSON and "false" for ADM. (Not applicable for CSV.)
boolean wrapperArray =
- format == SessionConfig.OutputFormat.CLEAN_JSON || format == SessionConfig.OutputFormat.LOSSLESS_JSON;
+ format == SessionConfig.OutputFormat.CLEAN_JSON || format == SessionConfig.OutputFormat.LOSSLESS_JSON
+ || format == SessionConfig.OutputFormat.LOSSLESS_ADM_JSON;
String wrapperParam = request.getParameter("wrapper-array");
if (wrapperParam != null) {
wrapperArray = Boolean.valueOf(wrapperParam);
@@ -167,6 +169,8 @@
case ADM:
HttpUtil.setContentType(response, "application/x-adm", request);
break;
+ case LOSSLESS_ADM_JSON:
+ // No need to reflect in output type; fall through
case CLEAN_JSON:
// No need to reflect "clean-ness" in output type; fall through
case LOSSLESS_JSON:
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
index 3211f62..df068c0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
@@ -33,6 +33,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.translator.IStatementExecutor.ResultDelivery;
import org.apache.asterix.translator.IStatementExecutor.Stats.ProfileType;
+import org.apache.asterix.translator.SessionConfig.ClientType;
import org.apache.asterix.translator.SessionConfig.OutputFormat;
import org.apache.asterix.translator.SessionConfig.PlanFormat;
import org.apache.commons.lang3.StringUtils;
@@ -60,6 +61,8 @@
STATEMENT("statement"),
FORMAT("format"),
CLIENT_ID("client_context_id"),
+ CLIENT_TYPE("client-type"),
+ DATAVERSE("dataverse"),
PRETTY("pretty"),
MODE("mode"),
TIMEOUT("timeout"),
@@ -70,12 +73,14 @@
LOGICAL_PLAN("logical-plan"),
OPTIMIZED_LOGICAL_PLAN("optimized-logical-plan"),
PARSE_ONLY("parse-only"),
+ COMPILE_ONLY("compile-only"),
READ_ONLY("readonly"),
JOB("job"),
PROFILE("profile"),
SIGNATURE("signature"),
MULTI_STATEMENT("multi-statement"),
- MAX_WARNINGS("max-warnings");
+ MAX_WARNINGS("max-warnings"),
+ SQL_COMPAT("sql-compat");
private final String str;
@@ -90,7 +95,8 @@
private enum Attribute {
HEADER("header"),
- LOSSLESS("lossless");
+ LOSSLESS("lossless"),
+ LOSSLESS_ADM("lossless-adm");
private final String str;
@@ -105,6 +111,8 @@
private static final Map<String, PlanFormat> planFormats = ImmutableMap.of(HttpUtil.ContentType.JSON,
PlanFormat.JSON, "clean_json", PlanFormat.JSON, "string", PlanFormat.STRING);
+ private static final Map<String, ClientType> clientTypes =
+ ImmutableMap.of("asterix", ClientType.ASTERIX, "jdbc", ClientType.JDBC);
private static final Map<String, Boolean> booleanValues =
ImmutableMap.of(Boolean.TRUE.toString(), Boolean.TRUE, Boolean.FALSE.toString(), Boolean.FALSE);
private static final Map<String, Boolean> csvHeaderValues =
@@ -115,6 +123,8 @@
private String path;
private String statement;
private String clientContextID;
+ private String dataverse;
+ private ClientType clientType = ClientType.ASTERIX;
private OutputFormat format = OutputFormat.CLEAN_JSON;
private ResultDelivery mode = ResultDelivery.IMMEDIATE;
private PlanFormat planFormat = PlanFormat.JSON;
@@ -124,6 +134,7 @@
private boolean pretty = false;
private boolean expressionTree = false;
private boolean parseOnly = false; // don't execute; simply check for syntax correctness and named parameters.
+ private boolean compileOnly = false; // don't execute; compile only.
private boolean readOnly = false; // only allow statements belonging to QUERY category, fail for other categories.
private boolean rewrittenExpressionTree = false;
private boolean logicalPlan = false;
@@ -132,6 +143,7 @@
private boolean isCSVWithHeader = false;
private boolean signature = true;
private boolean multiStatement = true;
+ private boolean sqlCompatMode = false;
private long timeout = TimeUnit.MILLISECONDS.toMillis(Long.MAX_VALUE);
private long maxResultReads = 1L;
private long maxWarnings = 0L;
@@ -197,6 +209,22 @@
this.clientContextID = clientContextID;
}
+ public ClientType getClientType() {
+ return clientType;
+ }
+
+ public void setClientType(ClientType clientType) {
+ this.clientType = Objects.requireNonNull(clientType);
+ }
+
+ public String getDataverse() {
+ return dataverse;
+ }
+
+ public void setDataverse(String dataverse) {
+ this.dataverse = dataverse;
+ }
+
public ResultDelivery getMode() {
return mode;
}
@@ -279,6 +307,14 @@
return parseOnly;
}
+ public void setCompileOnly(boolean compileOnly) {
+ this.compileOnly = compileOnly;
+ }
+
+ public boolean isCompileOnly() {
+ return compileOnly;
+ }
+
public void setReadOnly(boolean readOnly) {
this.readOnly = readOnly;
}
@@ -324,6 +360,14 @@
this.multiStatement = multiStatement;
}
+ public boolean isSQLCompatMode() {
+ return sqlCompatMode;
+ }
+
+ public void setSQLCompatMode(boolean sqlCompatMode) {
+ this.sqlCompatMode = sqlCompatMode;
+ }
+
public void setMaxWarnings(long maxWarnings) {
this.maxWarnings = maxWarnings;
}
@@ -340,6 +384,8 @@
object.put("pretty", pretty);
object.put("mode", mode.getName());
object.put("clientContextID", clientContextID);
+ object.put("clientType", clientType.toString());
+ object.put("dataverse", dataverse);
object.put("format", format.toString());
object.put("timeout", timeout);
object.put("maxResultReads", maxResultReads);
@@ -355,6 +401,7 @@
object.put("parseOnly", parseOnly);
object.put("readOnly", readOnly);
object.put("maxWarnings", maxWarnings);
+ object.put("sqlCompat", sqlCompatMode);
if (statementParams != null) {
for (Map.Entry<String, JsonNode> statementParam : statementParams.entrySet()) {
object.set('$' + statementParam.getKey(), statementParam.getValue());
@@ -413,6 +460,7 @@
throws HyracksDataException {
setStatement(valGetter.apply(req, Parameter.STATEMENT.str()));
setClientContextID(valGetter.apply(req, Parameter.CLIENT_ID.str()));
+ setDataverse(valGetter.apply(req, Parameter.DATAVERSE.str()));
setFormatIfExists(req, acceptHeader, Parameter.FORMAT.str(), valGetter);
setMode(parseIfExists(req, Parameter.MODE.str(), valGetter, getMode(), ResultDelivery::fromName));
@@ -429,12 +477,15 @@
parseBoolean(req, Parameter.REWRITTEN_EXPRESSION_TREE.str(), valGetter, isRewrittenExpressionTree()));
setLogicalPlan(parseBoolean(req, Parameter.LOGICAL_PLAN.str(), valGetter, isLogicalPlan()));
setParseOnly(parseBoolean(req, Parameter.PARSE_ONLY.str(), valGetter, isParseOnly()));
+ setCompileOnly(parseBoolean(req, Parameter.COMPILE_ONLY.str, valGetter, isCompileOnly()));
setReadOnly(parseBoolean(req, Parameter.READ_ONLY.str(), valGetter, isReadOnly()));
setOptimizedLogicalPlan(
parseBoolean(req, Parameter.OPTIMIZED_LOGICAL_PLAN.str(), valGetter, isOptimizedLogicalPlan()));
setMultiStatement(parseBoolean(req, Parameter.MULTI_STATEMENT.str(), valGetter, isMultiStatement()));
setJob(parseBoolean(req, Parameter.JOB.str(), valGetter, isJob()));
setSignature(parseBoolean(req, Parameter.SIGNATURE.str(), valGetter, isSignature()));
+ setClientType(parseIfExists(req, Parameter.CLIENT_TYPE.str(), valGetter, getClientType(), clientTypes::get));
+ setSQLCompatMode(parseBoolean(req, Parameter.SQL_COMPAT.str(), valGetter, isSQLCompatMode()));
}
protected void setExtraParams(JsonNode jsonRequest) throws HyracksDataException {
@@ -586,8 +637,11 @@
if (mimeSplits.length > 0) {
String format = mimeSplits[0].toLowerCase().trim();
if (format.equals(HttpUtil.ContentType.APPLICATION_JSON)) {
- return Pair.of(hasValue(mimeSplits, Attribute.LOSSLESS.str(), booleanValues)
- ? OutputFormat.LOSSLESS_JSON : OutputFormat.CLEAN_JSON, Boolean.FALSE);
+ return Pair
+ .of(hasValue(mimeSplits, Attribute.LOSSLESS.str(), booleanValues) ? OutputFormat.LOSSLESS_JSON
+ : hasValue(mimeSplits, Attribute.LOSSLESS_ADM.str(), booleanValues)
+ ? OutputFormat.LOSSLESS_ADM_JSON : OutputFormat.CLEAN_JSON,
+ Boolean.FALSE);
} else if (format.equals(HttpUtil.ContentType.TEXT_CSV)) {
return Pair.of(OutputFormat.CSV,
hasValue(mimeSplits, Attribute.HEADER.str(), csvHeaderValues) ? Boolean.TRUE : Boolean.FALSE);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index a043050..9f83aa8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -46,7 +46,6 @@
import org.apache.asterix.app.result.fields.PlansPrinter;
import org.apache.asterix.app.result.fields.ProfilePrinter;
import org.apache.asterix.app.result.fields.RequestIdPrinter;
-import org.apache.asterix.app.result.fields.SignaturePrinter;
import org.apache.asterix.app.result.fields.StatusPrinter;
import org.apache.asterix.app.result.fields.TypePrinter;
import org.apache.asterix.app.result.fields.WarningsPrinter;
@@ -63,6 +62,7 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.hyracks.bootstrap.ApplicationConfigurator;
import org.apache.asterix.lang.common.base.IParser;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.Statement;
@@ -145,6 +145,10 @@
response.setHeader("Access-Control-Allow-Origin", request.getHeader("Origin"));
}
response.setHeader("Access-Control-Allow-Headers", "Origin, X-Requested-With, Content-Type, Accept");
+ String server = getServerHeaderValue();
+ if (server != null) {
+ HttpUtil.setServerHeader(response, server);
+ }
response.setStatus(HttpResponseStatus.OK);
}
@@ -292,7 +296,7 @@
responsePrinter.addResultPrinter(new ParseOnlyResultPrinter(parseOnlyResult));
} else {
Map<String, byte[]> statementParams = org.apache.asterix.app.translator.RequestParameters
- .serializeParameterValues(param.getStatementParams());
+ .serializeParameterValues(param.getStatementParams(), sessionOutput.config().fmt());
setAccessControlHeaders(request, response);
stats.setProfileType(param.getProfileType());
IStatementExecutor.StatementProperties statementProperties =
@@ -305,9 +309,9 @@
}
errorCount = 0;
} catch (Exception | org.apache.asterix.lang.sqlpp.parser.TokenMgrError e) {
- handleExecuteStatementException(e, executionState, param);
+ handleExecuteStatementException(e, executionState, param, response);
response.setStatus(executionState.getHttpStatus());
- requestFailed(e, responsePrinter);
+ requestFailed(e, responsePrinter, executionState);
} finally {
executionState.finish();
}
@@ -330,9 +334,6 @@
if (param.getClientContextID() != null && !param.getClientContextID().isEmpty()) {
responsePrinter.addHeaderPrinter(new ClientContextIdPrinter(param.getClientContextID()));
}
- if (param.isSignature() && delivery != ResultDelivery.ASYNC && !param.isParseOnly()) {
- responsePrinter.addHeaderPrinter(SignaturePrinter.INSTANCE);
- }
if (sessionOutput.config().fmt() == SessionConfig.OutputFormat.ADM
|| sessionOutput.config().fmt() == SessionConfig.OutputFormat.CSV) {
responsePrinter.addHeaderPrinter(new TypePrinter(sessionOutput.config()));
@@ -419,7 +420,7 @@
}
protected boolean handleIFormattedException(IError error, IFormattedException ex,
- RequestExecutionState executionState, QueryServiceRequestParameters param) {
+ RequestExecutionState executionState, QueryServiceRequestParameters param, IServletResponse response) {
if (error instanceof ErrorCode) {
switch ((ErrorCode) error) {
case INVALID_REQ_PARAM_VAL:
@@ -451,7 +452,7 @@
}
protected void handleExecuteStatementException(Throwable t, RequestExecutionState executionState,
- QueryServiceRequestParameters param) {
+ QueryServiceRequestParameters param, IServletResponse response) {
if (t instanceof org.apache.asterix.lang.sqlpp.parser.TokenMgrError || t instanceof AlgebricksException) {
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("handleException: {}: {}", t.getMessage(), LogRedactionUtil.statement(param.toString()),
@@ -465,8 +466,8 @@
} else if (t instanceof IFormattedException) {
IFormattedException formattedEx = (IFormattedException) t;
Optional<IError> maybeError = formattedEx.getError();
- if (maybeError.isPresent()
- && handleIFormattedException(maybeError.get(), (IFormattedException) t, executionState, param)) {
+ if (maybeError.isPresent() && handleIFormattedException(maybeError.get(), (IFormattedException) t,
+ executionState, param, response)) {
return;
}
}
@@ -479,11 +480,14 @@
String handleUrl = getHandleUrl(param.getHost(), param.getPath(), delivery);
sessionOutput.setHandleAppender(ResultUtil.createResultHandleAppender(handleUrl));
SessionConfig sessionConfig = sessionOutput.config();
+ SessionConfig.ClientType clientType = param.getClientType();
SessionConfig.OutputFormat format = param.getFormat();
SessionConfig.PlanFormat planFormat = param.getPlanFormat();
+ sessionConfig.setClientType(clientType);
sessionConfig.setFmt(format);
sessionConfig.setPlanFormat(planFormat);
sessionConfig.setMaxWarnings(param.getMaxWarnings());
+ sessionConfig.setExecuteQuery(!param.isCompileOnly());
sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, true);
sessionConfig.set(SessionConfig.OOB_EXPR_TREE, param.isExpressionTree());
sessionConfig.set(SessionConfig.OOB_REWRITTEN_EXPR_TREE, param.isRewrittenExpressionTree());
@@ -492,11 +496,13 @@
sessionConfig.set(SessionConfig.OOB_HYRACKS_JOB, param.isJob());
sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, param.isPretty());
sessionConfig.set(SessionConfig.FORMAT_QUOTE_RECORD,
- format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON);
+ format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON
+ && format != SessionConfig.OutputFormat.LOSSLESS_ADM_JSON);
sessionConfig.set(SessionConfig.FORMAT_CSV_HEADER, param.isCSVWithHeader());
}
- protected void requestFailed(Throwable throwable, ResponsePrinter responsePrinter) {
+ protected void requestFailed(Throwable throwable, ResponsePrinter responsePrinter,
+ RequestExecutionState executionState) {
final ExecutionError executionError = ExecutionError.of(throwable);
responsePrinter.addResultPrinter(new ErrorsPrinter(Collections.singletonList(executionError)));
}
@@ -509,12 +515,32 @@
IRequestReference requestReference, String statementsText, IResultSet resultSet,
ResultProperties resultProperties, Stats stats, IStatementExecutor.StatementProperties statementProperties,
Map<String, String> optionalParameters, Map<String, IAObject> stmtParams, int stmtCategoryRestriction) {
- return new RequestParameters(requestReference, statementsText, resultSet, resultProperties, stats,
- statementProperties, null, param.getClientContextID(), optionalParameters, stmtParams,
- param.isMultiStatement(), stmtCategoryRestriction);
+ RequestParameters requestParameters = new RequestParameters(requestReference, statementsText, resultSet,
+ resultProperties, stats, statementProperties, null, param.getClientContextID(), param.getDataverse(),
+ optionalParameters, stmtParams, param.isMultiStatement(), stmtCategoryRestriction);
+ requestParameters.setPrintSignature(param.isSignature());
+ requestParameters.setSQLCompatMode(param.isSQLCompatMode());
+ return requestParameters;
}
protected static boolean isPrintingProfile(IStatementExecutor.Stats stats) {
return stats.getProfileType() == Stats.ProfileType.FULL && stats.getJobProfile() != null;
}
+
+ protected final String getServerHeaderValue() {
+ String name = getApplicationName();
+ if (name == null) {
+ return null;
+ }
+ String version = getApplicationVersion();
+ return version != null ? name + "/" + version : name;
+ }
+
+ protected String getApplicationName() {
+ return ApplicationConfigurator.APPLICATION_NAME;
+ }
+
+ protected String getApplicationVersion() {
+ return ApplicationConfigurator.getApplicationVersion(appCtx.getBuildProperties());
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
index f1652ea..58a37bd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
@@ -45,9 +45,12 @@
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.rebalance.NoOpDatasetRebalanceCallback;
import org.apache.asterix.utils.RebalanceUtil;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.collections4.IterableUtils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.http.api.IServletRequest;
@@ -181,7 +184,7 @@
if (datasetName == null) {
// Rebalances datasets in a given dataverse or all non-metadata datasets.
- List<Dataset> datasets = dataverseName == null ? getAllDatasetsForRebalance()
+ Iterable<Dataset> datasets = dataverseName == null ? getAllDatasetsForRebalance()
: getAllDatasetsForRebalance(dataverseName);
for (Dataset dataset : datasets) {
// By the time rebalanceDataset(...) is called, the dataset could have been dropped.
@@ -212,8 +215,8 @@
}
// Lists all datasets that should be rebalanced in a given datavserse.
- private List<Dataset> getAllDatasetsForRebalance(DataverseName dataverseName) throws Exception {
- List<Dataset> datasets;
+ private Iterable<Dataset> getAllDatasetsForRebalance(DataverseName dataverseName) throws Exception {
+ Iterable<Dataset> datasets;
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
try {
datasets = getDatasetsInDataverseForRebalance(dataverseName, mdTxnCtx);
@@ -226,13 +229,13 @@
}
// Lists all datasets that should be rebalanced.
- private List<Dataset> getAllDatasetsForRebalance() throws Exception {
+ private Iterable<Dataset> getAllDatasetsForRebalance() throws Exception {
List<Dataset> datasets = new ArrayList<>();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
try {
List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
for (Dataverse dv : dataverses) {
- datasets.addAll(getDatasetsInDataverseForRebalance(dv.getDataverseName(), mdTxnCtx));
+ CollectionUtils.addAll(datasets, getDatasetsInDataverseForRebalance(dv.getDataverseName(), mdTxnCtx));
}
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
@@ -243,10 +246,11 @@
}
// Gets all datasets in a dataverse for the rebalance operation, with a given metadata transaction context.
- private List<Dataset> getDatasetsInDataverseForRebalance(DataverseName dvName, MetadataTransactionContext mdTxnCtx)
- throws Exception {
+ private Iterable<Dataset> getDatasetsInDataverseForRebalance(DataverseName dvName,
+ MetadataTransactionContext mdTxnCtx) throws Exception {
return MetadataConstants.METADATA_DATAVERSE_NAME.equals(dvName) ? Collections.emptyList()
- : MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dvName);
+ : IterableUtils.filteredIterable(MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dvName),
+ DatasetUtil::isNotView);
}
// Rebalances a given dataset.
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
index 045d64c..cf535d9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
@@ -26,9 +26,11 @@
import java.io.StringWriter;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Set;
+import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@@ -39,6 +41,7 @@
import org.apache.asterix.app.result.fields.StatusPrinter;
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.translator.IStatementExecutor.Stats;
@@ -318,7 +321,8 @@
}
public static class ParseOnlyResult {
- private Set<VariableExpr> externalVariables;
+
+ private final Set<VariableExpr> externalVariables;
private static final String STMT_PARAM_LBL = "statement-parameters";
@@ -327,13 +331,29 @@
}
public String asJson() {
+ StringBuilder output = new StringBuilder();
+ output.append("{\"").append(STMT_PARAM_LBL).append("\":");
+ printStatementParameters(externalVariables, VariableExpr::getVar, output);
+ output.append('}');
+ return output.toString();
+ }
- ArrayList<String> positionalVars = new ArrayList<>();
- ArrayList<String> namedVars = new ArrayList<>();
+ public static <T> String printStatementParameters(Collection<T> externalVariables,
+ Function<T, VarIdentifier> varIdentAccessor) {
+ StringBuilder output = new StringBuilder(externalVariables.size() * 12);
+ printStatementParameters(externalVariables, varIdentAccessor, output);
+ return output.toString();
+ }
- for (VariableExpr extVarRef : externalVariables) {
- String varname = extVarRef.getVar().getValue();
- if (SqlppVariableUtil.isPositionalVariableIdentifier(extVarRef.getVar())) {
+ private static <T> void printStatementParameters(Collection<T> externalVariables,
+ Function<T, VarIdentifier> varIdentAccessor, StringBuilder output) {
+ List<String> positionalVars = new ArrayList<>();
+ List<String> namedVars = new ArrayList<>();
+
+ for (T extVarItem : externalVariables) {
+ VarIdentifier extVar = varIdentAccessor.apply(extVarItem);
+ String varname = extVar.getValue();
+ if (SqlppVariableUtil.isPositionalVariableIdentifier(extVar)) {
positionalVars.add(SqlppVariableUtil.toUserDefinedName(varname));
} else {
namedVars.add(SqlppVariableUtil.toUserDefinedName(varname));
@@ -341,14 +361,14 @@
}
Collections.sort(positionalVars);
Collections.sort(namedVars);
- final StringBuilder output = new StringBuilder();
- output.append("{\"").append(STMT_PARAM_LBL).append("\":[");
+
+ output.append('[');
boolean first = true;
for (String posVar : positionalVars) {
if (first) {
first = false;
} else {
- output.append(",");
+ output.append(',');
}
output.append(posVar);
}
@@ -358,10 +378,9 @@
} else {
output.append(",");
}
- output.append("\"").append(namedVar).append("\"");
+ output.append('"').append(namedVar).append('"');
}
- output.append("]}");
- return output.toString();
+ output.append(']');
}
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
index d7d5994..26fee28 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
@@ -135,6 +135,7 @@
final ObjectNode replicaJson = OBJECT_MAPPER.createObjectNode();
replicaJson.put("location", toHostPort(replica.getIdentifier().getLocation()));
replicaJson.put("status", replica.getStatus().toString());
+ replicaJson.put("nodeId", replica.getIdentifier().getNodeId());
replicasArray.add(replicaJson);
}
partitionJson.set("replicas", replicasArray);
@@ -167,11 +168,12 @@
final String partition = request.getParameter("partition");
final String host = request.getParameter("host");
final String port = request.getParameter("port");
- if (partition == null || host == null || port == null) {
+ final String nodeId = request.getParameter("nodeId");
+ if (partition == null || host == null || port == null || nodeId == null) {
return null;
}
- final InetSocketAddress replicaAddress = new InetSocketAddress(host, Integer.valueOf(port));
- return ReplicaIdentifier.of(Integer.valueOf(partition), replicaAddress);
+ final InetSocketAddress replicaAddress = new InetSocketAddress(host, Integer.parseInt(port));
+ return ReplicaIdentifier.of(Integer.parseInt(partition), nodeId, replicaAddress);
}
private void processPromote(IServletRequest request, IServletResponse response) throws HyracksDataException {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
index 4fe6582..4b70f95 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
@@ -137,7 +137,7 @@
RequestReference.of(UUID.randomUUID().toString(), "CC", System.currentTimeMillis());
final IRequestParameters requestParameters = new RequestParameters(requestReference, statement, null,
new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE), new IStatementExecutor.Stats(),
- new IStatementExecutor.StatementProperties(), null, null, null, statementParams, true);
+ new IStatementExecutor.StatementProperties(), null, null, null, null, statementParams, true);
translator.compileAndExecute(hcc, requestParameters);
executionPlans = translator.getExecutionPlans();
writer.flush();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
index 0242ecd..ddd3d64 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
@@ -195,7 +195,7 @@
@SuppressWarnings("unchecked")
protected void finish(ActiveEvent event) throws HyracksDataException {
if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "the job " + jobId + " finished");
+ LOGGER.log(level, "the job {} finished", jobId);
}
JobId lastJobId = jobId;
if (numRegistered != numDeRegistered) {
@@ -208,7 +208,7 @@
JobStatus jobStatus = status.getLeft();
List<Exception> exceptions = status.getRight();
if (LOGGER.isEnabled(level)) {
- LOGGER.log(level, "The job finished with status: " + jobStatus);
+ LOGGER.log(level, "The job finished with status: {}", jobStatus);
}
if (!jobSuccessfullyTerminated(jobStatus)) {
jobFailure = exceptions.isEmpty() ? new RuntimeDataException(ErrorCode.UNREPORTED_TASK_FAILURE_EXCEPTION)
@@ -440,8 +440,9 @@
private void cancelJob(Throwable th) {
cancelJobSafely(metadataProvider, th);
+ // we can come here due to a failure while in suspending state
final WaitForStateSubscriber cancelSubscriber =
- new WaitForStateSubscriber(this, EnumSet.of(ActivityState.STOPPED));
+ new WaitForStateSubscriber(this, EnumSet.of(ActivityState.STOPPED, ActivityState.TEMPORARILY_FAILED));
final Span span = Span.start(2, TimeUnit.MINUTES);
InvokeUtil.doUninterruptibly(() -> {
if (!cancelSubscriber.sync(span)) {
@@ -491,6 +492,7 @@
forceStop(subscriber, ie);
Thread.currentThread().interrupt();
} catch (Throwable e) {
+ LOGGER.error("forcing active job stop due to", e);
forceStop(subscriber, e);
} finally {
Thread.currentThread().setName(nameBefore);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index 0d63bca..6b5cae2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -54,7 +54,6 @@
public static final String ACTIVE_ENTITY_PROPERTY_NAME = "ActiveJob";
private final Map<EntityId, IActiveEntityEventsListener> entityEventListeners;
private final Map<JobId, EntityId> jobId2EntityId;
- private boolean initialized = false;
private boolean suspended = false;
public ActiveNotificationHandler() {
@@ -223,19 +222,6 @@
}
@Override
- public boolean isInitialized() {
- return initialized;
- }
-
- @Override
- public void setInitialized(boolean initialized) throws HyracksDataException {
- if (this.initialized) {
- throw new RuntimeDataException(ErrorCode.DOUBLE_INITIALIZATION_OF_ACTIVE_NOTIFICATION_HANDLER);
- }
- this.initialized = initialized;
- }
-
- @Override
public void recover() {
LOGGER.info("Starting active recovery");
for (IActiveEntityEventsListener listener : getEventListeners()) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
index 6944b25..b1c2f0a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
@@ -19,20 +19,22 @@
package org.apache.asterix.app.function;
import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.PLURAL;
+import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.SINGULAR;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
import java.util.ArrayList;
import java.util.List;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.metadata.declared.DataSource;
import org.apache.asterix.metadata.declared.DataSourceId;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.om.functions.IFunctionToDataSourceRewriter;
import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
import org.apache.asterix.om.types.ARecordType;
@@ -41,7 +43,6 @@
import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -75,15 +76,25 @@
MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
Dataset dataset = fetchDataset(metadataProvider, f);
- DataSourceId dsid = new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName());
List<LogicalVariable> variables = new ArrayList<>();
- if (dataset.getDatasetType() == DatasetType.INTERNAL) {
- int numPrimaryKeys = dataset.getPrimaryKeys().size();
- for (int i = 0; i < numPrimaryKeys; i++) {
- variables.add(context.newVar());
- }
+ switch (dataset.getDatasetType()) {
+ case INTERNAL:
+ int numPrimaryKeys = dataset.getPrimaryKeys().size();
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ variables.add(context.newVar());
+ }
+ break;
+ case EXTERNAL:
+ break;
+ default:
+ // VIEWS are not expected at this point
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, unnest.getSourceLocation(),
+ String.format("Unexpected %s type %s for %s %s", dataset(SINGULAR), dataset.getDatasetType(),
+ dataset(SINGULAR), DatasetUtil.getFullyQualifiedDisplayName(dataset)));
}
variables.add(unnest.getVariable());
+
+ DataSourceId dsid = new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName());
DataSource dataSource = metadataProvider.findDataSource(dsid);
boolean hasMeta = dataSource.hasMeta();
if (hasMeta) {
@@ -132,9 +143,9 @@
public static Dataset fetchDataset(MetadataProvider metadataProvider, AbstractFunctionCallExpression datasetFnCall)
throws CompilationException {
- Pair<DataverseName, String> datasetReference = FunctionUtil.parseDatasetFunctionArguments(datasetFnCall);
- DataverseName dataverseName = datasetReference.first;
- String datasetName = datasetReference.second;
+ DatasetFullyQualifiedName datasetReference = FunctionUtil.parseDatasetFunctionArguments(datasetFnCall);
+ DataverseName dataverseName = datasetReference.getDataverseName();
+ String datasetName = datasetReference.getDatasetName();
Dataset dataset;
try {
dataset = metadataProvider.findDataset(dataverseName, datasetName);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
index 2a22ac6..aed15c6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexReader.java
@@ -24,6 +24,8 @@
import org.apache.asterix.external.api.IRawRecord;
import org.apache.asterix.external.input.record.CharArrayRecord;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -36,6 +38,7 @@
import org.apache.hyracks.storage.common.IIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.util.JSONUtil;
public class DumpIndexReader extends FunctionReader {
@@ -98,10 +101,30 @@
recordBuilder.append("{\"values\":[");
for (int j = 0; j < tuple.getFieldCount(); ++j) {
bbis.setByteBuffer(ByteBuffer.wrap(tuple.getFieldData(j)), tuple.getFieldStart(j));
- recordBuilder.append(secondaryRecDesc.getFields()[j].deserialize(dis));
+ IAObject field = (IAObject) secondaryRecDesc.getFields()[j].deserialize(dis);
+ ATypeTag tag = field.getType().getTypeTag();
+ if (tag == ATypeTag.MISSING) {
+ continue;
+ }
+ if (isTemporal(tag)) {
+ JSONUtil.quoteAndEscape(recordBuilder, field.toString());
+ } else {
+ recordBuilder.append(field);
+ }
recordBuilder.append(",");
}
recordBuilder.deleteCharAt(recordBuilder.length() - 1);
recordBuilder.append("]}");
}
+
+ private static boolean isTemporal(ATypeTag typeTag) {
+ switch (typeTag) {
+ case DATE:
+ case TIME:
+ case DATETIME:
+ return true;
+ default:
+ return false;
+ }
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index 4cc6e43..4850bf6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -53,6 +53,7 @@
import org.apache.asterix.dataflow.data.nontagged.comparators.ListItemBinaryComparatorFactory;
import org.apache.asterix.dataflow.data.nontagged.valueproviders.PrimitiveValueProviderFactory;
import org.apache.asterix.formats.nontagged.AnyBinaryComparatorFactory;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
import org.apache.asterix.formats.nontagged.OrderedBinaryComparatorFactory;
import org.apache.asterix.formats.nontagged.OrderedLinearizeComparatorFactory;
import org.apache.asterix.metadata.utils.SecondaryCorrelatedTreeIndexOperationsHelper;
@@ -109,6 +110,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexLocalResource;
import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
@@ -175,6 +177,9 @@
registeredClasses.put("FixedLengthTypeTrait", FixedLengthTypeTrait.class);
registeredClasses.put("VarLengthTypeTrait", VarLengthTypeTrait.class);
+ // INullIntrospector
+ registeredClasses.put("NullIntrospector", NullIntrospector.class);
+
// ILSMOperationTrackerFactory
registeredClasses.put("PrimaryIndexOperationTrackerFactory", PrimaryIndexOperationTrackerFactory.class);
registeredClasses.put("SecondaryIndexOperationTrackerFactory", SecondaryIndexOperationTrackerFactory.class);
@@ -189,6 +194,7 @@
registeredClasses.put("NoOpIOOperationCallbackFactory", NoOpIOOperationCallbackFactory.class);
registeredClasses.put("LSMBTreeIOOperationCallbackFactory", LSMIndexIOOperationCallbackFactory.class);
registeredClasses.put("LSMIndexPageWriteCallbackFactory", LSMIndexPageWriteCallbackFactory.class);
+ registeredClasses.put("NoOpPageWriteCallbackFactory", NoOpPageWriteCallbackFactory.class);
// ILSMIOOperationSchedulerProvider
registeredClasses.put("AppendOnlyLinkedMetadataPageManagerFactory",
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractInternalRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractInternalRequestMessage.java
index 26aac63..7d58e4e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractInternalRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractInternalRequestMessage.java
@@ -80,8 +80,7 @@
final RuntimeDataException rejectionReason =
ExecuteStatementRequestMessage.getRejectionReason(ccSrv, nodeRequestId);
if (rejectionReason != null) {
- ExecuteStatementRequestMessage.sendRejection(rejectionReason, messageBroker, requestMessageId,
- nodeRequestId);
+ sendRejection(rejectionReason, messageBroker, requestMessageId, nodeRequestId);
return;
}
CCExtensionManager ccExtMgr = (CCExtensionManager) ccAppCtx.getExtensionManager();
@@ -94,7 +93,7 @@
ResponsePrinter printer = new ResponsePrinter(sessionOutput);
ResultProperties resultProperties = new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE, 1);
IRequestParameters requestParams = new RequestParameters(requestReference, "", null, resultProperties,
- new IStatementExecutor.Stats(), new IStatementExecutor.StatementProperties(), null, null,
+ new IStatementExecutor.Stats(), new IStatementExecutor.StatementProperties(), null, null, null,
additionalParams, Collections.emptyMap(), false);
MetadataManager.INSTANCE.init();
IStatementExecutor translator =
@@ -120,4 +119,14 @@
protected abstract Statement produceStatement();
+ private void sendRejection(Exception reason, CCMessageBroker messageBroker, long requestMessageId,
+ String requestNodeId) {
+ InternalRequestResponse msg = new InternalRequestResponse(requestMessageId);
+ msg.setError(reason);
+ try {
+ messageBroker.sendApplicationMessageToNC(msg, requestNodeId);
+ } catch (Exception e) {
+ LOGGER.log(Level.WARN, e.toString(), e);
+ }
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 29ee76d..0a5e033 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -66,7 +66,7 @@
import org.apache.logging.log4j.Logger;
public class ExecuteStatementRequestMessage implements ICcAddressedMessage {
- private static final long serialVersionUID = 2L;
+ private static final long serialVersionUID = 4L;
private static final Logger LOGGER = LogManager.getLogger();
//TODO: Make configurable: https://issues.apache.org/jira/browse/ASTERIXDB-2062
public static final long DEFAULT_NC_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(Long.MAX_VALUE);
@@ -79,6 +79,7 @@
private final SessionConfig sessionConfig;
private final ResultProperties resultProperties;
private final String clientContextID;
+ private final String defaultDataverseName;
private final String handleUrl;
private final Map<String, String> optionalParameters;
private final Map<String, byte[]> statementParameters;
@@ -88,23 +89,25 @@
private final IRequestReference requestReference;
private final boolean forceDropDataset;
private final boolean skipAdmissionPolicy;
+ private boolean sqlCompatMode;
public ExecuteStatementRequestMessage(String requestNodeId, long requestMessageId, ILangExtension.Language lang,
String statementsText, SessionConfig sessionConfig, ResultProperties resultProperties,
- String clientContextID, String handleUrl, Map<String, String> optionalParameters,
- Map<String, byte[]> statementParameters, boolean multiStatement, ProfileType profileType,
- int statementCategoryRestrictionMask, IRequestReference requestReference, boolean forceDropDataset) {
+ String clientContextID, String defaultDataverseName, String handleUrl,
+ Map<String, String> optionalParameters, Map<String, byte[]> statementParameters, boolean multiStatement,
+ ProfileType profileType, int statementCategoryRestrictionMask, IRequestReference requestReference,
+ boolean forceDropDataset) {
this(requestNodeId, requestMessageId, lang, statementsText, sessionConfig, resultProperties, clientContextID,
- handleUrl, optionalParameters, statementParameters, multiStatement, profileType,
+ defaultDataverseName, handleUrl, optionalParameters, statementParameters, multiStatement, profileType,
statementCategoryRestrictionMask, requestReference, forceDropDataset, false);
}
protected ExecuteStatementRequestMessage(String requestNodeId, long requestMessageId, ILangExtension.Language lang,
String statementsText, SessionConfig sessionConfig, ResultProperties resultProperties,
- String clientContextID, String handleUrl, Map<String, String> optionalParameters,
- Map<String, byte[]> statementParameters, boolean multiStatement, ProfileType profileType,
- int statementCategoryRestrictionMask, IRequestReference requestReference, boolean forceDropDataset,
- boolean skipAdmissionPolicy) {
+ String clientContextID, String defaultDataverseName, String handleUrl,
+ Map<String, String> optionalParameters, Map<String, byte[]> statementParameters, boolean multiStatement,
+ ProfileType profileType, int statementCategoryRestrictionMask, IRequestReference requestReference,
+ boolean forceDropDataset, boolean skipAdmissionPolicy) {
this.requestNodeId = requestNodeId;
this.requestMessageId = requestMessageId;
this.lang = lang;
@@ -121,6 +124,15 @@
this.requestReference = requestReference;
this.forceDropDataset = forceDropDataset;
this.skipAdmissionPolicy = skipAdmissionPolicy;
+ this.defaultDataverseName = defaultDataverseName;
+ }
+
+ public boolean isSQLCompatMode() {
+ return sqlCompatMode;
+ }
+
+ public void setSQLCompatMode(boolean sqlCompatMode) {
+ this.sqlCompatMode = sqlCompatMode;
}
@Override
@@ -139,6 +151,8 @@
IStorageComponentProvider storageComponentProvider = ccAppCtx.getStorageComponentProvider();
IStatementExecutorFactory statementExecutorFactory = ccApp.getStatementExecutorFactory();
ExecuteStatementResponseMessage responseMsg = new ExecuteStatementResponseMessage(requestMessageId);
+ final IStatementExecutor.StatementProperties statementProperties = new IStatementExecutor.StatementProperties();
+ responseMsg.setStatementProperties(statementProperties);
try {
List<Warning> warnings = new ArrayList<>();
IParser parser = compilationProvider.getParserFactory().createParser(statementsText);
@@ -160,13 +174,9 @@
compilationProvider, storageComponentProvider, new ResponsePrinter(sessionOutput));
final IStatementExecutor.Stats stats = new IStatementExecutor.Stats();
stats.setProfileType(profileType);
- final IStatementExecutor.StatementProperties statementProperties =
- new IStatementExecutor.StatementProperties();
Map<String, IAObject> stmtParams = RequestParameters.deserializeParameterValues(statementParameters);
final IRequestParameters requestParameters =
- new RequestParameters(requestReference, statementsText, null, resultProperties, stats,
- statementProperties, outMetadata, clientContextID, optionalParameters, stmtParams,
- multiStatement, statementCategoryRestrictionMask, forceDropDataset, skipAdmissionPolicy);
+ createRequestParameters(statementProperties, stmtParams, outMetadata, stats);
translator.compileAndExecute(ccApp.getHcc(), requestParameters);
translator.getWarnings(warnings, maxWarnings - warnings.size());
stats.updateTotalWarningsCount(parserTotalWarningsCount);
@@ -174,7 +184,6 @@
responseMsg.setResult(outWriter.toString());
responseMsg.setMetadata(outMetadata);
responseMsg.setStats(stats);
- responseMsg.setStatementProperties(statementProperties);
responseMsg.setExecutionPlans(translator.getExecutionPlans());
responseMsg.setWarnings(warnings);
} catch (AlgebricksException | HyracksException | org.apache.asterix.lang.sqlpp.parser.TokenMgrError pe) {
@@ -192,6 +201,17 @@
}
}
+ protected IRequestParameters createRequestParameters(IStatementExecutor.StatementProperties statementProperties,
+ Map<String, IAObject> stmtParams, IStatementExecutor.ResultMetadata outMetadata,
+ IStatementExecutor.Stats stats) {
+ RequestParameters requestParameters = new RequestParameters(requestReference, statementsText, null,
+ resultProperties, stats, statementProperties, outMetadata, clientContextID, defaultDataverseName,
+ optionalParameters, stmtParams, multiStatement, statementCategoryRestrictionMask, forceDropDataset,
+ skipAdmissionPolicy);
+ requestParameters.setSQLCompatMode(sqlCompatMode);
+ return requestParameters;
+ }
+
protected CCMessageBroker getMessageBroker(ICcApplicationContext ccAppCtx) {
ICCServiceContext ccSrvContext = ccAppCtx.getServiceContext();
return (CCMessageBroker) ccSrvContext.getMessageBroker();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
index cbc4d94..96e89a7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
@@ -34,6 +34,7 @@
import org.apache.asterix.common.storage.IndexCheckpoint;
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
import org.apache.hyracks.util.annotations.ThreadSafe;
import org.apache.hyracks.util.file.FileUtil;
import org.apache.logging.log4j.LogManager;
@@ -55,7 +56,7 @@
}
@Override
- public synchronized void init(long validComponentSequence, long lsn, long validComponentId)
+ public synchronized void init(long validComponentSequence, long lsn, long validComponentId, String masterNodeId)
throws HyracksDataException {
List<IndexCheckpoint> checkpoints;
try {
@@ -67,26 +68,34 @@
LOGGER.warn(() -> "Checkpoints found on initializing: " + indexPath);
delete();
}
- IndexCheckpoint firstCheckpoint = IndexCheckpoint.first(validComponentSequence, lsn, validComponentId);
+ IndexCheckpoint firstCheckpoint =
+ IndexCheckpoint.first(validComponentSequence, lsn, validComponentId, masterNodeId);
persist(firstCheckpoint);
}
@Override
- public synchronized void replicated(long componentSequence, long masterLsn, long componentId)
+ public synchronized void replicated(long componentSequence, long masterLsn, long componentId, String masterNodeId)
throws HyracksDataException {
final Long localLsn = getLatest().getMasterNodeFlushMap().get(masterLsn);
if (localLsn == null) {
- throw new IllegalStateException("Component flushed before lsn mapping was received");
+ throw new IllegalStateException("Component replicated before lsn mapping was received");
}
- flushed(componentSequence, localLsn, componentId);
+ flushed(componentSequence, localLsn, componentId, masterNodeId);
+ }
+
+ @Override
+ public synchronized void flushed(long componentSequence, long lsn, long componentId, String masterNodeId)
+ throws HyracksDataException {
+ final IndexCheckpoint latest = getLatest();
+ IndexCheckpoint nextCheckpoint =
+ IndexCheckpoint.next(latest, lsn, componentSequence, componentId, masterNodeId);
+ persist(nextCheckpoint);
+ deleteHistory(nextCheckpoint.getId(), HISTORY_CHECKPOINTS);
}
@Override
public synchronized void flushed(long componentSequence, long lsn, long componentId) throws HyracksDataException {
- final IndexCheckpoint latest = getLatest();
- IndexCheckpoint nextCheckpoint = IndexCheckpoint.next(latest, lsn, componentSequence, componentId);
- persist(nextCheckpoint);
- deleteHistory(nextCheckpoint.getId(), HISTORY_CHECKPOINTS);
+ flushed(componentSequence, lsn, componentId, null);
}
@Override
@@ -94,7 +103,7 @@
final IndexCheckpoint latest = getLatest();
latest.getMasterNodeFlushMap().put(masterLsn, localLsn);
final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(),
- latest.getValidComponentSequence(), latest.getLastComponentId());
+ latest.getValidComponentSequence(), latest.getLastComponentId(), null);
persist(next);
notifyAll();
}
@@ -119,7 +128,10 @@
@Override
public long getValidComponentSequence() throws HyracksDataException {
- return getLatest().getValidComponentSequence();
+ if (getCheckpointCount() > 0) {
+ return getLatest().getValidComponentSequence();
+ }
+ return AbstractLSMIndexFileManager.UNINITIALIZED_COMPONENT_SEQ;
}
@Override
@@ -151,17 +163,18 @@
@Override
public synchronized void setLastComponentId(long componentId) throws HyracksDataException {
final IndexCheckpoint latest = getLatest();
- final IndexCheckpoint next =
- IndexCheckpoint.next(latest, latest.getLowWatermark(), latest.getValidComponentSequence(), componentId);
+ final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(),
+ latest.getValidComponentSequence(), componentId, null);
persist(next);
}
@Override
- public synchronized void advanceValidComponentSequence(long componentSequence) throws HyracksDataException {
+ public synchronized void advanceValidComponent(long componentSequence, long componentId)
+ throws HyracksDataException {
final IndexCheckpoint latest = getLatest();
if (componentSequence > latest.getValidComponentSequence()) {
- final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(), componentSequence,
- latest.getLastComponentId());
+ final IndexCheckpoint next =
+ IndexCheckpoint.next(latest, latest.getLowWatermark(), componentSequence, componentId, null);
persist(next);
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 9aa433f..f532352 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -21,12 +21,10 @@
import java.io.IOException;
import java.rmi.RemoteException;
import java.rmi.server.UnicastRemoteObject;
-import java.util.Arrays;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
-import java.util.stream.Collectors;
import org.apache.asterix.active.ActiveManager;
import org.apache.asterix.common.api.IConfigValidator;
@@ -38,7 +36,6 @@
import org.apache.asterix.common.api.IPropertiesFactory;
import org.apache.asterix.common.api.IReceptionist;
import org.apache.asterix.common.api.IReceptionistFactory;
-import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.config.ActiveProperties;
import org.apache.asterix.common.config.BuildProperties;
import org.apache.asterix.common.config.CompilerProperties;
@@ -56,6 +53,7 @@
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.common.replication.IReplicationChannel;
import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.replication.IReplicationStrategyFactory;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.storage.IReplicaManager;
import org.apache.asterix.common.transactions.IRecoveryManager;
@@ -179,7 +177,8 @@
@Override
public void initialize(IRecoveryManagerFactory recoveryManagerFactory, IReceptionistFactory receptionistFactory,
- IConfigValidatorFactory configValidatorFactory, boolean initialRun) throws IOException {
+ IConfigValidatorFactory configValidatorFactory, IReplicationStrategyFactory replicationStrategyFactory,
+ boolean initialRun) throws IOException {
ioManager = getServiceContext().getIoManager();
int ioQueueLen = getServiceContext().getAppConfig().getInt(NCConfig.Option.IO_QUEUE_SIZE);
threadExecutor =
@@ -225,10 +224,8 @@
new DatasetLifecycleManager(storageProperties, localResourceRepository, txnSubsystem.getLogManager(),
virtualBufferCache, indexCheckpointManagerProvider, ioManager.getIODevices().size());
final String nodeId = getServiceContext().getNodeId();
- final ClusterPartition[] nodePartitions = metadataProperties.getNodePartitions().get(nodeId);
- final Set<Integer> nodePartitionsIds =
- Arrays.stream(nodePartitions).map(ClusterPartition::getPartitionId).collect(Collectors.toSet());
- replicaManager = new ReplicaManager(this, nodePartitionsIds);
+ final Set<Integer> nodePartitions = metadataProperties.getNodeActivePartitions(nodeId);
+ replicaManager = new ReplicaManager(this, nodePartitions);
isShuttingdown = false;
activeManager = new ActiveManager(threadExecutor, getServiceContext().getNodeId(),
activeProperties.getMemoryComponentGlobalBudget(), compilerProperties.getFrameSize(),
@@ -236,7 +233,7 @@
receptionist = receptionistFactory.create();
if (replicationProperties.isReplicationEnabled()) {
- replicationManager = new ReplicationManager(this, replicationProperties);
+ replicationManager = new ReplicationManager(this, replicationStrategyFactory, replicationProperties);
//pass replication manager to replication required object
//LogManager to replicate logs
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index 0359cf1..6736642 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -160,15 +160,13 @@
@Override
public void startLocalRecovery(Set<Integer> partitions) throws IOException, ACIDException {
state = SystemState.RECOVERING;
- LOGGER.info("starting recovery ...");
-
+ LOGGER.info("starting recovery for partitions {}", partitions);
long readableSmallestLSN = logMgr.getReadableSmallestLSN();
Checkpoint checkpointObject = checkpointManager.getLatest();
long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLsn();
if (lowWaterMarkLSN < readableSmallestLSN) {
lowWaterMarkLSN = readableSmallestLSN;
}
-
//delete any recovery files from previous failed recovery attempts
deleteRecoveryTemporaryFiles();
@@ -293,7 +291,7 @@
final IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
((INcApplicationContext) (serviceCtx.getApplicationContext())).getIndexCheckpointManagerProvider();
- Map<Long, LocalResource> resourcesMap = localResourceRepository.loadAndGetAllResources();
+ Map<Long, LocalResource> resourcesMap = localResourceRepository.getResources(r -> true, partitions);
final Map<Long, Long> resourceId2MaxLSNMap = new HashMap<>();
TxnEntityId tempKeyTxnEntityId = new TxnEntityId(-1, -1, -1, null, -1, false);
@@ -363,10 +361,8 @@
datasetLifecycleManager.register(localResource.getPath(), index);
datasetLifecycleManager.open(localResource.getPath());
try {
- final DatasetResourceReference resourceReference =
- DatasetResourceReference.of(localResource);
- maxDiskLastLsn =
- indexCheckpointManagerProvider.get(resourceReference).getLowWatermark();
+ maxDiskLastLsn = getResourceLowWaterMark(localResource, datasetLifecycleManager,
+ indexCheckpointManagerProvider);
} catch (HyracksDataException e) {
datasetLifecycleManager.close(localResource.getPath());
throw e;
@@ -374,7 +370,12 @@
//#. set resourceId and maxDiskLastLSN to the map
resourceId2MaxLSNMap.put(resourceId, maxDiskLastLsn);
} else {
- maxDiskLastLsn = resourceId2MaxLSNMap.get(resourceId);
+ if (!resourceId2MaxLSNMap.containsKey(resourceId)) {
+ maxDiskLastLsn = getResourceLowWaterMark(localResource, datasetLifecycleManager,
+ indexCheckpointManagerProvider);
+ } else {
+ maxDiskLastLsn = resourceId2MaxLSNMap.get(resourceId);
+ }
}
// lsn @ maxDiskLastLsn is either a flush log or a master replica log
if (lsn >= maxDiskLastLsn) {
@@ -502,7 +503,8 @@
final List<DatasetResourceReference> partitionResources = localResourceRepository.getResources(resource -> {
DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
return dsResource.getPartition() == partition;
- }).values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
+ }, Collections.singleton(partition)).values().stream().map(DatasetResourceReference::of)
+ .collect(Collectors.toList());
for (DatasetResourceReference indexRef : partitionResources) {
try {
final IIndexCheckpointManager idxCheckpointMgr = idxCheckpointMgrProvider.get(indexRef);
@@ -535,6 +537,7 @@
if (flush) {
appCtx.getDatasetLifecycleManager().flushAllDatasets();
}
+ cleanUp(partitions);
} catch (IOException | ACIDException e) {
throw HyracksDataException.create(e);
} finally {
@@ -568,6 +571,10 @@
FileUtils.deleteQuietly(recoveryFolderPath.toFile());
}
+ protected void cleanUp(Set<Integer> partitions) throws HyracksDataException {
+ // the cleanup is currently done by PersistentLocalResourceRepository#clean
+ }
+
private String getRecoveryDirPath() {
String logDir = logMgr.getLogManagerProperties().getLogDir();
if (!logDir.endsWith(File.separator)) {
@@ -858,6 +865,19 @@
index.resetCurrentComponentIndex();
}
+ private long getResourceLowWaterMark(LocalResource localResource, IDatasetLifecycleManager datasetLifecycleManager,
+ IIndexCheckpointManagerProvider indexCheckpointManagerProvider) throws HyracksDataException {
+ long maxDiskLastLsn;
+ try {
+ final DatasetResourceReference resourceReference = DatasetResourceReference.of(localResource);
+ maxDiskLastLsn = indexCheckpointManagerProvider.get(resourceReference).getLowWatermark();
+ } catch (HyracksDataException e) {
+ datasetLifecycleManager.close(localResource.getPath());
+ throw e;
+ }
+ return maxDiskLastLsn;
+ }
+
private class JobEntityCommits {
private static final String PARTITION_FILE_NAME_SEPARATOR = "_";
private final long txnId;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
index ad70cf4..7c4b59c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
@@ -18,7 +18,8 @@
*/
package org.apache.asterix.app.nc;
-import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -30,6 +31,7 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.replication.IPartitionReplica;
import org.apache.asterix.common.storage.IReplicaManager;
import org.apache.asterix.common.storage.ReplicaIdentifier;
@@ -37,9 +39,7 @@
import org.apache.asterix.replication.api.PartitionReplica;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.hyracks.api.client.NodeStatus;
-import org.apache.hyracks.api.config.IApplicationConfig;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.common.controllers.NCConfig;
import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.storage.common.LocalResource;
import org.apache.hyracks.util.annotations.ThreadSafe;
@@ -54,16 +54,19 @@
/**
* the partitions to which the current node is master
*/
- private final Set<Integer> partitions = new HashSet<>();
+ private final Map<Integer, Object> partitions = new HashMap<>();
/**
* current replicas
*/
private final Map<ReplicaIdentifier, PartitionReplica> replicas = new HashMap<>();
- private final Object replicaSyncLock = new Object();
+ private final Set<Integer> nodeOwnedPartitions = new HashSet<>();
public ReplicaManager(INcApplicationContext appCtx, Set<Integer> partitions) {
this.appCtx = appCtx;
- this.partitions.addAll(partitions);
+ for (Integer partition : partitions) {
+ this.partitions.put(partition, new Object());
+ }
+ setNodeOwnedPartitions(appCtx);
}
@Override
@@ -75,7 +78,7 @@
LOGGER.warn("Ignoring request to add replica. Node is not ACTIVE yet. Current status: {}", nodeStatus);
return;
}
- if (!partitions.contains(id.getPartition())) {
+ if (!partitions.containsKey(id.getPartition())) {
throw new IllegalStateException(
"This node is not the current master of partition(" + id.getPartition() + ")");
}
@@ -94,7 +97,6 @@
}
PartitionReplica replica = replicas.remove(id);
appCtx.getReplicationManager().unregister(replica);
-
}
@Override
@@ -104,30 +106,42 @@
}
@Override
+ public synchronized IPartitionReplica getReplica(ReplicaIdentifier id) {
+ return replicas.get(id);
+ }
+
+ @Override
public synchronized Set<Integer> getPartitions() {
- return Collections.unmodifiableSet(partitions);
+ return Collections.unmodifiableSet(partitions.keySet());
+ }
+
+ @Override
+ public synchronized void setActivePartitions(Set<Integer> activePartitions) {
+ partitions.clear();
+ for (Integer partition : activePartitions) {
+ partitions.put(partition, new Object());
+ }
}
@Override
public synchronized void promote(int partition) throws HyracksDataException {
- if (partitions.contains(partition)) {
+ if (partitions.containsKey(partition)) {
return;
}
+ LOGGER.warn("promoting partition {}", partition);
final PersistentLocalResourceRepository localResourceRepository =
(PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
localResourceRepository.cleanup(partition);
final IRecoveryManager recoveryManager = appCtx.getTransactionSubsystem().getRecoveryManager();
recoveryManager.replayReplicaPartitionLogs(Stream.of(partition).collect(Collectors.toSet()), true);
- partitions.add(partition);
+ partitions.put(partition, new Object());
}
@Override
public synchronized void release(int partition) throws HyracksDataException {
- if (!partitions.contains(partition)) {
+ if (!partitions.containsKey(partition)) {
return;
}
- final IDatasetLifecycleManager datasetLifecycleManager = appCtx.getDatasetLifecycleManager();
- datasetLifecycleManager.flushDataset(appCtx.getReplicationManager().getReplicationStrategy());
closePartitionResources(partition);
final List<IPartitionReplica> partitionReplicas = getReplicas(partition);
for (IPartitionReplica replica : partitionReplicas) {
@@ -137,26 +151,48 @@
}
@Override
- public Object getReplicaSyncLock() {
- return replicaSyncLock;
+ public synchronized Object getPartitionSyncLock(int partition) {
+ Object syncLock = partitions.get(partition);
+ if (syncLock == null) {
+ throw new IllegalStateException("partition " + partition + " is not active on this node");
+ }
+ return syncLock;
}
- private void closePartitionResources(int partition) throws HyracksDataException {
+ @Override
+ public synchronized List<IPartitionReplica> getReplicas() {
+ return new ArrayList<>(replicas.values());
+ }
+
+ @Override
+ public boolean isPartitionOwner(int partition) {
+ return nodeOwnedPartitions.contains(partition);
+ }
+
+ public void closePartitionResources(int partition) throws HyracksDataException {
+ final IDatasetLifecycleManager datasetLifecycleManager = appCtx.getDatasetLifecycleManager();
+ //TODO(mhubail) we can flush only datasets of the requested partition
+ datasetLifecycleManager.flushAllDatasets();
final PersistentLocalResourceRepository resourceRepository =
(PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
final Map<Long, LocalResource> partitionResources = resourceRepository.getPartitionResources(partition);
- final IDatasetLifecycleManager datasetLifecycleManager = appCtx.getDatasetLifecycleManager();
for (LocalResource resource : partitionResources.values()) {
- datasetLifecycleManager.close(resource.getPath());
+ datasetLifecycleManager.closeIfOpen(resource.getPath());
}
+ datasetLifecycleManager.closePartition(partition);
}
private boolean isSelf(ReplicaIdentifier id) {
- IApplicationConfig appConfig = appCtx.getServiceContext().getAppConfig();
- String host = appConfig.getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS);
- int port = appConfig.getInt(NCConfig.Option.REPLICATION_LISTEN_PORT);
+ String nodeId = appCtx.getServiceContext().getNodeId();
+ return id.getNodeId().equals(nodeId);
+ }
- final InetSocketAddress replicaAddress = new InetSocketAddress(host, port);
- return id.equals(ReplicaIdentifier.of(id.getPartition(), replicaAddress));
+ private void setNodeOwnedPartitions(INcApplicationContext appCtx) {
+ ClusterPartition[] clusterPartitions =
+ appCtx.getMetadataProperties().getNodePartitions().get(appCtx.getServiceContext().getNodeId());
+ if (clusterPartitions != null) {
+ nodeOwnedPartitions.addAll(Arrays.stream(clusterPartitions).map(ClusterPartition::getPartitionId)
+ .collect(Collectors.toList()));
+ }
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/LocalStorageCleanupTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/LocalStorageCleanupTask.java
new file mode 100644
index 0000000..dd7a663
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/LocalStorageCleanupTask.java
@@ -0,0 +1,67 @@
+/*
+ * 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 LocalStorageCleanupTask 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.app.nc.task;
+
+import java.util.Set;
+
+import org.apache.asterix.common.api.INCLifecycleTask;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.dataflow.DatasetLocalResource;
+import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.control.CcId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.service.IControllerService;
+
+public class LocalStorageCleanupTask implements INCLifecycleTask {
+
+ private static final long serialVersionUID = 1L;
+ private final int metadataPartitionId;
+
+ public LocalStorageCleanupTask(int metadataPartitionId) {
+ this.metadataPartitionId = metadataPartitionId;
+ }
+
+ @Override
+ public void perform(CcId ccId, IControllerService cs) throws HyracksDataException {
+ INcApplicationContext appContext = (INcApplicationContext) cs.getApplicationContext();
+ PersistentLocalResourceRepository localResourceRepository =
+ (PersistentLocalResourceRepository) appContext.getLocalResourceRepository();
+ deleteInvalidMetadataIndexes(localResourceRepository);
+ final Set<Integer> nodePartitions = appContext.getReplicaManager().getPartitions();
+ localResourceRepository.deleteCorruptedResources();
+ for (Integer partition : nodePartitions) {
+ localResourceRepository.cleanup(partition);
+ }
+ }
+
+ private void deleteInvalidMetadataIndexes(PersistentLocalResourceRepository localResourceRepository)
+ throws HyracksDataException {
+ localResourceRepository.deleteInvalidIndexes(r -> {
+ DatasetLocalResource lr = (DatasetLocalResource) r.getResource();
+ return MetadataIndexImmutableProperties.isMetadataDataset(lr.getDatasetId())
+ && lr.getPartition() != metadataPartitionId;
+ });
+ }
+
+ @Override
+ public String toString() {
+ return "LocalStorageCleanupTask{" + "metadataPartitionId=" + metadataPartitionId + '}';
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/UpdateNodeStatusTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/UpdateNodeStatusTask.java
index a1e11c2..fe579ad 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/UpdateNodeStatusTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/UpdateNodeStatusTask.java
@@ -18,25 +18,45 @@
*/
package org.apache.asterix.app.nc.task;
+import java.util.Set;
+
import org.apache.asterix.common.api.INCLifecycleTask;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.storage.IReplicaManager;
import org.apache.hyracks.api.client.NodeStatus;
import org.apache.hyracks.api.control.CcId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.service.IControllerService;
import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class UpdateNodeStatusTask implements INCLifecycleTask {
- private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = LogManager.getLogger();
+ private static final long serialVersionUID = 2L;
private final NodeStatus status;
+ private Set<Integer> activePartitions;
- public UpdateNodeStatusTask(NodeStatus status) {
+ public UpdateNodeStatusTask(NodeStatus status, Set<Integer> activePartitions) {
this.status = status;
+ this.activePartitions = activePartitions;
}
@Override
- public void perform(CcId ccId, IControllerService cs) {
+ public void perform(CcId ccId, IControllerService cs) throws HyracksDataException {
NodeControllerService ncs = (NodeControllerService) cs;
ncs.setNodeStatus(status);
+ if (status != NodeStatus.ACTIVE) {
+ updateNodeActivePartitions(cs);
+ }
+ }
+
+ private void updateNodeActivePartitions(IControllerService cs) {
+ INcApplicationContext appCtx = (INcApplicationContext) cs.getApplicationContext();
+ IReplicaManager replicaManager = appCtx.getReplicaManager();
+ LOGGER.info("updating node active partitions to {}", activePartitions);
+ replicaManager.setActivePartitions(activePartitions);
}
@Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
index c4e4f82..22a0a84 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NcLifecycleCoordinator.java
@@ -26,19 +26,18 @@
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.stream.Collectors;
import org.apache.asterix.app.nc.task.BindMetadataNodeTask;
import org.apache.asterix.app.nc.task.CheckpointTask;
import org.apache.asterix.app.nc.task.ExportMetadataNodeTask;
import org.apache.asterix.app.nc.task.LocalRecoveryTask;
+import org.apache.asterix.app.nc.task.LocalStorageCleanupTask;
import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
import org.apache.asterix.app.nc.task.RetrieveLibrariesTask;
import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
@@ -51,7 +50,6 @@
import org.apache.asterix.app.replication.message.RegistrationTasksResponseMessage;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
@@ -102,7 +100,7 @@
@Override
public void notifyNodeFailure(String nodeId, InetSocketAddress replicaAddress) throws HyracksDataException {
pendingStartupCompletionNodes.remove(nodeId);
- clusterManager.updateNodeState(nodeId, false, null);
+ clusterManager.updateNodeState(nodeId, false, null, null);
if (nodeId.equals(metadataNodeId)) {
clusterManager.updateMetadataNode(metadataNodeId, false);
}
@@ -138,7 +136,8 @@
private void process(RegistrationTasksRequestMessage msg) throws HyracksDataException {
final String nodeId = msg.getNodeId();
nodeSecretsMap.put(nodeId, msg.getSecrets());
- List<INCLifecycleTask> tasks = buildNCRegTasks(msg.getNodeId(), msg.getNodeStatus(), msg.getState());
+ List<INCLifecycleTask> tasks =
+ buildNCRegTasks(msg.getNodeId(), msg.getNodeStatus(), msg.getState(), msg.getActivePartitions());
RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeId, tasks);
try {
messageBroker.sendApplicationMessageToNC(response, msg.getNodeId());
@@ -157,7 +156,7 @@
return;
}
if (msg.isSuccess()) {
- clusterManager.updateNodeState(msg.getNodeId(), true, msg.getLocalCounters());
+ clusterManager.updateNodeState(msg.getNodeId(), true, msg.getLocalCounters(), msg.getActivePartitions());
if (msg.getNodeId().equals(metadataNodeId)) {
clusterManager.updateMetadataNode(metadataNodeId, true);
}
@@ -167,7 +166,8 @@
}
}
- protected List<INCLifecycleTask> buildNCRegTasks(String nodeId, NodeStatus nodeStatus, SystemState state) {
+ protected List<INCLifecycleTask> buildNCRegTasks(String nodeId, NodeStatus nodeStatus, SystemState state,
+ Set<Integer> activePartitions) {
LOGGER.info("Building registration tasks for node {} with status {} and system state: {}", nodeId, nodeStatus,
state);
final boolean isMetadataNode = nodeId.equals(metadataNodeId);
@@ -175,7 +175,7 @@
case ACTIVE:
return buildActiveNCRegTasks(isMetadataNode);
case IDLE:
- return buildIdleNcRegTasks(nodeId, isMetadataNode, state);
+ return buildIdleNcRegTasks(nodeId, isMetadataNode, state, activePartitions);
default:
return new ArrayList<>();
}
@@ -210,20 +210,23 @@
}
}
- protected List<INCLifecycleTask> buildIdleNcRegTasks(String newNodeId, boolean metadataNode, SystemState state) {
+ protected List<INCLifecycleTask> buildIdleNcRegTasks(String newNodeId, boolean metadataNode, SystemState state,
+ Set<Integer> activePartitions) {
final List<INCLifecycleTask> tasks = new ArrayList<>();
- tasks.add(new UpdateNodeStatusTask(NodeStatus.BOOTING));
+ Set<Integer> nodeActivePartitions = getNodeActivePartitions(newNodeId, activePartitions, metadataNode);
+ tasks.add(new UpdateNodeStatusTask(NodeStatus.BOOTING, nodeActivePartitions));
+ int metadataPartitionId = clusterManager.getMetadataPartition().getPartitionId();
+ tasks.add(new LocalStorageCleanupTask(metadataPartitionId));
if (state == SystemState.CORRUPTED) {
- // need to perform local recovery for node partitions
- LocalRecoveryTask rt = new LocalRecoveryTask(Arrays.stream(clusterManager.getNodePartitions(newNodeId))
- .map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
+ // need to perform local recovery for node active partitions
+ LocalRecoveryTask rt = new LocalRecoveryTask(nodeActivePartitions);
tasks.add(rt);
}
if (replicationEnabled) {
tasks.add(new StartReplicationServiceTask());
}
if (metadataNode) {
- tasks.add(new MetadataBootstrapTask(clusterManager.getMetadataPartition().getPartitionId()));
+ tasks.add(new MetadataBootstrapTask(metadataPartitionId));
}
tasks.add(new CheckpointTask());
tasks.add(new StartLifecycleComponentsTask());
@@ -241,7 +244,7 @@
tasks.add(new ExportMetadataNodeTask(true));
tasks.add(new BindMetadataNodeTask());
}
- tasks.add(new UpdateNodeStatusTask(NodeStatus.ACTIVE));
+ tasks.add(new UpdateNodeStatusTask(NodeStatus.ACTIVE, nodeActivePartitions));
return tasks;
}
@@ -298,6 +301,13 @@
return true;
}
+ protected Set<Integer> getNodeActivePartitions(String nodeId, Set<Integer> nodePartitions, boolean metadataNode) {
+ if (metadataNode) {
+ nodePartitions.add(clusterManager.getMetadataPartition().getPartitionId());
+ }
+ return nodePartitions;
+ }
+
private void notifyFailedReplica(IClusterStateManager clusterManager, String nodeID,
InetSocketAddress replicaAddress) {
LOGGER.info("notify replica failure of nodeId {} at {}", nodeID, replicaAddress);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
index 79fa7c8..1309369 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/NCLifecycleTaskReportMessage.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.replication.message;
+import java.util.Set;
+
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
import org.apache.asterix.common.replication.INCLifecycleMessage;
@@ -31,11 +33,14 @@
private final boolean success;
private Throwable exception;
private final NcLocalCounters localCounters;
+ private final Set<Integer> activePartitions;
- public NCLifecycleTaskReportMessage(String nodeId, boolean success, NcLocalCounters localCounters) {
+ public NCLifecycleTaskReportMessage(String nodeId, boolean success, NcLocalCounters localCounters,
+ Set<Integer> activePartitions) {
this.nodeId = nodeId;
this.success = success;
this.localCounters = localCounters;
+ this.activePartitions = activePartitions;
}
@Override
@@ -67,4 +72,8 @@
public MessageType getType() {
return MessageType.REGISTRATION_TASKS_RESULT;
}
+
+ public Set<Integer> getActivePartitions() {
+ return activePartitions;
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
index c2cc63c..fb50b3e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksRequestMessage.java
@@ -20,6 +20,7 @@
import java.util.HashMap;
import java.util.Map;
+import java.util.Set;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
@@ -42,20 +43,22 @@
protected final String nodeId;
protected final NodeStatus nodeStatus;
protected final Map<String, Object> secrets;
+ protected final Set<Integer> activePartitions;
public RegistrationTasksRequestMessage(String nodeId, NodeStatus nodeStatus, SystemState state,
- Map<String, Object> secretsEphemeral) {
+ Map<String, Object> secretsEphemeral, Set<Integer> activePartitions) {
this.state = state;
this.nodeId = nodeId;
this.nodeStatus = nodeStatus;
this.secrets = new HashMap<>(secretsEphemeral);
+ this.activePartitions = activePartitions;
}
public static void send(CcId ccId, NodeControllerService cs, NodeStatus nodeStatus, SystemState systemState,
- Map<String, Object> secretsEphemeral) throws HyracksDataException {
+ Map<String, Object> secretsEphemeral, Set<Integer> activePartitions) throws HyracksDataException {
try {
- RegistrationTasksRequestMessage msg =
- new RegistrationTasksRequestMessage(cs.getId(), nodeStatus, systemState, secretsEphemeral);
+ RegistrationTasksRequestMessage msg = new RegistrationTasksRequestMessage(cs.getId(), nodeStatus,
+ systemState, secretsEphemeral, activePartitions);
((INCMessageBroker) cs.getContext().getMessageBroker()).sendMessageToCC(ccId, msg);
} catch (Exception e) {
LOGGER.log(Level.ERROR, "Unable to send RegistrationTasksRequestMessage to CC", e);
@@ -88,4 +91,8 @@
public Map<String, Object> getSecrets() {
return secrets;
}
+
+ public Set<Integer> getActivePartitions() {
+ return activePartitions;
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
index 1227a6f..f0a4a7c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/RegistrationTasksResponseMessage.java
@@ -18,7 +18,9 @@
*/
package org.apache.asterix.app.replication.message;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import org.apache.asterix.common.api.INCLifecycleTask;
import org.apache.asterix.common.api.INcApplicationContext;
@@ -72,7 +74,10 @@
}
NcLocalCounters localCounter = success ? NcLocalCounters.collect(getCcId(),
(NodeControllerService) appCtx.getServiceContext().getControllerService()) : null;
- NCLifecycleTaskReportMessage result = new NCLifecycleTaskReportMessage(nodeId, success, localCounter);
+ // wrap the returned partitions in a hash set to make it serializable
+ Set<Integer> nodeActivePartitions = new HashSet<>(appCtx.getReplicaManager().getPartitions());
+ NCLifecycleTaskReportMessage result =
+ new NCLifecycleTaskReportMessage(nodeId, success, localCounter, nodeActivePartitions);
result.setException(exception);
try {
broker.sendMessageToCC(getCcId(), result);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractResultsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractResultsPrinter.java
new file mode 100644
index 0000000..621fde1
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractResultsPrinter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.app.result.fields;
+
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionOutput;
+
+abstract class AbstractResultsPrinter implements IResponseFieldPrinter {
+ public static final String FIELD_NAME = "results";
+
+ protected final IApplicationContext appCtx;
+ protected final IStatementExecutor.Stats stats;
+ protected final SessionOutput sessionOutput;
+
+ AbstractResultsPrinter(IApplicationContext appCtx, IStatementExecutor.Stats stats, SessionOutput sessionOutput) {
+ this.appCtx = appCtx;
+ this.stats = stats;
+ this.sessionOutput = sessionOutput;
+ }
+
+ @Override
+ public String getName() {
+ return FIELD_NAME;
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
index ccf970c..d98472e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
@@ -26,7 +26,6 @@
import org.apache.asterix.app.result.ResponsePrinter;
import org.apache.asterix.app.result.ResultReader;
import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.common.api.IResponseFieldPrinter;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.translator.IStatementExecutor;
import org.apache.asterix.translator.SessionOutput;
@@ -36,24 +35,19 @@
import org.apache.hyracks.api.result.IResultSet;
import org.apache.hyracks.api.result.ResultSetId;
-public class NcResultPrinter implements IResponseFieldPrinter {
+public class NcResultPrinter extends AbstractResultsPrinter {
private final IStatementExecutor.ResultDelivery delivery;
private final ExecuteStatementResponseMessage responseMsg;
- private final IApplicationContext appCtx;
private final IResultSet resultSet;
- private final SessionOutput sessionOutput;
- private final IStatementExecutor.Stats stats;
public NcResultPrinter(IApplicationContext appCtx, ExecuteStatementResponseMessage responseMsg,
IResultSet resultSet, IStatementExecutor.ResultDelivery delivery, SessionOutput sessionOutput,
IStatementExecutor.Stats stats) {
- this.appCtx = appCtx;
+ super(appCtx, stats, sessionOutput);
this.responseMsg = responseMsg;
this.delivery = delivery;
this.resultSet = resultSet;
- this.sessionOutput = sessionOutput;
- this.stats = stats;
}
@Override
@@ -73,9 +67,4 @@
pw.append(responseMsg.getResult());
}
}
-
- @Override
- public String getName() {
- return ResultsPrinter.FIELD_NAME;
- }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
index 52198de..ee2d7d5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
@@ -23,37 +23,25 @@
import org.apache.asterix.api.http.server.ResultUtil;
import org.apache.asterix.app.result.ResultReader;
import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.common.api.IResponseFieldPrinter;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.translator.IStatementExecutor;
import org.apache.asterix.translator.SessionOutput;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-public class ResultsPrinter implements IResponseFieldPrinter {
+public class ResultsPrinter extends AbstractResultsPrinter {
- public static final String FIELD_NAME = "results";
- private final IApplicationContext appCtx;
private final ARecordType recordType;
private final ResultReader resultReader;
- private final IStatementExecutor.Stats stats;
- private final SessionOutput sessionOutput;
public ResultsPrinter(IApplicationContext appCtx, ResultReader resultReader, ARecordType recordType,
IStatementExecutor.Stats stats, SessionOutput sessionOutput) {
- this.appCtx = appCtx;
+ super(appCtx, stats, sessionOutput);
this.recordType = recordType;
this.resultReader = resultReader;
- this.stats = stats;
- this.sessionOutput = sessionOutput;
}
@Override
public void print(PrintWriter pw) throws HyracksDataException {
ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, recordType);
}
-
- @Override
- public String getName() {
- return FIELD_NAME;
- }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
index fe9d2be..048584c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
@@ -19,27 +19,147 @@
package org.apache.asterix.app.result.fields;
import java.io.PrintWriter;
+import java.util.LinkedHashSet;
+import java.util.List;
+import org.apache.asterix.api.common.ResultMetadata;
import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
+import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.util.JSONUtil;
+
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
public class SignaturePrinter implements IResponseFieldPrinter {
- public static final SignaturePrinter INSTANCE = new SignaturePrinter();
+ public static final SignaturePrinter INSTANCE = new SignaturePrinter(null);
private static final String FIELD_NAME = "signature";
+ private static final String NAME_FIELD_NAME = "name";
+ private static final String TYPE_FIELD_NAME = "type";
+ private static final String OPTIONAL_MODIFIER = "?";
+
+ private final String signature;
+
+ public SignaturePrinter(String signature) {
+ this.signature = signature;
+ }
@Override
public void print(PrintWriter pw) {
pw.print("\t\"");
pw.print(FIELD_NAME);
- pw.print("\": {\n");
- pw.print("\t");
- ResultUtil.printField(pw, "*", "*", false);
- pw.print("\n\t}");
+ pw.print("\": ");
+ if (signature == null) {
+ pw.print("{\n\t");
+ ResultUtil.printField(pw, "*", "*", false);
+ pw.print("\n\t}");
+ } else {
+ pw.print(signature);
+ }
}
@Override
public String getName() {
return FIELD_NAME;
}
+
+ public static String generateFlatSignature(ResultMetadata resultMetadata) {
+ List<Object> typeInfo = resultMetadata.getOutputTypes();
+ if (typeInfo == null || typeInfo.size() != 1) {
+ return null;
+ }
+ IAType outputType = TypeComputeUtils.getActualType((IAType) typeInfo.get(0));
+ if (outputType.getTypeTag() != ATypeTag.OBJECT) {
+ return null;
+ }
+ ARecordType outputRecordType = (ARecordType) outputType;
+ String[] fieldNames;
+ IAType[] fieldTypes;
+ Pair<String[], IAType[]> p = generateFlatSignatureFromOpenType(outputRecordType);
+ if (p != null) {
+ fieldNames = p.first;
+ fieldTypes = p.second;
+ } else {
+ fieldNames = outputRecordType.getFieldNames();
+ fieldTypes = outputRecordType.getFieldTypes();
+ }
+ if (fieldNames == null || fieldNames.length == 0) {
+ return null;
+ }
+ ObjectNode signatureNode = JSONUtil.createObject();
+ ArrayNode fieldNameArrayNode = signatureNode.putArray(NAME_FIELD_NAME);
+ ArrayNode fieldTypeArrayNode = signatureNode.putArray(TYPE_FIELD_NAME);
+ for (int i = 0, n = fieldNames.length; i < n; i++) {
+ fieldNameArrayNode.add(fieldNames[i]);
+ fieldTypeArrayNode.add(printFieldType(fieldTypes[i]));
+ }
+ return signatureNode.toString();
+ }
+
+ private static Pair<String[], IAType[]> generateFlatSignatureFromOpenType(ARecordType outputRecordType) {
+ if (!outputRecordType.isOpen() || !outputRecordType.knowsAllPossibleAdditonalFieldNames()) {
+ return null;
+ }
+ IRecordTypeAnnotation fieldOrderAnn =
+ outputRecordType.findAnnotation(IRecordTypeAnnotation.Kind.RECORD_FIELD_ORDER);
+ if (fieldOrderAnn == null) {
+ return null;
+ }
+ LinkedHashSet<String> fieldNamesOrdered = ((RecordFieldOrderAnnotation) fieldOrderAnn).getFieldNames();
+ int numFields = fieldNamesOrdered.size();
+
+ String[] fieldNames = new String[numFields];
+ IAType[] fieldTypes = new IAType[numFields];
+ int i = 0;
+ for (String fieldName : fieldNamesOrdered) {
+ IAType fieldType;
+ if (outputRecordType.isClosedField(fieldName)) {
+ fieldType = outputRecordType.getFieldType(fieldName);
+ } else if (outputRecordType.getAllPossibleAdditonalFieldNames().contains(fieldName)) {
+ fieldType = BuiltinType.ANY;
+ } else {
+ return null;
+ }
+ fieldNames[i] = fieldName;
+ fieldTypes[i] = fieldType;
+ i++;
+ }
+ return new Pair<>(fieldNames, fieldTypes);
+ }
+
+ private static String printFieldType(IAType type) {
+ boolean optional = false;
+ if (type.getTypeTag() == ATypeTag.UNION) {
+ AUnionType fieldTypeUnion = (AUnionType) type;
+ optional = fieldTypeUnion.isNullableType() || fieldTypeUnion.isMissableType();
+ type = fieldTypeUnion.getActualType();
+ }
+ String typeName;
+ switch (type.getTypeTag()) {
+ case OBJECT:
+ case ARRAY:
+ case MULTISET:
+ typeName = type.getDisplayName();
+ break;
+ default:
+ typeName = type.getTypeName();
+ break;
+ }
+ return optional ? typeName + OPTIONAL_MODIFIER : typeName;
+ }
+
+ public static SignaturePrinter newInstance(ExecutionPlans executionPlans) {
+ String signature = executionPlans.getSignature();
+ return signature != null ? new SignaturePrinter(signature) : INSTANCE;
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 1d1466c..438397c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -39,9 +39,11 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Optional;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
import org.apache.asterix.active.ActivityState;
@@ -85,6 +87,7 @@
import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.common.functions.ExternalFunctionLanguage;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.common.metadata.IMetadataLockUtil;
import org.apache.asterix.common.utils.JobUtils;
@@ -97,17 +100,15 @@
import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
-import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IQueryRewriter;
import org.apache.asterix.lang.common.base.IReturningStatement;
import org.apache.asterix.lang.common.base.IRewriterFactory;
import org.apache.asterix.lang.common.base.IStatementRewriter;
import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.expression.CallExpr;
import org.apache.asterix.lang.common.expression.IndexedTypeExpression;
-import org.apache.asterix.lang.common.expression.LiteralExpr;
import org.apache.asterix.lang.common.expression.TypeExpression;
import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
-import org.apache.asterix.lang.common.literal.MissingLiteral;
+import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.statement.AdapterDropStatement;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
@@ -121,6 +122,7 @@
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
+import org.apache.asterix.lang.common.statement.CreateViewStatement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -149,10 +151,14 @@
import org.apache.asterix.lang.common.statement.SynonymDropStatement;
import org.apache.asterix.lang.common.statement.TypeDecl;
import org.apache.asterix.lang.common.statement.TypeDropStatement;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.statement.ViewDropStatement;
import org.apache.asterix.lang.common.statement.WriteStatement;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.ViewUtil;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -178,6 +184,7 @@
import org.apache.asterix.metadata.entities.Library;
import org.apache.asterix.metadata.entities.NodeGroup;
import org.apache.asterix.metadata.entities.Synonym;
+import org.apache.asterix.metadata.entities.ViewDetails;
import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
import org.apache.asterix.metadata.functions.ExternalFunctionCompilerUtil;
import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
@@ -188,6 +195,7 @@
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.MetadataUtil;
import org.apache.asterix.metadata.utils.TypeUtil;
+import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -227,6 +235,7 @@
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.data.IAWriterFactory;
import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
@@ -249,6 +258,7 @@
import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+import org.apache.hyracks.util.OptionalBoolean;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -280,6 +290,7 @@
protected final IMetadataLockUtil lockUtil;
protected final IResponsePrinter responsePrinter;
protected final WarningCollector warningCollector;
+ protected final ReentrantReadWriteLock compilationLock;
public QueryTranslator(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
ILangCompilationProvider compilationProvider, ExecutorService executorService,
@@ -298,6 +309,7 @@
this.executorService = executorService;
this.responsePrinter = responsePrinter;
this.warningCollector = new WarningCollector();
+ this.compilationLock = appCtx.getCompilationLock();
if (appCtx.getServiceContext().getAppConfig().getBoolean(CCConfig.Option.ENFORCE_FRAME_WRITER_PROTOCOL)) {
this.jobFlags.add(JobFlag.ENFORCE_CONTRACT);
}
@@ -338,10 +350,8 @@
}
validateOperation(appCtx, activeDataverse, stmt);
MetadataProvider metadataProvider = MetadataProvider.create(appCtx, activeDataverse);
- metadataProvider.getConfig().putAll(config);
- metadataProvider.setWriterFactory(writerFactory);
- metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
- metadataProvider.setOutputFile(outputFile);
+ configureMetadataProvider(metadataProvider, config, resultSerializerFactoryProvider, writerFactory,
+ outputFile, requestParameters, stmt);
IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
rewriteStatement(stmt, stmtRewriter, metadataProvider); // Rewrite the statement's AST.
Statement.Kind kind = stmt.getKind();
@@ -419,6 +429,12 @@
case SYNONYM_DROP:
handleDropSynonymStatement(metadataProvider, stmt);
break;
+ case CREATE_VIEW:
+ handleCreateViewStatement(metadataProvider, stmt, stmtRewriter, requestParameters);
+ break;
+ case VIEW_DROP:
+ handleViewDropStatement(metadataProvider, stmt);
+ break;
case LOAD:
handleLoadStatement(metadataProvider, stmt, hcc);
break;
@@ -431,10 +447,10 @@
metadataProvider.setMaxResultReads(maxResultReads);
}
handleInsertUpsertStatement(metadataProvider, stmt, hcc, resultSet, resultDelivery, outMetadata,
- stats, false, requestParameters, stmtParams, stmtRewriter);
+ stats, requestParameters, stmtParams, stmtRewriter);
break;
case DELETE:
- handleDeleteStatement(metadataProvider, stmt, hcc, false, stmtParams, stmtRewriter);
+ handleDeleteStatement(metadataProvider, stmt, hcc, stmtParams, stmtRewriter);
break;
case CREATE_FEED:
handleCreateFeedStatement(metadataProvider, stmt);
@@ -502,6 +518,27 @@
}
}
+ protected void configureMetadataProvider(MetadataProvider metadataProvider, Map<String, String> config,
+ IResultSerializerFactoryProvider resultSerializerFactoryProvider, IAWriterFactory writerFactory,
+ FileSplit outputFile, IRequestParameters requestParameters, Statement statement) {
+ if (statement.getKind() == Statement.Kind.QUERY && requestParameters.isSQLCompatMode()) {
+ metadataProvider.getConfig().put(SqlppQueryRewriter.SQL_COMPAT_OPTION, Boolean.TRUE.toString());
+ }
+ metadataProvider.getConfig().putAll(config);
+ metadataProvider.setWriterFactory(writerFactory);
+ metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
+ metadataProvider.setOutputFile(outputFile);
+ }
+
+ protected DataverseDecl getRequestDataverseDecl(IRequestParameters requestParameters) throws AlgebricksException {
+ String requestDataverseName = requestParameters.getDefaultDataverseName();
+ if (requestDataverseName == null) {
+ return null;
+ }
+ DataverseName dvName = DataverseName.createFromCanonicalForm(requestDataverseName);
+ return new DataverseDecl(dvName, true);
+ }
+
protected void handleSetStatement(Statement stmt, Map<String, String> config) throws CompilationException {
SetStatement ss = (SetStatement) stmt;
String pname = ss.getPropName();
@@ -545,7 +582,17 @@
DataverseName dvName = stmtUseDataverse.getDataverseName();
Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
if (dv == null) {
- throw new MetadataException(ErrorCode.UNKNOWN_DATAVERSE, stmtUseDataverse.getSourceLocation(), dvName);
+ if (stmtUseDataverse.getIfExists()) {
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(
+ Warning.of(stmtUseDataverse.getSourceLocation(), ErrorCode.UNKNOWN_DATAVERSE, dvName));
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return activeDataverse;
+ } else {
+ throw new MetadataException(ErrorCode.UNKNOWN_DATAVERSE, stmtUseDataverse.getSourceLocation(),
+ dvName);
+ }
}
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
return dv;
@@ -639,50 +686,22 @@
metadataProvider.validateDatabaseObjectName(dd.getDataverse(), datasetName, stmt.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(dd.getDataverse());
TypeExpression itemTypeExpr = dd.getItemType();
- DataverseName itemTypeDataverseName;
- String itemTypeName;
- boolean itemTypeAnonymous;
- switch (itemTypeExpr.getTypeKind()) {
- case TYPEREFERENCE:
- TypeReferenceExpression itemTypeRefExpr = (TypeReferenceExpression) itemTypeExpr;
- Pair<DataverseName, Identifier> itemTypeIdent = itemTypeRefExpr.getIdent();
- itemTypeDataverseName = itemTypeIdent.first != null ? itemTypeIdent.first : dataverseName;
- itemTypeName = itemTypeRefExpr.getIdent().second.getValue();
- itemTypeAnonymous = false;
- break;
- case RECORD:
- itemTypeDataverseName = dataverseName;
- itemTypeName = TypeUtil.createDatasetInlineTypeName(datasetName, false);
- itemTypeAnonymous = true;
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
- String.valueOf(itemTypeExpr.getTypeKind()));
- }
+ Triple<DataverseName, String, Boolean> itemTypeQualifiedName =
+ extractDatasetItemTypeName(dataverseName, datasetName, itemTypeExpr, false, stmt.getSourceLocation());
+ DataverseName itemTypeDataverseName = itemTypeQualifiedName.first;
+ String itemTypeName = itemTypeQualifiedName.second;
+ boolean itemTypeAnonymous = itemTypeQualifiedName.third;
TypeExpression metaItemTypeExpr = dd.getMetaItemType();
DataverseName metaItemTypeDataverseName = null;
String metaItemTypeName = null;
boolean metaItemTypeAnonymous;
if (metaItemTypeExpr != null) {
- switch (metaItemTypeExpr.getTypeKind()) {
- case TYPEREFERENCE:
- TypeReferenceExpression metaItemTypeRefExpr = (TypeReferenceExpression) metaItemTypeExpr;
- Pair<DataverseName, Identifier> metaItemTypeIdent = metaItemTypeRefExpr.getIdent();
- metaItemTypeDataverseName =
- metaItemTypeIdent.first != null ? metaItemTypeIdent.first : dataverseName;
- metaItemTypeName = metaItemTypeRefExpr.getIdent().second.getValue();
- metaItemTypeAnonymous = false;
- break;
- case RECORD:
- metaItemTypeDataverseName = dataverseName;
- metaItemTypeName = TypeUtil.createDatasetInlineTypeName(datasetName, true);
- metaItemTypeAnonymous = true;
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
- String.valueOf(metaItemTypeExpr.getTypeKind()));
- }
+ Triple<DataverseName, String, Boolean> metaItemTypeQualifiedName = extractDatasetItemTypeName(dataverseName,
+ datasetName, metaItemTypeExpr, true, stmt.getSourceLocation());
+ metaItemTypeDataverseName = metaItemTypeQualifiedName.first;
+ metaItemTypeName = metaItemTypeQualifiedName.second;
+ metaItemTypeAnonymous = metaItemTypeQualifiedName.third;
} else {
metaItemTypeAnonymous = true; // doesn't matter
}
@@ -704,7 +723,7 @@
}
}
- protected void doCreateDatasetStatement(MetadataProvider metadataProvider, DatasetDecl dd,
+ protected Optional<? extends Dataset> doCreateDatasetStatement(MetadataProvider metadataProvider, DatasetDecl dd,
DataverseName dataverseName, String datasetName, DataverseName itemTypeDataverseName,
TypeExpression itemTypeExpr, String itemTypeName, TypeExpression metaItemTypeExpr,
DataverseName metaItemTypeDataverseName, String metaItemTypeName, IHyracksClientConnection hcc,
@@ -731,39 +750,26 @@
}
IDatasetDetails datasetDetails;
- Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
+ Dataset ds = metadataProvider.findDataset(dataverseName, datasetName, true);
if (ds != null) {
+ if (ds.getDatasetType() == DatasetType.VIEW) {
+ throw new CompilationException(ErrorCode.VIEW_EXISTS, sourceLoc,
+ DatasetUtil.getFullyQualifiedDisplayName(ds));
+ }
if (dd.getIfNotExists()) {
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
- return;
+ return Optional.empty();
} else {
throw new CompilationException(ErrorCode.DATASET_EXISTS, sourceLoc, datasetName, dataverseName);
}
}
- IAType itemType;
- boolean itemTypeIsInline = false;
- switch (itemTypeExpr.getTypeKind()) {
- case TYPEREFERENCE:
- itemTypeEntity = metadataProvider.findTypeEntity(itemTypeDataverseName, itemTypeName);
- if (itemTypeEntity == null || itemTypeEntity.getIsAnonymous()) {
- // anonymous types cannot be referred from CREATE DATASET
- throw new AsterixException(ErrorCode.UNKNOWN_TYPE, sourceLoc,
- DatasetUtil.getFullyQualifiedDisplayName(itemTypeDataverseName, itemTypeName));
- }
- itemType = itemTypeEntity.getDatatype();
- validateDatasetItemType(dsType, itemType, false, sourceLoc);
- break;
- case RECORD:
- itemType = translateType(itemTypeDataverseName, itemTypeName, itemTypeExpr, mdTxnCtx);
- validateDatasetItemType(dsType, itemType, false, sourceLoc);
- itemTypeEntity = new Datatype(itemTypeDataverseName, itemTypeName, itemType, true);
- itemTypeIsInline = true;
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
- String.valueOf(itemTypeExpr.getTypeKind()));
- }
+ Pair<Datatype, Boolean> itemTypePair = fetchDatasetItemType(mdTxnCtx, dsType, itemTypeDataverseName,
+ itemTypeName, itemTypeExpr, false, metadataProvider, sourceLoc);
+ itemTypeEntity = itemTypePair.first;
+ IAType itemType = itemTypeEntity.getDatatype();
+ boolean itemTypeIsInline = itemTypePair.second;
+
String ngName = ngNameId != null ? ngNameId
: configureNodegroupForDataset(appCtx, dd.getHints(), dataverseName, datasetName, metadataProvider,
sourceLoc);
@@ -780,30 +786,12 @@
switch (dsType) {
case INTERNAL:
if (metaItemTypeExpr != null) {
- switch (metaItemTypeExpr.getTypeKind()) {
- case TYPEREFERENCE:
- metaItemTypeEntity =
- metadataProvider.findTypeEntity(metaItemTypeDataverseName, metaItemTypeName);
- if (metaItemTypeEntity == null || metaItemTypeEntity.getIsAnonymous()) {
- // anonymous types cannot be referred from CREATE DATASET
- throw new AsterixException(ErrorCode.UNKNOWN_TYPE, sourceLoc, DatasetUtil
- .getFullyQualifiedDisplayName(metaItemTypeDataverseName, metaItemTypeName));
- }
- metaItemType = metaItemTypeEntity.getDatatype();
- validateDatasetItemType(dsType, metaItemType, true, sourceLoc);
- break;
- case RECORD:
- metaItemType = translateType(metaItemTypeDataverseName, metaItemTypeName,
- metaItemTypeExpr, mdTxnCtx);
- validateDatasetItemType(dsType, metaItemType, true, sourceLoc);
- metaItemTypeEntity =
- new Datatype(metaItemTypeDataverseName, metaItemTypeName, metaItemType, true);
- metaItemTypeIsInline = true;
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
- String.valueOf(metaItemTypeExpr.getTypeKind()));
- }
+ Pair<Datatype, Boolean> metaItemTypePair =
+ fetchDatasetItemType(mdTxnCtx, dsType, metaItemTypeDataverseName, metaItemTypeName,
+ metaItemTypeExpr, true, metadataProvider, sourceLoc);
+ metaItemTypeEntity = metaItemTypePair.first;
+ metaItemType = metaItemTypeEntity.getDatatype();
+ metaItemTypeIsInline = metaItemTypePair.second;
}
ARecordType metaRecType = (ARecordType) metaItemType;
@@ -948,22 +936,75 @@
}
throw e;
}
+ return Optional.of(dataset);
+ }
+
+ protected Triple<DataverseName, String, Boolean> extractDatasetItemTypeName(DataverseName datasetDataverseName,
+ String datasetName, TypeExpression itemTypeExpr, boolean isMetaItemType, SourceLocation sourceLoc)
+ throws CompilationException {
+ switch (itemTypeExpr.getTypeKind()) {
+ case TYPEREFERENCE:
+ TypeReferenceExpression itemTypeRefExpr = (TypeReferenceExpression) itemTypeExpr;
+ Pair<DataverseName, Identifier> itemTypeIdent = itemTypeRefExpr.getIdent();
+ DataverseName typeDataverseName =
+ itemTypeIdent.first != null ? itemTypeIdent.first : datasetDataverseName;
+ String typeName = itemTypeRefExpr.getIdent().second.getValue();
+ return new Triple<>(typeDataverseName, typeName, false);
+ case RECORD:
+ String inlineTypeName = TypeUtil.createDatasetInlineTypeName(datasetName, isMetaItemType);
+ return new Triple<>(datasetDataverseName, inlineTypeName, true);
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(itemTypeExpr.getTypeKind()));
+ }
+ }
+
+ protected Pair<Datatype, Boolean> fetchDatasetItemType(MetadataTransactionContext mdTxnCtx, DatasetType datasetType,
+ DataverseName itemTypeDataverseName, String itemTypeName, TypeExpression itemTypeExpr,
+ boolean isMetaItemType, MetadataProvider metadataProvider, SourceLocation sourceLoc)
+ throws AlgebricksException {
+ switch (itemTypeExpr.getTypeKind()) {
+ case TYPEREFERENCE:
+ Datatype itemTypeEntity = metadataProvider.findTypeEntity(itemTypeDataverseName, itemTypeName);
+ if (itemTypeEntity == null || itemTypeEntity.getIsAnonymous()) {
+ // anonymous types cannot be referred from CREATE DATASET/VIEW
+ throw new AsterixException(ErrorCode.UNKNOWN_TYPE, sourceLoc,
+ DatasetUtil.getFullyQualifiedDisplayName(itemTypeDataverseName, itemTypeName));
+ }
+ IAType itemType = itemTypeEntity.getDatatype();
+ validateDatasetItemType(datasetType, itemType, isMetaItemType, sourceLoc);
+ return new Pair<>(itemTypeEntity, false);
+ case RECORD:
+ itemType = translateType(itemTypeDataverseName, itemTypeName, itemTypeExpr, mdTxnCtx);
+ validateDatasetItemType(datasetType, itemType, isMetaItemType, sourceLoc);
+ itemTypeEntity = new Datatype(itemTypeDataverseName, itemTypeName, itemType, true);
+ return new Pair<>(itemTypeEntity, true);
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(itemTypeExpr.getTypeKind()));
+ }
}
protected void validateDatasetItemType(DatasetType datasetType, IAType itemType, boolean isMetaItemType,
SourceLocation sourceLoc) throws AlgebricksException {
if (itemType.getTypeTag() != ATypeTag.OBJECT) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- String.format(StringUtils.capitalize(dataset()) + " %s has to be a record type.",
+ String.format("%s %s has to be a record type.",
+ datasetType == DatasetType.VIEW ? "view" : StringUtils.capitalize(dataset()),
isMetaItemType ? "meta type" : "type"));
}
+ if (datasetType == DatasetType.VIEW) {
+ ViewUtil.validateViewItemType((ARecordType) itemType, sourceLoc);
+ }
}
protected Map<String, String> createExternalDatasetProperties(DataverseName dataverseName, DatasetDecl dd,
Datatype itemType, MetadataProvider metadataProvider, MetadataTransactionContext mdTxnCtx)
throws AlgebricksException {
ExternalDetailsDecl externalDetails = (ExternalDetailsDecl) dd.getDatasetDetailsDecl();
- return externalDetails.getProperties();
+ Map<String, String> properties = externalDetails.getProperties();
+ ExternalDataUtils.validateType(properties, (ARecordType) itemType.getDatatype());
+ return properties;
}
protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset,
@@ -1172,13 +1213,21 @@
}
}
+ boolean isFieldFromSchema = projectTypePrime != null;
IAType fieldTypePrime;
boolean fieldTypeNullable, fieldTypeMissable;
if (projectTypeExpr == null) {
+ // the type of the indexed field is NOT specified in the DDL
+ if (stmtCreateIndex.hasCastDefaultNull()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ stmtCreateIndex.getSourceLocation(),
+ "CAST modifier is used without specifying " + "the type of the indexed field");
+ }
fieldTypePrime = projectTypePrime;
fieldTypeNullable = projectTypeNullable;
fieldTypeMissable = projectTypeMissable;
} else {
+ // the type of the indexed field is explicitly specified in the DDL
if (stmtCreateIndex.isEnforced()) {
if (!projectTypeExpr.isUnknownable()) {
throw new CompilationException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL,
@@ -1186,7 +1235,7 @@
}
// don't allow creating an enforced index on a closed-type field, fields that
// are part of schema get the field type, if it's not null, then the field is closed-type
- if (projectTypePrime != null) {
+ if (isFieldFromSchema) {
throw new CompilationException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD,
indexedElement.getSourceLocation(), String.valueOf(projectPath));
}
@@ -1195,10 +1244,13 @@
throw new CompilationException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED,
indexedElement.getSourceLocation(), indexType);
}
- if (projectTypePrime != null) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR,
- indexedElement.getSourceLocation(), "Typed index on \"" + projectPath
- + "\" field could be created only for open datatype");
+ if (isFieldFromSchema) {
+ // allow overriding the type of the closed-field only if CAST modifier is used
+ if (!stmtCreateIndex.hasCastDefaultNull()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ indexedElement.getSourceLocation(), "Typed index on \"" + projectPath
+ + "\" field could be created only for open datatype");
+ }
}
}
@@ -1231,8 +1283,30 @@
indexFieldTypes.add(fieldTypes);
}
+ boolean unknownKeyOptionAllowed =
+ (indexType == IndexType.BTREE || indexType == IndexType.ARRAY) && !isSecondaryPrimary;
+ if (stmtCreateIndex.hasExcludeUnknownKey() && !unknownKeyOptionAllowed) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "can only specify exclude/include unknown key for B-Tree & Array indexes");
+ }
+ boolean castDefaultNullAllowed = indexType == IndexType.BTREE && !isSecondaryPrimary;
+ if (stmtCreateIndex.hasCastDefaultNull() && !castDefaultNullAllowed) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "CAST modifier is only allowed for B-Tree indexes");
+ }
+ if (stmtCreateIndex.getCastDefaultNull().getOrElse(false)) {
+ if (stmtCreateIndex.isEnforced()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "CAST modifier cannot be specified together with ENFORCED");
+ }
+ }
Index.IIndexDetails indexDetails;
if (Index.IndexCategory.of(indexType) == Index.IndexCategory.ARRAY) {
+ if (!stmtCreateIndex.hasExcludeUnknownKey()
+ || !stmtCreateIndex.getExcludeUnknownKey().getOrElse(false)) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "Array indexes must specify EXCLUDE UNKNOWN KEY.");
+ }
if (!hadUnnest) {
// prohibited by the grammar
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
@@ -1243,11 +1317,6 @@
throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
String.valueOf(indexType));
}
- if (indexedElementsCount > 1) {
- // TODO (GLENN): Add in support for composite atomic / array indexes.
- throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
- String.valueOf(indexType));
- }
List<Index.ArrayIndexElement> indexElementList = new ArrayList<>(indexedElementsCount);
for (int i = 0; i < indexedElementsCount; i++) {
@@ -1276,8 +1345,14 @@
}
switch (Index.IndexCategory.of(indexType)) {
case VALUE:
+ Map<String, String> castConfig = TypeUtil.validateConfiguration(stmtCreateIndex.getCastConfig(),
+ stmtCreateIndex.getSourceLocation());
+ String datetimeFormat = TypeUtil.getDatetimeFormat(castConfig);
+ String dateFormat = TypeUtil.getDateFormat(castConfig);
+ String timeFormat = TypeUtil.getTimeFormat(castConfig);
indexDetails = new Index.ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators,
- keyFieldTypes, overridesFieldTypes);
+ keyFieldTypes, overridesFieldTypes, stmtCreateIndex.getExcludeUnknownKey(),
+ stmtCreateIndex.getCastDefaultNull(), datetimeFormat, dateFormat, timeFormat);
break;
case TEXT:
indexDetails = new Index.TextIndexDetails(keyFieldNames, keyFieldSourceIndicators,
@@ -1485,10 +1560,19 @@
// Get snapshot from External File System
externalFilesSnapshot = ExternalIndexingOperations.getSnapshotFromExternalFileSystem(ds);
// Add an entry for the files index
+ OptionalBoolean excludeUnknownKey =
+ ((Index.ValueIndexDetails) index.getIndexDetails()).getExcludeUnknownKey();
+ OptionalBoolean castDefaultNull =
+ ((Index.ValueIndexDetails) index.getIndexDetails()).getCastDefaultNull();
+ String datetimeFormat = ((Index.ValueIndexDetails) index.getIndexDetails()).getCastDatetimeFormat();
+ String dateFormat = ((Index.ValueIndexDetails) index.getIndexDetails()).getCastDateFormat();
+ String timeFormat = ((Index.ValueIndexDetails) index.getIndexDetails()).getCastTimeFormat();
+
filesIndex = new Index(index.getDataverseName(), index.getDatasetName(),
IndexingConstants.getFilesIndexName(index.getDatasetName()), IndexType.BTREE,
new Index.ValueIndexDetails(ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
- ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false),
+ ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false, excludeUnknownKey,
+ castDefaultNull, datetimeFormat, dateFormat, timeFormat),
false, false, MetadataUtil.PENDING_ADD_OP);
MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
// Add files to the external files index
@@ -1844,23 +1928,31 @@
List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverseName);
for (Dataset dataset : datasets) {
String datasetName = dataset.getDatasetName();
- List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
DatasetType dsType = dataset.getDatasetType();
- if (dsType == DatasetType.INTERNAL) {
- for (Index index : indexes) {
- jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset, sourceLoc));
- }
- } else if (dsType == DatasetType.EXTERNAL) {
- for (Index index : indexes) {
- if (ExternalIndexingOperations.isFileIndex(index)) {
- jobsToExecute.add(
- ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, dataset));
- } else {
+ switch (dsType) {
+ case INTERNAL:
+ List<Index> indexes =
+ MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ for (Index index : indexes) {
jobsToExecute
.add(IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset, sourceLoc));
}
- }
- externalDatasetsToDeregister.add(dataset);
+ break;
+ case EXTERNAL:
+ indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+ for (Index index : indexes) {
+ if (ExternalIndexingOperations.isFileIndex(index)) {
+ jobsToExecute.add(ExternalIndexingOperations
+ .buildDropFilesIndexJobSpec(metadataProvider, dataset));
+ } else {
+ jobsToExecute.add(
+ IndexUtil.buildDropIndexJobSpec(index, metadataProvider, dataset, sourceLoc));
+ }
+ }
+ externalDatasetsToDeregister.add(dataset);
+ break;
+ case VIEW:
+ break;
}
}
@@ -2009,7 +2101,7 @@
throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
}
}
- ds = metadataProvider.findDataset(dataverseName, datasetName);
+ ds = metadataProvider.findDataset(dataverseName, datasetName, true);
if (ds == null) {
if (ifExists) {
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
@@ -2018,6 +2110,9 @@
throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
dataverseName);
}
+ } else if (ds.getDatasetType() == DatasetType.VIEW) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
+ dataverseName);
}
validateDatasetState(metadataProvider, ds, sourceLoc);
@@ -2412,6 +2507,275 @@
}
}
+ public void handleCreateViewStatement(MetadataProvider metadataProvider, Statement stmt,
+ IStatementRewriter stmtRewriter, IRequestParameters requestParameters) throws Exception {
+ CreateViewStatement cvs = (CreateViewStatement) stmt;
+ String viewName = cvs.getViewName();
+ metadataProvider.validateDatabaseObjectName(cvs.getDataverseName(), viewName, stmt.getSourceLocation());
+ DataverseName dataverseName = getActiveDataverseName(cvs.getDataverseName());
+
+ DataverseName viewItemTypeDataverseName;
+ String viewItemTypeName;
+ boolean viewItemTypeAnonymous;
+ if (cvs.hasItemType()) {
+ Triple<DataverseName, String, Boolean> viewTypeQualifiedName = extractDatasetItemTypeName(dataverseName,
+ viewName, cvs.getItemType(), false, stmt.getSourceLocation());
+ viewItemTypeDataverseName = viewTypeQualifiedName.first;
+ viewItemTypeName = viewTypeQualifiedName.second;
+ viewItemTypeAnonymous = viewTypeQualifiedName.third;
+ } else {
+ viewItemTypeDataverseName = MetadataBuiltinEntities.ANY_OBJECT_DATATYPE.getDataverseName();
+ viewItemTypeName = MetadataBuiltinEntities.ANY_OBJECT_DATATYPE.getDatatypeName();
+ viewItemTypeAnonymous = false;
+ }
+
+ lockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName,
+ viewItemTypeDataverseName, viewItemTypeName, viewItemTypeAnonymous, null, null, false, null, null, true,
+ DatasetType.VIEW, null);
+ try {
+ doCreateView(metadataProvider, cvs, dataverseName, viewName, viewItemTypeDataverseName, viewItemTypeName,
+ stmtRewriter, requestParameters);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ metadataProvider.setDefaultDataverse(activeDataverse);
+ }
+ }
+
+ protected CreateResult doCreateView(MetadataProvider metadataProvider, CreateViewStatement cvs,
+ DataverseName dataverseName, String viewName, DataverseName itemTypeDataverseName, String itemTypeName,
+ IStatementRewriter stmtRewriter, IRequestParameters requestParameters) throws Exception {
+ SourceLocation sourceLoc = cvs.getSourceLocation();
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ if (dv == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+ }
+ Dataset existingDataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, viewName);
+ if (existingDataset != null) {
+ if (DatasetUtil.isNotView(existingDataset)) {
+ throw new CompilationException(ErrorCode.DATASET_EXISTS, sourceLoc,
+ existingDataset.getDatasetName(), existingDataset.getDataverseName());
+ }
+ if (cvs.getIfNotExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return CreateResult.NOOP;
+ } else if (!cvs.getReplaceIfExists()) {
+ throw new CompilationException(ErrorCode.VIEW_EXISTS, sourceLoc,
+ existingDataset.getDatasetFullyQualifiedName());
+ }
+ }
+
+ DatasetFullyQualifiedName viewQualifiedName = new DatasetFullyQualifiedName(dataverseName, viewName);
+
+ Datatype itemTypeEntity = null;
+ boolean itemTypeIsInline = false;
+ CreateViewStatement.KeyDecl primaryKeyDecl = cvs.getPrimaryKeyDecl();
+ List<String> primaryKeyFields = null;
+ List<CreateViewStatement.ForeignKeyDecl> foreignKeyDecls = cvs.getForeignKeyDecls();
+ List<ViewDetails.ForeignKey> foreignKeys = null;
+ String datetimeFormat = null, dateFormat = null, timeFormat = null;
+ if (cvs.hasItemType()) {
+ Pair<Datatype, Boolean> itemTypePair = fetchDatasetItemType(mdTxnCtx, DatasetType.VIEW,
+ itemTypeDataverseName, itemTypeName, cvs.getItemType(), false, metadataProvider, sourceLoc);
+ itemTypeEntity = itemTypePair.first;
+ itemTypeIsInline = itemTypePair.second;
+ ARecordType itemType = (ARecordType) itemTypeEntity.getDatatype();
+ if (primaryKeyDecl != null) {
+ primaryKeyFields = ValidateUtil.validateViewKeyFields(primaryKeyDecl, itemType, false, sourceLoc);
+ }
+ if (foreignKeyDecls != null) {
+ foreignKeys = new ArrayList<>(foreignKeyDecls.size());
+ for (CreateViewStatement.ForeignKeyDecl foreignKeyDecl : foreignKeyDecls) {
+ List<String> foreignKeyFields =
+ ValidateUtil.validateViewKeyFields(foreignKeyDecl, itemType, true, sourceLoc);
+ DataverseName refDataverseName = foreignKeyDecl.getReferencedDataverseName();
+ if (refDataverseName == null) {
+ refDataverseName = dataverseName;
+ } else {
+ Dataverse refDataverse = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, refDataverseName);
+ if (refDataverse == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc,
+ refDataverseName);
+ }
+ }
+ String refDatasetName = foreignKeyDecl.getReferencedDatasetName().getValue();
+ boolean isSelfRef = refDataverseName.equals(dataverseName) && refDatasetName.equals(viewName);
+ DatasetType refDatasetType;
+ DatasetFullyQualifiedName refQualifiedName;
+ List<String> refPrimaryKeyFields;
+ if (isSelfRef) {
+ refDatasetType = DatasetType.VIEW;
+ refQualifiedName = viewQualifiedName;
+ refPrimaryKeyFields = primaryKeyFields;
+ } else {
+ // findDataset() will acquire lock on referenced dataset (view)
+ Dataset refDataset = metadataProvider.findDataset(refDataverseName, refDatasetName, true);
+ if (refDataset == null || DatasetUtil.isNotView(refDataset)) {
+ throw new CompilationException(ErrorCode.UNKNOWN_VIEW, sourceLoc,
+ DatasetUtil.getFullyQualifiedDisplayName(refDataverseName, refDatasetName));
+ }
+ ViewDetails refViewDetails = (ViewDetails) refDataset.getDatasetDetails();
+ refDatasetType = refDataset.getDatasetType();
+ refQualifiedName = new DatasetFullyQualifiedName(refDataverseName, refDatasetName);
+ refPrimaryKeyFields = refViewDetails.getPrimaryKeyFields();
+ }
+
+ if (refPrimaryKeyFields == null) {
+ throw new CompilationException(ErrorCode.INVALID_FOREIGN_KEY_DEFINITION_REF_PK_NOT_FOUND,
+ sourceLoc, DatasetUtil.getDatasetTypeDisplayName(refDatasetType),
+ DatasetUtil.getFullyQualifiedDisplayName(refDataverseName, refDatasetName));
+ } else if (refPrimaryKeyFields.size() != foreignKeyFields.size()) {
+ throw new CompilationException(ErrorCode.INVALID_FOREIGN_KEY_DEFINITION_REF_PK_MISMATCH,
+ sourceLoc, DatasetUtil.getDatasetTypeDisplayName(refDatasetType),
+ DatasetUtil.getFullyQualifiedDisplayName(refDataverseName, refDatasetName));
+ } else if (isSelfRef
+ && !OperatorPropertiesUtil.disjoint(refPrimaryKeyFields, foreignKeyFields)) {
+ throw new CompilationException(ErrorCode.INVALID_FOREIGN_KEY_DEFINITION, sourceLoc);
+ }
+
+ foreignKeys.add(new ViewDetails.ForeignKey(foreignKeyFields, refQualifiedName));
+ }
+ }
+
+ Map<String, String> viewConfig =
+ TypeUtil.validateConfiguration(cvs.getViewConfiguration(), cvs.getSourceLocation());
+ datetimeFormat = TypeUtil.getDatetimeFormat(viewConfig);
+ dateFormat = TypeUtil.getDateFormat(viewConfig);
+ timeFormat = TypeUtil.getTimeFormat(viewConfig);
+
+ } else {
+ if (primaryKeyDecl != null) {
+ throw new CompilationException(ErrorCode.INVALID_PRIMARY_KEY_DEFINITION, cvs.getSourceLocation());
+ }
+ if (foreignKeyDecls != null) {
+ throw new CompilationException(ErrorCode.INVALID_FOREIGN_KEY_DEFINITION, cvs.getSourceLocation());
+ }
+ if (cvs.getViewConfiguration() != null) {
+ throw new CompilationException(ErrorCode.ILLEGAL_SET_PARAMETER, cvs.getSourceLocation(),
+ cvs.getViewConfiguration().keySet().iterator().next());
+ }
+ }
+
+ if (existingDataset != null) {
+ ViewDetails existingViewDetails = (ViewDetails) existingDataset.getDatasetDetails();
+ List<String> existingPrimaryKeyFields = existingViewDetails.getPrimaryKeyFields();
+ // For now don't allow view replacement if existing view has primary keys and they are different
+ // from the new view's primary keys, because there could be another view that references
+ // these primary keys via its foreign keys declaration.
+ // In the future we should relax this check: scan datasets metadata and allow replacement in this case
+ // if there's no view that references this view
+ boolean allowToReplace =
+ existingPrimaryKeyFields == null || existingPrimaryKeyFields.equals(primaryKeyFields);
+ if (!allowToReplace) {
+ throw new CompilationException(ErrorCode.CANNOT_CHANGE_PRIMARY_KEY, cvs.getSourceLocation(),
+ DatasetUtil.getDatasetTypeDisplayName(existingDataset.getDatasetType()),
+ DatasetUtil.getFullyQualifiedDisplayName(existingDataset));
+ }
+ }
+
+ // Check whether the view is usable:
+ // create a view declaration for this function,
+ // and a query body that queries this view:
+ ViewDecl viewDecl = new ViewDecl(viewQualifiedName, cvs.getViewBodyExpression());
+ viewDecl.setSourceLocation(sourceLoc);
+ IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
+ Query wrappedQuery = queryRewriter.createViewAccessorQuery(viewDecl);
+ metadataProvider.setDefaultDataverse(dv);
+ apiFramework.reWriteQuery(declaredFunctions, Collections.singletonList(viewDecl), metadataProvider,
+ wrappedQuery, sessionOutput, false, false, Collections.emptyList(), warningCollector);
+
+ List<List<Triple<DataverseName, String, String>>> dependencies =
+ ViewUtil.getViewDependencies(viewDecl, foreignKeys, queryRewriter);
+
+ ViewDetails viewDetails = new ViewDetails(cvs.getViewBody(), dependencies, cvs.getDefaultNull(),
+ primaryKeyFields, foreignKeys, datetimeFormat, dateFormat, timeFormat);
+
+ Dataset view = new Dataset(dataverseName, viewName, itemTypeDataverseName, itemTypeName,
+ MetadataConstants.METADATA_NODEGROUP_NAME, "", Collections.emptyMap(), viewDetails,
+ Collections.emptyMap(), DatasetType.VIEW, 0, MetadataUtil.PENDING_NO_OP);
+ if (existingDataset == null) {
+ if (itemTypeIsInline) {
+ MetadataManager.INSTANCE.addDatatype(mdTxnCtx, itemTypeEntity);
+ }
+ MetadataManager.INSTANCE.addDataset(mdTxnCtx, view);
+ } else {
+ if (itemTypeIsInline) {
+ MetadataManager.INSTANCE.updateDatatype(mdTxnCtx, itemTypeEntity);
+ }
+ MetadataManager.INSTANCE.updateDataset(mdTxnCtx, view);
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return existingDataset != null ? CreateResult.REPLACED : CreateResult.CREATED;
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ }
+ }
+
+ public void handleViewDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+ ViewDropStatement stmtDrop = (ViewDropStatement) stmt;
+ SourceLocation sourceLoc = stmtDrop.getSourceLocation();
+ String viewName = stmtDrop.getViewName().getValue();
+ metadataProvider.validateDatabaseObjectName(stmtDrop.getDataverseName(), viewName, sourceLoc);
+ DataverseName dataverseName = getActiveDataverseName(stmtDrop.getDataverseName());
+ lockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, viewName);
+ try {
+ doDropView(metadataProvider, stmtDrop, dataverseName, viewName);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
+ }
+ }
+
+ protected boolean doDropView(MetadataProvider metadataProvider, ViewDropStatement stmtViewDrop,
+ DataverseName dataverseName, String viewName) throws Exception {
+ SourceLocation sourceLoc = stmtViewDrop.getSourceLocation();
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ // Check if the dataverse exists
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ if (dv == null) {
+ if (stmtViewDrop.getIfExists()) {
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(stmtViewDrop.getSourceLocation(), ErrorCode.UNKNOWN_DATAVERSE,
+ dataverseName));
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return false;
+ } else {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+ }
+ }
+ Dataset dataset = metadataProvider.findDataset(dataverseName, viewName, true);
+ if (dataset == null) {
+ if (stmtViewDrop.getIfExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return false;
+ } else {
+ throw new CompilationException(ErrorCode.UNKNOWN_VIEW, sourceLoc,
+ DatasetUtil.getFullyQualifiedDisplayName(dataverseName, viewName));
+ }
+ } else if (DatasetUtil.isNotView(dataset)) {
+ throw new CompilationException(ErrorCode.UNKNOWN_VIEW, sourceLoc,
+ DatasetUtil.getFullyQualifiedDisplayName(dataverseName, viewName));
+ }
+ MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, viewName, false);
+ if (TypeUtil.isDatasetInlineTypeName(dataset, dataset.getItemTypeDataverseName(),
+ dataset.getItemTypeName())) {
+ MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, dataset.getItemTypeDataverseName(),
+ dataset.getItemTypeName());
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return true;
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ }
+ }
+
protected void handleDeclareFunctionStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
FunctionDecl fds = (FunctionDecl) stmt;
FunctionSignature signature = fds.getSignature();
@@ -2448,7 +2812,7 @@
}
}
- protected void doCreateFunction(MetadataProvider metadataProvider, CreateFunctionStatement cfs,
+ protected CreateResult doCreateFunction(MetadataProvider metadataProvider, CreateFunctionStatement cfs,
FunctionSignature functionSignature, IStatementRewriter stmtRewriter, IRequestParameters requestParameters)
throws Exception {
DataverseName dataverseName = functionSignature.getDataverseName();
@@ -2465,7 +2829,7 @@
if (existingFunction != null) {
if (cfs.getIfNotExists()) {
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
- return;
+ return CreateResult.NOOP;
} else if (!cfs.getReplaceIfExists()) {
throw new CompilationException(ErrorCode.FUNCTION_EXISTS, cfs.getSourceLocation(),
functionSignature.toString(false));
@@ -2475,6 +2839,7 @@
existingInlineTypes = Collections.emptyList();
}
+ IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
Map<TypeSignature, Datatype> newInlineTypes;
Function function;
if (cfs.isExternal()) {
@@ -2514,7 +2879,7 @@
newInlineTypes.put(paramTypeSignature, paramInlineTypeEntity);
}
VarIdentifier paramName = paramPair.getFirst();
- paramNames.add(stmtRewriter.toFunctionParameterName(paramName));
+ paramNames.add(queryRewriter.toFunctionParameterName(paramName));
}
TypeSignature returnTypeSignature;
@@ -2569,7 +2934,7 @@
for (Pair<VarIdentifier, TypeExpression> paramPair : paramList) {
VarIdentifier paramName = paramPair.getFirst();
paramVars.add(paramName);
- paramNames.add(stmtRewriter.toFunctionParameterName(paramName));
+ paramNames.add(queryRewriter.toFunctionParameterName(paramName));
if (paramPair.getSecond() != null) {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
paramName.toString());
@@ -2581,25 +2946,17 @@
// and a query body calls this function with each argument set to 'missing'
FunctionDecl fd = new FunctionDecl(functionSignature, paramVars, cfs.getFunctionBodyExpression(), true);
fd.setSourceLocation(sourceLoc);
- CallExpr fcall = new CallExpr(functionSignature,
- Collections.nCopies(paramVars.size(), new LiteralExpr(MissingLiteral.INSTANCE)));
- fcall.setSourceLocation(sourceLoc);
- metadataProvider.setDefaultDataverse(dv);
- Query wrappedQuery = new Query(false);
- wrappedQuery.setSourceLocation(sourceLoc);
- wrappedQuery.setBody(fcall);
- wrappedQuery.setTopLevel(false);
- List<FunctionDecl> fdList = new ArrayList<>(declaredFunctions);
+
+ Query wrappedQuery = queryRewriter.createFunctionAccessorQuery(fd);
+ List<FunctionDecl> fdList = new ArrayList<>(declaredFunctions.size() + 1);
+ fdList.addAll(declaredFunctions);
fdList.add(fd);
- apiFramework.reWriteQuery(fdList, metadataProvider, wrappedQuery, sessionOutput, false, false,
+ metadataProvider.setDefaultDataverse(dv);
+ apiFramework.reWriteQuery(fdList, null, metadataProvider, wrappedQuery, sessionOutput, false, false,
Collections.emptyList(), warningCollector);
- Expression fdNormBody = fd.getNormalizedFuncBody();
- if (fdNormBody == null) {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
- functionSignature.toString());
- }
+
List<List<Triple<DataverseName, String, String>>> dependencies =
- FunctionUtil.getFunctionDependencies(rewriterFactory.createQueryRewriter(), fdNormBody);
+ FunctionUtil.getFunctionDependencies(fd, queryRewriter);
newInlineTypes = Collections.emptyMap();
function = new Function(functionSignature, paramNames, null, null, cfs.getFunctionBody(),
@@ -2630,10 +2987,11 @@
}
MetadataManager.INSTANCE.updateFunction(mdTxnCtx, function);
}
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
if (LOGGER.isInfoEnabled()) {
LOGGER.info("Installed function: " + functionSignature);
}
- MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return existingFunction != null ? CreateResult.REPLACED : CreateResult.CREATED;
} catch (Exception e) {
abort(e, e, mdTxnCtx);
throw e;
@@ -2879,8 +3237,8 @@
}
}
- protected void doCreateLibrary(MetadataProvider metadataProvider, DataverseName dataverseName, String libraryName,
- String libraryHash, CreateLibraryStatement cls, IHyracksClientConnection hcc,
+ protected CreateResult doCreateLibrary(MetadataProvider metadataProvider, DataverseName dataverseName,
+ String libraryName, String libraryHash, CreateLibraryStatement cls, IHyracksClientConnection hcc,
IRequestParameters requestParameters) throws Exception {
JobUtils.ProgressState progress = ProgressState.NO_PROGRESS;
boolean prepareJobSuccessful = false;
@@ -2937,6 +3295,7 @@
MetadataManager.INSTANCE.updateLibrary(mdTxnCtx, newLibrary);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return existingLibrary != null ? CreateResult.REPLACED : CreateResult.CREATED;
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
@@ -3100,7 +3459,7 @@
}
}
- protected void doCreateSynonym(MetadataProvider metadataProvider, CreateSynonymStatement css,
+ protected CreateResult doCreateSynonym(MetadataProvider metadataProvider, CreateSynonymStatement css,
DataverseName dataverseName, String synonymName, DataverseName objectDataverseName, String objectName)
throws Exception {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -3119,13 +3478,14 @@
warningCollector
.warn(Warning.of(css.getSourceLocation(), ErrorCode.SYNONYM_EXISTS, synonymName));
}
- return;
+ return CreateResult.NOOP;
}
throw new CompilationException(ErrorCode.SYNONYM_EXISTS, css.getSourceLocation(), synonymName);
}
synonym = new Synonym(dataverseName, synonymName, objectDataverseName, objectName);
MetadataManager.INSTANCE.addSynonym(metadataProvider.getMetadataTxnContext(), synonym);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return CreateResult.CREATED;
} catch (Exception e) {
abort(e, e, mdTxnCtx);
throw e;
@@ -3191,7 +3551,7 @@
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (spec != null) {
+ if (spec != null && sessionConfig.isExecuteQuery()) {
runJob(hcc, spec);
}
} catch (Exception e) {
@@ -3206,7 +3566,7 @@
public JobSpecification handleInsertUpsertStatement(MetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc, IResultSet resultSet, ResultDelivery resultDelivery,
- ResultMetadata outMetadata, Stats stats, boolean compileOnly, IRequestParameters requestParameters,
+ ResultMetadata outMetadata, Stats stats, IRequestParameters requestParameters,
Map<String, IAObject> stmtParams, IStatementRewriter stmtRewriter) throws Exception {
InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
String datasetName = stmtInsertUpsert.getDatasetName();
@@ -3216,12 +3576,14 @@
final IMetadataLocker locker = new IMetadataLocker() {
@Override
public void lock() throws AlgebricksException {
+ compilationLock.readLock().lock();
lockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
}
@Override
public void unlock() {
metadataProvider.getLocks().unlock();
+ compilationLock.readLock().unlock();
}
};
final IStatementCompiler compiler = () -> {
@@ -3231,10 +3593,10 @@
try {
metadataProvider.setWriteTransaction(true);
final JobSpecification jobSpec =
- rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert, stmtParams, stmtRewriter);
+ rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert, stmtParams);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- return jobSpec;
+ return !sessionConfig.isExecuteQuery() ? null : jobSpec;
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
@@ -3242,14 +3604,6 @@
throw e;
}
};
- if (compileOnly) {
- locker.lock();
- try {
- return compiler.compile();
- } finally {
- locker.unlock();
- }
- }
if (stmtInsertUpsert.getReturnExpression() != null) {
deliverResult(hcc, resultSet, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
@@ -3270,8 +3624,8 @@
}
public JobSpecification handleDeleteStatement(MetadataProvider metadataProvider, Statement stmt,
- IHyracksClientConnection hcc, boolean compileOnly, Map<String, IAObject> stmtParams,
- IStatementRewriter stmtRewriter) throws Exception {
+ IHyracksClientConnection hcc, Map<String, IAObject> stmtParams, IStatementRewriter stmtRewriter)
+ throws Exception {
DeleteStatement stmtDelete = (DeleteStatement) stmt;
String datasetName = stmtDelete.getDatasetName();
metadataProvider.validateDatabaseObjectName(stmtDelete.getDataverseName(), datasetName,
@@ -3286,14 +3640,14 @@
CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
datasetName, stmtDelete.getCondition(), stmtDelete.getVarCounter(), stmtDelete.getQuery());
clfrqs.setSourceLocation(stmt.getSourceLocation());
- JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, clfrqs.getQuery(), clfrqs, stmtParams,
- stmtRewriter, null);
+ JobSpecification jobSpec =
+ rewriteCompileQuery(hcc, metadataProvider, clfrqs.getQuery(), clfrqs, stmtParams, null);
afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
- if (jobSpec != null && !compileOnly) {
+ if (jobSpec != null && sessionConfig.isExecuteQuery()) {
runJob(hcc, jobSpec);
}
return jobSpec;
@@ -3310,13 +3664,13 @@
@Override
public JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector,
MetadataProvider metadataProvider, Query query, ICompiledDmlStatement stmt,
- Map<String, IAObject> stmtParams, IStatementRewriter stmtRewriter, IRequestParameters requestParameters)
+ Map<String, IAObject> stmtParams, IRequestParameters requestParameters)
throws AlgebricksException, ACIDException {
- Map<VarIdentifier, IAObject> externalVars = createExternalVariables(stmtParams, stmtRewriter);
+ Map<VarIdentifier, IAObject> externalVars = createExternalVariables(query, stmtParams);
// Query Rewriting (happens under the same ongoing metadata transaction)
- Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
+ Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
// Query Compilation (happens under the same ongoing metadata transaction)
@@ -3326,14 +3680,14 @@
}
private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
- MetadataProvider metadataProvider, InsertStatement insertUpsert, Map<String, IAObject> stmtParams,
- IStatementRewriter stmtRewriter) throws AlgebricksException, ACIDException {
+ MetadataProvider metadataProvider, InsertStatement insertUpsert, Map<String, IAObject> stmtParams)
+ throws AlgebricksException, ACIDException {
SourceLocation sourceLoc = insertUpsert.getSourceLocation();
- Map<VarIdentifier, IAObject> externalVars = createExternalVariables(stmtParams, stmtRewriter);
+ Map<VarIdentifier, IAObject> externalVars = createExternalVariables(insertUpsert, stmtParams);
// Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
- Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
+ Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
metadataProvider, insertUpsert, sessionOutput, true, true, externalVars.keySet(), warningCollector);
InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
@@ -3811,6 +4165,7 @@
final IMetadataLocker locker = new IMetadataLocker() {
@Override
public void lock() {
+ compilationLock.readLock().lock();
}
@Override
@@ -3818,6 +4173,7 @@
metadataProvider.getLocks().unlock();
// release external datasets' locks acquired during compilation of the query
ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
+ compilationLock.readLock().unlock();
}
};
final IStatementCompiler compiler = () -> {
@@ -3825,8 +4181,8 @@
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
try {
- final JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, null, stmtParams,
- stmtRewriter, requestParameters);
+ final JobSpecification jobSpec =
+ rewriteCompileQuery(hcc, metadataProvider, query, null, stmtParams, requestParameters);
// update stats with count of compile-time warnings. needs to be adapted for multi-statement.
stats.updateTotalWarningsCount(warningCollector.getTotalWarningsCount());
afterCompile();
@@ -4333,9 +4689,14 @@
appCtx.getRequestTracker().track(clientRequest);
}
- protected void validateStatements(IRequestParameters requestParameters) throws CompilationException {
+ protected void validateStatements(IRequestParameters requestParameters)
+ throws AlgebricksException, HyracksDataException {
validateStatements(statements, requestParameters.isMultiStatement(),
requestParameters.getStatementCategoryRestrictionMask());
+ DataverseDecl requestDataverseDecl = getRequestDataverseDecl(requestParameters);
+ if (requestDataverseDecl != null) {
+ statements.add(0, requestDataverseDecl);
+ }
}
public static void validateStatements(List<Statement> statements, boolean allowMultiStatement,
@@ -4376,20 +4737,31 @@
return i == 0;
}
- private Map<VarIdentifier, IAObject> createExternalVariables(Map<String, IAObject> stmtParams,
- IStatementRewriter stmtRewriter) {
- if (stmtParams == null || stmtParams.isEmpty()) {
- return Collections.emptyMap();
- }
- Map<VarIdentifier, IAObject> m = new HashMap<>();
- for (Map.Entry<String, IAObject> me : stmtParams.entrySet()) {
- String paramName = me.getKey();
- String extVarName = stmtRewriter.toExternalVariableName(paramName);
- if (extVarName != null) {
- m.put(new VarIdentifier(extVarName), me.getValue());
+ private Map<VarIdentifier, IAObject> createExternalVariables(IReturningStatement stmt,
+ Map<String, IAObject> stmtParams) throws CompilationException {
+ if (sessionConfig.isExecuteQuery()) {
+ if (stmtParams == null || stmtParams.isEmpty()) {
+ return Collections.emptyMap();
}
+ IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
+ Map<VarIdentifier, IAObject> result = new HashMap<>();
+ for (Map.Entry<String, IAObject> me : stmtParams.entrySet()) {
+ result.put(queryRewriter.toExternalVariableName(me.getKey()), me.getValue());
+ }
+ return result;
+ } else {
+ // compile only. extract statement parameters from the statement body and bind to NULL
+ IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
+ Set<VariableExpr> extVars = queryRewriter.getExternalVariables(stmt.getBody());
+ if (extVars.isEmpty()) {
+ return Collections.emptyMap();
+ }
+ Map<VarIdentifier, IAObject> result = new HashMap<>();
+ for (VariableExpr extVar : extVars) {
+ result.put(extVar.getVar(), ANull.NULL);
+ }
+ return result;
}
- return m;
}
protected void validateDatasetState(MetadataProvider metadataProvider, Dataset dataset, SourceLocation sourceLoc)
@@ -4422,4 +4794,11 @@
throws CompilationException {
ExternalDataUtils.validateAdapterSpecificProperties(configuration, srcLoc, warningCollector);
}
+
+ protected enum CreateResult {
+ NOOP,
+ CREATED,
+ REPLACED
+ }
+
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
index 6c8f21c..8bc6b76 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
@@ -25,6 +25,7 @@
import org.apache.asterix.common.api.IRequestReference;
import org.apache.asterix.external.parser.JSONDataParser;
+import org.apache.asterix.external.parser.LosslessADMJSONDataParser;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.om.base.IAObject;
@@ -34,6 +35,7 @@
import org.apache.asterix.translator.IStatementExecutor.StatementProperties;
import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.asterix.translator.ResultProperties;
+import org.apache.asterix.translator.SessionConfig;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.result.IResultSet;
@@ -58,40 +60,44 @@
private final boolean multiStatement;
private final int statementCategoryRestrictionMask;
private final String statement;
+ private final String defaultDataverseName;
private final boolean forceDropDataset;
private final boolean skipAdmissionPolicy;
+ private boolean printSignature;
+ private boolean sqlCompatMode;
public RequestParameters(IRequestReference requestReference, String statement, IResultSet resultSet,
ResultProperties resultProperties, Stats stats, StatementProperties statementProperties,
- IStatementExecutor.ResultMetadata outMetadata, String clientContextId,
+ IStatementExecutor.ResultMetadata outMetadata, String clientContextId, String defaultDataverseName,
Map<String, String> optionalParameters, Map<String, IAObject> statementParameters, boolean multiStatement) {
this(requestReference, statement, resultSet, resultProperties, stats, statementProperties, outMetadata,
- clientContextId, optionalParameters, statementParameters, multiStatement, NO_CATEGORY_RESTRICTION_MASK);
+ clientContextId, defaultDataverseName, optionalParameters, statementParameters, multiStatement,
+ NO_CATEGORY_RESTRICTION_MASK);
}
public RequestParameters(IRequestReference requestReference, String statement, IResultSet resultSet,
ResultProperties resultProperties, Stats stats, StatementProperties statementProperties,
- IStatementExecutor.ResultMetadata outMetadata, String clientContextId,
+ IStatementExecutor.ResultMetadata outMetadata, String clientContextId, String defaultDataverseName,
Map<String, String> optionalParameters, Map<String, IAObject> statementParameters, boolean multiStatement,
int statementCategoryRestrictionMask) {
this(requestReference, statement, resultSet, resultProperties, stats, statementProperties, outMetadata,
- clientContextId, optionalParameters, statementParameters, multiStatement,
+ clientContextId, defaultDataverseName, optionalParameters, statementParameters, multiStatement,
statementCategoryRestrictionMask, false);
}
public RequestParameters(IRequestReference requestReference, String statement, IResultSet resultSet,
ResultProperties resultProperties, Stats stats, StatementProperties statementProperties,
- IStatementExecutor.ResultMetadata outMetadata, String clientContextId,
+ IStatementExecutor.ResultMetadata outMetadata, String clientContextId, String defaultDataverseName,
Map<String, String> optionalParameters, Map<String, IAObject> statementParameters, boolean multiStatement,
int statementCategoryRestrictionMask, boolean forceDropDataset) {
this(requestReference, statement, resultSet, resultProperties, stats, statementProperties, outMetadata,
- clientContextId, optionalParameters, statementParameters, multiStatement,
+ clientContextId, defaultDataverseName, optionalParameters, statementParameters, multiStatement,
statementCategoryRestrictionMask, forceDropDataset, false);
}
public RequestParameters(IRequestReference requestReference, String statement, IResultSet resultSet,
ResultProperties resultProperties, Stats stats, StatementProperties statementProperties,
- IStatementExecutor.ResultMetadata outMetadata, String clientContextId,
+ IStatementExecutor.ResultMetadata outMetadata, String clientContextId, String defaultDataverseName,
Map<String, String> optionalParameters, Map<String, IAObject> statementParameters, boolean multiStatement,
int statementCategoryRestrictionMask, boolean forceDropDataset, boolean skipAdmissionPolicy) {
this.requestReference = requestReference;
@@ -107,6 +113,7 @@
this.multiStatement = multiStatement;
this.statementCategoryRestrictionMask = statementCategoryRestrictionMask;
this.forceDropDataset = forceDropDataset;
+ this.defaultDataverseName = defaultDataverseName;
this.skipAdmissionPolicy = skipAdmissionPolicy;
}
@@ -180,12 +187,35 @@
return requestReference;
}
- public static Map<String, byte[]> serializeParameterValues(Map<String, JsonNode> inParams)
- throws HyracksDataException {
+ @Override
+ public String getDefaultDataverseName() {
+ return defaultDataverseName;
+ }
+
+ public boolean isPrintSignature() {
+ return printSignature;
+ }
+
+ public void setPrintSignature(boolean printSignature) {
+ this.printSignature = printSignature;
+ }
+
+ @Override
+ public boolean isSQLCompatMode() {
+ return sqlCompatMode;
+ }
+
+ public void setSQLCompatMode(boolean sqlCompatMode) {
+ this.sqlCompatMode = sqlCompatMode;
+ }
+
+ public static Map<String, byte[]> serializeParameterValues(Map<String, JsonNode> inParams,
+ SessionConfig.OutputFormat format) throws HyracksDataException {
if (inParams == null || inParams.isEmpty()) {
return null;
}
- JSONDataParser parser = new JSONDataParser(null, null);
+ JSONDataParser parser = format == SessionConfig.OutputFormat.LOSSLESS_ADM_JSON
+ ? new LosslessADMJSONDataParser(null) : new JSONDataParser(null, null);
ByteArrayAccessibleOutputStream buffer = new ByteArrayAccessibleOutputStream();
DataOutputStream bufferDataOutput = new DataOutputStream(buffer);
Map<String, byte[]> m = new HashMap<>();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java
index c76d9b8..98a1820 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ApplicationConfigurator.java
@@ -24,6 +24,7 @@
import java.util.Properties;
import org.apache.asterix.common.config.AsterixProperties;
+import org.apache.asterix.common.config.BuildProperties;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.hyracks.api.config.IConfigManager;
@@ -36,9 +37,11 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-class ApplicationConfigurator {
+public class ApplicationConfigurator {
private static final Logger LOGGER = LogManager.getLogger();
+ public static final String APPLICATION_NAME = "Apache AsterixDB";
+
private ApplicationConfigurator() {
}
@@ -83,4 +86,8 @@
LOGGER.warn("JRE version \"" + javaVersion + "\" is untested");
}
}
+
+ public static String getApplicationVersion(BuildProperties buildProperties) {
+ return buildProperties.getAllProps().get("git.build.version");
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index e1c39a0..e6ef8df 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -23,6 +23,7 @@
import java.util.Collections;
import java.util.List;
import java.util.Set;
+import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import org.apache.asterix.app.message.StorageCleanupRequestMessage;
@@ -43,6 +44,7 @@
import org.apache.asterix.metadata.entities.Dataverse;
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.metadata.utils.ExternalIndexingOperations;
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.hyracks.api.application.ICCServiceContext;
@@ -63,6 +65,7 @@
protected final IHyracksClientConnection hcc;
protected volatile boolean recoveryCompleted;
protected volatile boolean recovering;
+ protected Future<?> recoveryFuture;
public GlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
IStorageComponentProvider componentProvider) {
@@ -97,7 +100,7 @@
* Perform recovery on a different thread to avoid deadlocks in
* {@link org.apache.asterix.common.cluster.IClusterStateManager}
*/
- serviceCtx.getControllerService().getExecutor().submit(() -> {
+ recoveryFuture = serviceCtx.getControllerService().getExecutor().submit(() -> {
try {
recover(appCtx);
} catch (Throwable e) {
@@ -126,6 +129,9 @@
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
recoveryCompleted = true;
recovering = false;
+ synchronized (this) {
+ recoveryFuture = null;
+ }
LOGGER.info("Global Recovery Completed. Refreshing cluster state...");
appCtx.getClusterStateManager().refreshState();
}
@@ -137,7 +143,8 @@
for (Dataverse dataverse : dataverses) {
List<Dataset> dataverseDatasets =
MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverse.getDataverseName());
- dataverseDatasets.stream().mapToInt(Dataset::getDatasetId).forEach(validDatasetIds::add);
+ dataverseDatasets.stream().filter(DatasetUtil::isNotView).mapToInt(Dataset::getDatasetId)
+ .forEach(validDatasetIds::add);
}
ICcApplicationContext ccAppCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
final List<String> ncs = new ArrayList<>(ccAppCtx.getClusterStateManager().getParticipantNodes());
@@ -164,6 +171,12 @@
@Override
public void notifyStateChange(ClusterState newState) {
+ synchronized (this) {
+ if (recovering && newState == ClusterState.UNUSABLE && recoveryFuture != null) {
+ // interrupt the recovery attempt since cluster became unusable during global recovery
+ recoveryFuture.cancel(true);
+ }
+ }
if (newState != ClusterState.ACTIVE && newState != ClusterState.RECOVERING) {
recoveryCompleted = false;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index c148c92..be1cc7c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -35,7 +35,6 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import org.apache.asterix.algebra.base.ILangExtension;
import org.apache.asterix.api.http.server.BasicAuthServlet;
@@ -69,6 +68,8 @@
import org.apache.asterix.common.config.PropertiesFactory;
import org.apache.asterix.common.config.StorageProperties;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.replication.IReplicationStrategyFactory;
+import org.apache.asterix.common.replication.ReplicationStrategyFactory;
import org.apache.asterix.common.transactions.Checkpoint;
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
@@ -80,7 +81,6 @@
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.messaging.MessagingChannelInterfaceFactory;
import org.apache.asterix.messaging.NCMessageBroker;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.asterix.translator.Receptionist;
import org.apache.asterix.util.MetadataBuiltinFunctions;
import org.apache.asterix.utils.RedactionUtil;
@@ -163,7 +163,7 @@
updateOnNodeJoin();
}
runtimeContext.initialize(getRecoveryManagerFactory(), getReceptionistFactory(), getConfigValidatorFactory(),
- runtimeContext.getNodeProperties().isInitialRun());
+ getReplicationStrategyFactory(), runtimeContext.getNodeProperties().isInitialRun());
MessagingProperties messagingProperties = runtimeContext.getMessagingProperties();
NCMessageBroker messageBroker = new NCMessageBroker(controllerService, messagingProperties);
this.ncServiceCtx.setMessageBroker(messageBroker);
@@ -194,6 +194,10 @@
return Receptionist::new;
}
+ protected IReplicationStrategyFactory getReplicationStrategyFactory() {
+ return new ReplicationStrategyFactory();
+ }
+
protected IConfigValidatorFactory getConfigValidatorFactory() {
return ConfigValidator::new;
}
@@ -300,7 +304,8 @@
apiServer != null ? Collections.singletonMap(SYS_AUTH_HEADER, apiServer.ctx().get(SYS_AUTH_HEADER))
: Collections.emptyMap();
RegistrationTasksRequestMessage.send(ccId, (NodeControllerService) ncServiceCtx.getControllerService(),
- currentStatus, systemState, httpSecrets);
+ currentStatus, systemState, httpSecrets,
+ runtimeContext.getMetadataProperties().getNodeActivePartitions(nodeId));
}
@Override
@@ -312,16 +317,7 @@
}
private void performLocalCleanUp() throws HyracksDataException {
- //Delete working area files from failed jobs
runtimeContext.getIoManager().deleteWorkspaceFiles();
- // Reclaim storage for orphaned index artifacts in NCs.
- final Set<Integer> nodePartitions = runtimeContext.getReplicaManager().getPartitions();
- final PersistentLocalResourceRepository localResourceRepository =
- (PersistentLocalResourceRepository) runtimeContext.getLocalResourceRepository();
- localResourceRepository.deleteCorruptedResources();
- for (Integer partition : nodePartitions) {
- localResourceRepository.cleanup(partition);
- }
}
private void updateOnNodeJoin() {
@@ -340,7 +336,11 @@
@Override
public IFileDeviceResolver getFileDeviceResolver() {
return (relPath, devices) -> {
- int ioDeviceIndex = Math.abs(StoragePathUtil.getPartitionNumFromRelativePath(relPath) % devices.size());
+ int partition = StoragePathUtil.getPartitionNumFromRelativePath(relPath);
+ if (partition < 0) {
+ return devices.get(0);
+ }
+ int ioDeviceIndex = Math.abs(partition % devices.size());
return devices.get(ioDeviceIndex);
};
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index 9472da5..dcd52a0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -253,7 +253,7 @@
clfrqs = new CompiledStatements.CompiledUpsertStatement(feedConn.getDataverseName(),
feedConn.getDatasetName(), feedConnQuery, stmtUpsert.getVarCounter(), null, null);
}
- return statementExecutor.rewriteCompileQuery(hcc, metadataProvider, feedConnQuery, clfrqs, null, null, null);
+ return statementExecutor.rewriteCompileQuery(hcc, metadataProvider, feedConnQuery, clfrqs, null, null);
}
private static JobSpecification combineIntakeCollectJobs(MetadataProvider metadataProvider, Feed feed,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index ccb73b6..b0dc162 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -354,7 +354,7 @@
// Drops dataset files of a given dataset.
private static void dropDatasetFiles(Dataset dataset, MetadataProvider metadataProvider,
IHyracksClientConnection hcc) throws Exception {
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL || dataset.getDatasetType() == DatasetType.VIEW) {
return;
}
List<JobSpecification> jobs = new ArrayList<>();
@@ -389,7 +389,7 @@
// Gets the primary key permutation for upserts.
private static int[] getPrimaryKeyPermutationForUpsert(Dataset dataset) {
- // upsertIndicatorVar + prev record
+ // (upsert) operationVar + prev record
int f = 2;
// add the previous meta second
if (dataset.hasMetaPart()) {
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index 51c71c2..98ede89 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -28,7 +28,7 @@
txn.log.dir=target/tmp/asterix_nc2/txnlog
core.dump.dir=target/tmp/asterix_nc2/coredump
iodevices=asterix_nc2/iodevice1
-iodevices=asterix_nc1/iodevice2
+iodevices=asterix_nc2/iodevice2
nc.api.port=19005
#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
diff --git a/asterixdb/asterix-app/src/main/resources/entrypoint.py b/asterixdb/asterix-app/src/main/resources/entrypoint.py
index aba4f29..7bad7ef 100755
--- a/asterixdb/asterix-app/src/main/resources/entrypoint.py
+++ b/asterixdb/asterix-app/src/main/resources/entrypoint.py
@@ -246,11 +246,11 @@
return
pos += read
while pos < self.sz:
- vszchunk = sys.stdin.buffer.read1()
+ vszchunk = sys.stdin.buffer.read1(FRAMESZ)
if len(vszchunk) == 0:
self.alive = False
return
- self.readview = None
+ self.readview.release()
self.readbuf.extend(vszchunk)
self.readview = memoryview(self.readbuf)
pos += len(vszchunk)
@@ -258,7 +258,7 @@
self.unpacked_msg = list(self.unpacker)
self.msg_type = MessageType(self.unpacked_msg[0])
self.type_handler[self.msg_type](self)
- except BaseException as e:
+ except BaseException:
self.handle_error(traceback.format_exc())
def send_msg(self):
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
index 50e1155..68fb9a8 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
@@ -71,7 +71,7 @@
final RequestReference requestReference = RequestReference.of("1", "node1", System.currentTimeMillis());
RequestParameters requestParameters = new RequestParameters(requestReference, "select 1", null, null, null,
- null, null, "1", null, null, true);
+ null, null, "1", null, null, null, true);
ClientRequest request = new ClientRequest(requestParameters);
request.setJobId(new JobId(1));
request.markCancellable();
@@ -88,7 +88,7 @@
// Tests the case that the job cancellation hit some exception from Hyracks.
final RequestReference requestReference2 = RequestReference.of("2", "node1", System.currentTimeMillis());
requestParameters = new RequestParameters(requestReference2, "select 1", null, null, null, null, null, "2",
- null, null, true);
+ null, null, null, true);
ClientRequest request2 = new ClientRequest(requestParameters);
request2.setJobId(new JobId(2));
request2.markCancellable();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index c668fb6..0b80881 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -28,7 +28,6 @@
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.app.nc.TransactionSubsystem;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.config.TransactionProperties;
import org.apache.asterix.common.context.DatasetLifecycleManager;
import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -53,6 +52,7 @@
import org.apache.asterix.metadata.utils.MetadataUtil;
import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.formats.FormatUtils;
import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
@@ -67,6 +67,8 @@
import org.apache.asterix.transaction.management.service.logging.LogReader;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
@@ -440,7 +442,7 @@
storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
BTreeSearchOperatorDescriptor searchOpDesc = new BTreeSearchOperatorDescriptor(spec, primaryIndexInfo.rDesc,
null, null, true, true, indexDataflowHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, filterFields, filterFields, false);
+ NoOpOperationCallbackFactory.INSTANCE, filterFields, filterFields, false, null);
BTreeSearchOperatorNodePushable searchOp =
searchOpDesc.createPushRuntime(ctx, primaryIndexInfo.getSearchRecordDescriptorProvider(),
ctx.getTaskAttemptId().getTaskId().getPartition(), 1);
@@ -718,10 +720,8 @@
indicator == Index.RECORD_INDICATOR ? recordType.getFieldNames() : metaType.getFieldNames();
keyFieldNames.add(Arrays.asList(fieldNames[primaryKeyIndexes[i]]));
}
- index = new Index(dataset.getDataverseName(), dataset.getDatasetName(), dataset.getDatasetName(),
- IndexType.BTREE,
- new Index.ValueIndexDetails(keyFieldNames, primaryKeyIndicators, keyFieldTypes, false), false, true,
- MetadataUtil.PENDING_NO_OP);
+ index = Index.createPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName(), keyFieldNames,
+ primaryKeyIndicators, keyFieldTypes, MetadataUtil.PENDING_NO_OP);
List<String> nodes = Collections.singletonList(ExecutionTestUtil.integrationUtil.ncs[0].getId());
CcApplicationContext appCtx =
(CcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext();
@@ -816,10 +816,10 @@
RecordDescriptor upsertOutRecDesc = getUpsertOutRecDesc(primaryIndexInfo.rDesc, dataset,
filterFields == null ? 0 : filterFields.length, recordType, metaType);
// fix pk fields
- int diff = upsertOutRecDesc.getFieldCount() - primaryIndexInfo.rDesc.getFieldCount();
+ int start = 1 + (dataset.hasMetaPart() ? 2 : 1) + (filterFields == null ? 0 : filterFields.length);
int[] pkFieldsInCommitOp = new int[dataset.getPrimaryKeys().size()];
for (int i = 0; i < pkFieldsInCommitOp.length; i++) {
- pkFieldsInCommitOp[i] = diff + i;
+ pkFieldsInCommitOp[i] = start++;
}
CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(), pkFieldsInCommitOp,
true, ctx.getTaskAttemptId().getTaskId().getPartition(), true);
@@ -830,19 +830,26 @@
private RecordDescriptor getUpsertOutRecDesc(RecordDescriptor inputRecordDesc, Dataset dataset, int numFilterFields,
ARecordType itemType, ARecordType metaItemType) throws Exception {
- ITypeTraits[] outputTypeTraits =
- new ITypeTraits[inputRecordDesc.getFieldCount() + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
- ISerializerDeserializer<?>[] outputSerDes = new ISerializerDeserializer[inputRecordDesc.getFieldCount()
- + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields];
+ // 1 boolean field at the beginning to indicate whether the operation was upsert or delete
+ int numOutFields = 1 + (dataset.hasMetaPart() ? 2 : 1) + numFilterFields + inputRecordDesc.getFieldCount();
+ ITypeTraits[] outputTypeTraits = new ITypeTraits[numOutFields];
+ ISerializerDeserializer<?>[] outputSerDes = new ISerializerDeserializer[numOutFields];
- // add the previous record first
+ ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
+ ITypeTraitProvider typeTraitProvider = FormatUtils.getDefaultFormat().getTypeTraitProvider();
int f = 0;
- outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(itemType);
+ // add the upsert indicator boolean field
+ outputSerDes[f] = serdeProvider.getSerializerDeserializer(BuiltinType.AINT8);
+ outputTypeTraits[f] = typeTraitProvider.getTypeTrait(BuiltinType.AINT8);
+ f++;
+ // add the previous record
+ outputSerDes[f] = serdeProvider.getSerializerDeserializer(itemType);
+ outputTypeTraits[f] = typeTraitProvider.getTypeTrait(itemType);
f++;
// add the previous meta second
if (dataset.hasMetaPart()) {
- outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(metaItemType);
- outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(metaItemType);
+ outputSerDes[f] = serdeProvider.getSerializerDeserializer(metaItemType);
+ outputTypeTraits[f] = typeTraitProvider.getTypeTrait(metaItemType);
f++;
}
// add the previous filter third
@@ -857,10 +864,8 @@
}
}
fieldIdx = i;
- outputTypeTraits[f] = FormatUtils.getDefaultFormat().getTypeTraitProvider()
- .getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
- outputSerDes[f] = FormatUtils.getDefaultFormat().getSerdeProvider()
- .getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
+ outputTypeTraits[f] = typeTraitProvider.getTypeTrait(itemType.getFieldTypes()[fieldIdx]);
+ outputSerDes[f] = serdeProvider.getSerializerDeserializer(itemType.getFieldTypes()[fieldIdx]);
f++;
}
for (int j = 0; j < inputRecordDesc.getFieldCount(); j++) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
index 8969f18..b0de85e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
@@ -210,7 +210,8 @@
exchangeOperator2.setPhysicalOperator(new HashPartitionExchangePOperator(Collections.emptyList(), null));
exchangeOperator2.getInputs().add(new MutableObject<>(groupByOperator));
- LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE));
+ LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+ ConstantExpression.MISSING.getValue());
secondJoin.setExecutionMode(PARTITIONED);
secondJoin.setPhysicalOperator(new NestedLoopJoinPOperator(secondJoin.getJoinKind(),
AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
@@ -270,7 +271,8 @@
order2.setPhysicalOperator(new StableSortPOperator());
order2.getInputs().add(new MutableObject<>(replicateOperator));
- LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE));
+ LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+ ConstantExpression.MISSING.getValue());
secondJoin.setExecutionMode(PARTITIONED);
secondJoin.setPhysicalOperator(new NestedLoopJoinPOperator(secondJoin.getJoinKind(),
AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
index e3e4069..5974e28 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
@@ -97,7 +97,8 @@
sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, true);
sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, true);
sessionConfig.set(SessionConfig.FORMAT_QUOTE_RECORD,
- format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON);
+ format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON
+ && format != SessionConfig.OutputFormat.LOSSLESS_ADM_JSON);
return new SessionOutput(sessionConfig, resultWriter, resultPrefix, resultPostfix, appendHandle, appendStatus);
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
index 9cc295e..b80fa30 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/ClusterStateManagerTest.java
@@ -19,7 +19,9 @@
package org.apache.asterix.runtime;
import java.util.Collections;
+import java.util.HashSet;
import java.util.Map;
+import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
@@ -200,7 +202,8 @@
private void notifyNodeStartupCompletion(CcApplicationContext applicationContext, String nodeId)
throws HyracksDataException {
- NCLifecycleTaskReportMessage msg = new NCLifecycleTaskReportMessage(nodeId, true, mockLocalCounters());
+ NCLifecycleTaskReportMessage msg =
+ new NCLifecycleTaskReportMessage(nodeId, true, mockLocalCounters(), getNodeActivePartitions(nodeId));
applicationContext.getNcLifecycleCoordinator().process(msg);
}
@@ -262,4 +265,20 @@
Mockito.when(localCounters.getMaxTxnId()).thenReturn(1000L);
return localCounters;
}
+
+ private static Set<Integer> getNodeActivePartitions(String nodeId) {
+ Set<Integer> activePartitions = new HashSet<>();
+ switch (nodeId) {
+ case NC1:
+ activePartitions.add(0);
+ break;
+ case NC2:
+ activePartitions.add(1);
+ break;
+ case NC3:
+ activePartitions.add(2);
+ break;
+ }
+ return activePartitions;
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index 3bfea56..c71e602 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -31,9 +31,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.asterix.active.ActivityState;
-import org.apache.asterix.active.CountRetryPolicyFactory;
import org.apache.asterix.active.EntityId;
-import org.apache.asterix.active.InfiniteRetryPolicyFactory;
import org.apache.asterix.active.NoRetryPolicyFactory;
import org.apache.asterix.app.active.ActiveNotificationHandler;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -68,6 +66,7 @@
import org.apache.hyracks.api.job.JobStatus;
import org.apache.hyracks.control.cc.ClusterControllerService;
import org.apache.hyracks.control.cc.application.CCServiceContext;
+import org.apache.hyracks.util.CountRetryPolicy;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -148,7 +147,7 @@
nodeControllers[1] = new TestNodeControllerActor(nodes[1], clusterController);
listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory, entityId,
new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations,
- new InfiniteRetryPolicyFactory());
+ x -> new CountRetryPolicy(1000));
users = new TestUserActor[3];
users[0] = newUser("Till", appCtx);
users[1] = newUser("Mike", appCtx);
@@ -536,8 +535,7 @@
public void testRecoveryFailureAfterOneAttemptCompilationFailure() throws Exception {
handler.unregisterListener(listener);
listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory, entityId,
- new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations,
- new CountRetryPolicyFactory(1));
+ new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations, x -> new CountRetryPolicy(1));
testStartWhenStartSucceed();
WaitForStateSubscriber tempFailSubscriber =
new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
@@ -580,8 +578,7 @@
public void testRecoveryFailureAfterOneAttemptRuntimeFailure() throws Exception {
handler.unregisterListener(listener);
listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory, entityId,
- new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations,
- new CountRetryPolicyFactory(1));
+ new ArrayList<>(allDatasets), statementExecutor, appCtx, hcc, locations, x -> new CountRetryPolicy(1));
testStartWhenStartSucceed();
WaitForStateSubscriber tempFailSubscriber =
new WaitForStateSubscriber(listener, EnumSet.of(ActivityState.TEMPORARILY_FAILED));
@@ -1543,7 +1540,7 @@
AlgebricksAbsolutePartitionConstraint locations = new AlgebricksAbsolutePartitionConstraint(nodes);
additionalListeners[i] = listener = new TestEventsListener(clusterController, nodeControllers, jobIdFactory,
entityId, new ArrayList<>(allDatasets), statementExecutor, ccAppCtx, hcc, locations,
- new InfiniteRetryPolicyFactory());
+ x -> new CountRetryPolicy(1000));
}
Action suspension = users[0].suspendAllActivities(handler);
suspension.sync();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayDataset.java
new file mode 100644
index 0000000..16088b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayDataset.java
@@ -0,0 +1,238 @@
+/*
+ * 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.test.array;
+
+import static org.apache.asterix.test.array.ArrayElement.TableField;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.JsonNode;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.node.ArrayNode;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.node.IntNode;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.node.JsonNodeFactory;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.node.ObjectNode;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.node.TextNode;
+
+public class ArrayDataset {
+ public static final String[] ADDITIONAL_GROUPS = new String[] { "extra_grouping_5_a", "extra_grouping_5_b" };
+ public static final String[] ADDITIONAL_FIELDS = new String[] { "extra_integer_1_a", "extra_integer_1_b" };
+ public static final String[] VALUES_FOR_ADDITIONAL_GROUPS = new String[] { "1", "2", "3", "4", "5" };
+ public static final String[] VALUES_FOR_ADDITIONAL_FIELDS = new String[] { "1", "2" };
+
+ public static final String[] CONTAINED_OBJECT_NAMES = new String[] { "contained_object_1", "contained_object_2" };
+ public static final String[] CONTAINER_OBJECT_NAMES = new String[] { "container_object_A", "container_object_B",
+ "container_object_C", "container_object_D", "container_object_E" };
+
+ private String fromBaseQuery;
+
+ @Override
+ public String toString() {
+ return fromBaseQuery;
+ }
+
+ public static class Builder {
+ private final JsonNodeFactory jsonNodeFactory = new JsonNodeFactory(false);
+ private final List<ArrayElement> builderElements = new ArrayList<>();
+
+ public void addElement(ArrayElement element) {
+ builderElements.add(element);
+ }
+
+ public ArrayDataset build() {
+ ArrayElement e = builderElements.stream().filter(m -> !m.unnestList.isEmpty()).findFirst().orElseThrow();
+ List<TableField> f = builderElements.stream().filter(m -> m.unnestList.isEmpty())
+ .map(m -> m.projectList.get(0)).collect(Collectors.toUnmodifiableList());
+ int arrayNestingLevel = e.unnestList.size();
+
+ // First, build the innermost array component (i.e. the SELECT clause of the DDL, or lack thereof).
+ StringBuilder sb = new StringBuilder();
+ sb.append("WITH group1 AS (\n");
+ sb.append("\tFROM ").append(BaseWisconsinTable.TABLE_NAME).append(" D\n\t");
+ appendGroupByClause(sb, e.unnestList.size());
+ sb.append("\tGROUP AS G1\n");
+ appendSelectClauseWithGroupTerm(sb, "G1", arrayNestingLevel, e.unnestList.get(arrayNestingLevel - 1),
+ buildStartingSelectValue(e));
+ sb.append(")\n");
+
+ // If we have any nested arrays, additional groups.
+ for (int i = 1; i < e.unnestList.size(); i++) {
+ String joinFieldName = BaseWisconsinTable.GROUPING_FIELDS[arrayNestingLevel - 1].fieldName;
+ sb.append(", group").append(i + 1).append(" AS (\n");
+ sb.append("\tFROM ").append(BaseWisconsinTable.TABLE_NAME).append(" D\n\t");
+ sb.append("JOIN group").append(i).append(" G").append(i).append("\n\tON ");
+ sb.append("D.").append(joinFieldName);
+ sb.append(" = G").append(i).append('.').append(joinFieldName).append("\n\t");
+ appendGroupByClause(sb, arrayNestingLevel - 1);
+ sb.append("\tGROUP AS G").append(i + 1).append('\n');
+ appendSelectClauseWithGroupTerm(sb, "G" + (i + 1), arrayNestingLevel - 1,
+ e.unnestList.get(arrayNestingLevel - 2), String.format("%s_inner.G%d", "G" + (i + 1), i));
+ sb.append(")\n");
+ arrayNestingLevel--;
+ }
+
+ // Add the final SELECT clause.
+ String joinFieldName = BaseWisconsinTable.GROUPING_FIELDS[arrayNestingLevel - 1].fieldName;
+ sb.append("FROM ").append(BaseWisconsinTable.TABLE_NAME).append(" D\n");
+ sb.append("JOIN group").append(e.unnestList.size()).append(" G").append(e.unnestList.size());
+ sb.append("\nON D.").append(joinFieldName).append(" = G");
+ sb.append(e.unnestList.size()).append('.').append(joinFieldName).append("\n");
+ appendSelectClauseWithoutGroupTerm(sb, "G" + e.unnestList.size(), arrayNestingLevel - 1,
+ e.unnestList.get(0), f);
+
+ // Return the new array dataset.
+ ArrayDataset arrayDataset = new ArrayDataset();
+ arrayDataset.fromBaseQuery = sb.toString();
+ return arrayDataset;
+ }
+
+ private void appendGroupByClause(StringBuilder sb, int numGroupFields) {
+ sb.append("GROUP BY D.").append(BaseWisconsinTable.GROUPING_FIELDS[0].fieldName).append(" ");
+ for (int i = 1; i < numGroupFields; i++) {
+ sb.append(", D.").append(BaseWisconsinTable.GROUPING_FIELDS[i].fieldName).append(' ');
+ }
+ sb.append('\n');
+ }
+
+ private void appendSelectClauseWithGroupTerm(StringBuilder sb, String groupAlias, int numGroupFields,
+ TableField groupField, String groupValue) {
+ ObjectNode selectClauseNode = new ObjectNode(jsonNodeFactory);
+
+ // Append the GROUP BY fields.
+ for (int i = 0; i < numGroupFields; i++) {
+ selectClauseNode.put(BaseWisconsinTable.GROUPING_FIELDS[i].fieldName,
+ String.format("$D.%s$", BaseWisconsinTable.GROUPING_FIELDS[i].fieldName));
+ }
+
+ // Append two extra groups alongside the group-to-be-added.
+ List<JsonNode> additionalGroupValues = Stream.of(VALUES_FOR_ADDITIONAL_GROUPS)
+ .map(v -> new IntNode(Integer.parseInt(v))).collect(Collectors.toList());
+ for (String additionalGroup : ADDITIONAL_GROUPS) {
+ selectClauseNode.set(additionalGroup, new ArrayNode(jsonNodeFactory, additionalGroupValues));
+ }
+
+ // Create the group field.
+ String q = String.format("$( FROM %s %<s_inner SELECT VALUE %s )$", groupAlias, groupValue);
+ appendNestedFieldsToObjectNode(groupField, selectClauseNode, new TextNode(q));
+
+ // Serialize our object into a SELECT clause.
+ sb.append("\tSELECT VALUE ").append(buildJSONForQuery(selectClauseNode));
+ }
+
+ private void appendSelectClauseWithoutGroupTerm(StringBuilder sb, String groupAlias, int numGroupFields,
+ TableField groupField, List<TableField> auxiliaryFields) {
+ ObjectNode selectClauseNode = new ObjectNode(jsonNodeFactory);
+
+ // Append the GROUP BY fields.
+ for (int i = 0; i < numGroupFields; i++) {
+ selectClauseNode.put(BaseWisconsinTable.GROUPING_FIELDS[i].fieldName,
+ String.format("$D.%s$", BaseWisconsinTable.GROUPING_FIELDS[i].fieldName));
+ }
+
+ // Append two extra groups alongside the group-to-be-added.
+ List<JsonNode> additionalGroupValues = Stream.of(VALUES_FOR_ADDITIONAL_GROUPS)
+ .map(v -> new IntNode(Integer.parseInt(v))).collect(Collectors.toList());
+ for (String additionalGroup : ADDITIONAL_GROUPS) {
+ selectClauseNode.set(additionalGroup, new ArrayNode(jsonNodeFactory, additionalGroupValues));
+ }
+
+ // Add / create our auxiliary objects.
+ for (TableField field : auxiliaryFields) {
+ String v = buildTableFieldValue("D." + field.getSourceField().fieldName, field);
+ appendNestedFieldsToObjectNode(field, selectClauseNode, new TextNode(String.format("$%s$", v)));
+ }
+
+ // Finally, add our group field. This array should already be formed.
+ String q = String.format("$%s.%s$", groupAlias, groupField.getFullFieldName());
+ appendNestedFieldsToObjectNode(groupField, selectClauseNode, new TextNode(q));
+
+ // Serialize our object into a SELECT clause.
+ sb.append("SELECT VALUE ").append(buildJSONForQuery(selectClauseNode));
+ }
+
+ private String buildStartingSelectValue(ArrayElement element) {
+ StringBuilder sb = new StringBuilder();
+ if (element.projectList.isEmpty()) {
+ TableField workingField = element.unnestList.get(element.unnestList.size() - 1);
+ String fieldName = "G1_inner.D." + workingField.getSourceField().fieldName;
+ sb.append(buildTableFieldValue(fieldName, workingField));
+
+ } else {
+ ObjectNode selectValueNode = new ObjectNode(jsonNodeFactory);
+
+ // Append extra fields within the object item. These values will always be fixed.
+ for (int i = 0; i < ADDITIONAL_FIELDS.length; i++) {
+ int additionalFieldValue = Integer.parseInt(VALUES_FOR_ADDITIONAL_FIELDS[i]);
+ selectValueNode.set(ADDITIONAL_FIELDS[i], new IntNode(additionalFieldValue));
+ }
+
+ // Append the items within our object itself.
+ for (TableField workingField : element.projectList) {
+ String fieldName = "G1_inner.D." + workingField.getSourceField().fieldName;
+ String v = buildTableFieldValue(fieldName, workingField);
+ appendNestedFieldsToObjectNode(workingField, selectValueNode,
+ new TextNode(String.format("$%s$", v)));
+ }
+
+ // Serialize our value node.
+ sb.append(buildJSONForQuery(selectValueNode));
+ }
+
+ return sb.toString();
+ }
+
+ private void appendNestedFieldsToObjectNode(TableField field, ObjectNode objectNode, TextNode endpointValue) {
+ ObjectNode workingObject = objectNode;
+ for (int i = 0; i < field.getFieldName().size(); i++) {
+ String fieldPart = field.getFieldName().get(i);
+ if (i < field.getFieldName().size() - 1) {
+ if (workingObject.get(fieldPart) == null) {
+ ObjectNode objectInsideWorkingObject = new ObjectNode(jsonNodeFactory);
+ workingObject.set(fieldPart, objectInsideWorkingObject);
+ workingObject = objectInsideWorkingObject;
+
+ } else {
+ workingObject = (ObjectNode) workingObject.get(fieldPart);
+ }
+
+ } else {
+ workingObject.set(fieldPart, endpointValue);
+ }
+ }
+ }
+
+ private String buildTableFieldValue(String fieldName, TableField field) {
+ switch (field.getFieldType()) {
+ case BIGINT:
+ return fieldName;
+ case DOUBLE:
+ return String.format("(%s + 0.5)", fieldName.replace("double", "integer"));
+ case STRING:
+ return String.format("CODEPOINT_TO_STRING([100 + %s])", fieldName);
+ }
+ throw new UnsupportedOperationException("Unsupported type for field: " + field.getFieldType().toString());
+ }
+
+ private String buildJSONForQuery(ObjectNode node) {
+ return node.toString().replace("\"$", "").replace("$\"", "").replace("\\\"", "\"");
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayElement.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayElement.java
new file mode 100644
index 0000000..6e41ead
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayElement.java
@@ -0,0 +1,77 @@
+/*
+ * 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.test.array;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ArrayElement {
+ final List<TableField> unnestList = new ArrayList<>();
+ final List<TableField> projectList = new ArrayList<>();
+ final int elementPosition;
+ final Kind kind;
+
+ public ArrayElement(Kind kind, int elementPosition) {
+ this.elementPosition = elementPosition;
+ this.kind = kind;
+ }
+
+ public
+
+ static class TableField {
+ private final List<String> fieldName;
+ private final BaseWisconsinTable.Field sourceField;
+
+ TableField(List<String> name, BaseWisconsinTable.Field field) {
+ fieldName = name;
+ sourceField = field;
+ }
+
+ List<String> getFieldName() {
+ return fieldName;
+ }
+
+ String getFullFieldName() {
+ return String.join(".", fieldName);
+ }
+
+ String getLastFieldName() {
+ return fieldName.get(fieldName.size() - 1);
+ }
+
+ BaseWisconsinTable.Field getSourceField() {
+ return sourceField;
+ }
+
+ public BaseWisconsinTable.Field.Type getFieldType() {
+ return (sourceField == null) ? null : sourceField.fieldType;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("%s [%s]", getFullFieldName(), (sourceField == null) ? "NONE" : sourceField.fieldName);
+ }
+ }
+
+ enum Kind {
+ ATOMIC,
+ UNNEST_VALUE,
+ UNNEST_OBJECT
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayIndex.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayIndex.java
new file mode 100644
index 0000000..f49cdb0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayIndex.java
@@ -0,0 +1,197 @@
+/*
+ * 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.test.array;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ArrayIndex {
+ private final List<ArrayElement> elements;
+ private final List<List<String>> arrayPath;
+ private String ddlStatement;
+ private String indexName;
+ private String datasetName;
+
+ @Override
+ public String toString() {
+ return ddlStatement;
+ }
+
+ public List<ArrayElement> getElements() {
+ return elements;
+ }
+
+ public List<List<String>> getArrayPath() {
+ return arrayPath;
+ }
+
+ public String getIndexName() {
+ return indexName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ private ArrayIndex(List<ArrayElement> elements, List<List<String>> arrayPath) {
+ this.elements = Collections.unmodifiableList(elements);
+ this.arrayPath = Collections.unmodifiableList(arrayPath);
+ }
+
+ public static class Builder {
+ public static abstract class ValueSupplier {
+ public abstract BaseWisconsinTable.Field getAtomicBaseField();
+
+ public abstract BaseWisconsinTable.Field getArrayBaseField();
+
+ public abstract List<String> getFieldName(BaseWisconsinTable.Field baseField);
+
+ public abstract List<String> getGroupFieldName(int nestingLevel);
+ }
+
+ private int numberOfAtomicPrefixes;
+ private int numberOfFieldsInArray;
+ private int numberOfAtomicSuffixes;
+ private boolean isArrayOfScalars;
+ private int depthOfArray;
+
+ private final String indexName;
+ private final String datasetName;
+ private ValueSupplier valueSupplier;
+
+ public void setNumberOfAtomicPrefixes(int numberOfAtomicPrefixes) {
+ this.numberOfAtomicPrefixes = numberOfAtomicPrefixes;
+ }
+
+ public void setNumberOfFieldsInArray(int numberOfFieldsInArray) {
+ this.numberOfFieldsInArray = numberOfFieldsInArray;
+ }
+
+ public void setNumberOfAtomicSuffixes(int numberOfAtomicSuffixes) {
+ this.numberOfAtomicSuffixes = numberOfAtomicSuffixes;
+ }
+
+ public void setDepthOfArray(int depthOfArray) {
+ this.depthOfArray = depthOfArray;
+ }
+
+ public void setIsArrayOfScalars(boolean isArrayOfScalars) {
+ this.isArrayOfScalars = isArrayOfScalars;
+ }
+
+ public void setValueSupplier(ValueSupplier valueSupplier) {
+ this.valueSupplier = valueSupplier;
+ }
+
+ public Builder(String indexName, String datasetName) {
+ this.indexName = indexName;
+ this.datasetName = datasetName;
+ }
+
+ public ArrayIndex build() {
+ final List<ArrayElement> elements = new ArrayList<>();
+ final List<List<String>> arrayPath = new ArrayList<>();
+
+ for (int i = 0; i < numberOfAtomicPrefixes; i++) {
+ ArrayElement element = new ArrayElement(ArrayElement.Kind.ATOMIC, elements.size());
+ BaseWisconsinTable.Field field = valueSupplier.getAtomicBaseField();
+ List<String> fieldName = valueSupplier.getFieldName(field);
+ element.projectList.add(new ArrayElement.TableField(fieldName, field));
+ elements.add(element);
+ }
+
+ ArrayElement arrayElement = new ArrayElement(
+ (isArrayOfScalars) ? ArrayElement.Kind.UNNEST_VALUE : ArrayElement.Kind.UNNEST_OBJECT,
+ elements.size());
+ for (int i = 0; i < depthOfArray; i++) {
+ ArrayElement.TableField tableField;
+ if (isArrayOfScalars && i == depthOfArray - 1) {
+ BaseWisconsinTable.Field field = valueSupplier.getArrayBaseField();
+ List<String> fieldName = valueSupplier.getFieldName(field);
+ tableField = new ArrayElement.TableField(fieldName, field);
+ } else {
+ List<String> fieldName = valueSupplier.getGroupFieldName(i + 1);
+ tableField = new ArrayElement.TableField(fieldName, null);
+ }
+ arrayElement.unnestList.add(tableField);
+ arrayPath.add(tableField.getFieldName());
+ }
+ if (!isArrayOfScalars) {
+ for (int i = 0; i < numberOfFieldsInArray; i++) {
+ BaseWisconsinTable.Field field = valueSupplier.getArrayBaseField();
+ List<String> fieldName = valueSupplier.getFieldName(field);
+ arrayElement.projectList.add(new ArrayElement.TableField(fieldName, field));
+ }
+ }
+ elements.add(arrayElement);
+
+ for (int i = 0; i < numberOfAtomicSuffixes; i++) {
+ ArrayElement element = new ArrayElement(ArrayElement.Kind.ATOMIC, elements.size());
+ BaseWisconsinTable.Field field = valueSupplier.getAtomicBaseField();
+ List<String> fieldName = valueSupplier.getFieldName(field);
+ element.projectList.add(new ArrayElement.TableField(fieldName, field));
+ elements.add(element);
+ }
+
+ ArrayIndex index = new ArrayIndex(elements, arrayPath);
+ index.ddlStatement = buildIndexDDL(index.elements);
+ index.datasetName = datasetName;
+ index.indexName = indexName;
+ return index;
+ }
+
+ private String buildIndexDDL(List<ArrayElement> elements) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("CREATE INDEX ").append(indexName);
+ sb.append(" ON ").append(datasetName).append(" ( ");
+ for (int i = 0; i < elements.size(); i++) {
+ ArrayElement e = elements.get(i);
+ if (!e.unnestList.isEmpty()) {
+ sb.append("( ");
+ }
+ for (ArrayElement.TableField unnestPart : e.unnestList) {
+ sb.append("UNNEST ");
+ sb.append(unnestPart.getFullFieldName());
+ sb.append(" ");
+ }
+ if (e.projectList.isEmpty()) {
+ sb.append(": ").append(e.unnestList.get(e.unnestList.size() - 1).getFieldType()).append(" )");
+ } else if (!e.unnestList.isEmpty()) {
+ sb.append("SELECT ");
+ List<ArrayElement.TableField> projectList = e.projectList;
+ for (int j = 0; j < projectList.size(); j++) {
+ sb.append(projectList.get(j).getFullFieldName());
+ sb.append(": ").append(projectList.get(j).getFieldType());
+ sb.append((j != projectList.size() - 1) ? ", " : " ");
+ }
+ sb.append(" )");
+ } else {
+ sb.append(e.projectList.get(0).getFullFieldName());
+ sb.append(": ").append(e.projectList.get(0).getFieldType());
+ }
+ if (i < elements.size() - 1) {
+ sb.append(", ");
+ }
+ }
+ sb.append(" ) EXCLUDE UNKNOWN KEY;\n");
+ return sb.toString();
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayQuery.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayQuery.java
new file mode 100644
index 0000000..325e061
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayQuery.java
@@ -0,0 +1,494 @@
+/*
+ * 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.test.array;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+public class ArrayQuery {
+ private final List<FromExpr> fromExprs = new ArrayList<>();
+ private final List<UnnestStep> unnestSteps = new ArrayList<>();
+ private final List<JoinStep> joinSteps = new ArrayList<>();
+ private final List<SelectExpr> selectExprs = new ArrayList<>();
+ private final List<Conjunct> whereConjuncts = new ArrayList<>();
+ private String queryString;
+
+ @Override
+ public String toString() {
+ return queryString;
+ }
+
+ public List<FromExpr> getFromExprs() {
+ return fromExprs;
+ }
+
+ public List<UnnestStep> getUnnestSteps() {
+ return unnestSteps;
+ }
+
+ public List<JoinStep> getJoinSteps() {
+ return joinSteps;
+ }
+
+ public List<SelectExpr> getSelectExprs() {
+ return selectExprs;
+ }
+
+ public List<Conjunct> getWhereConjuncts() {
+ return whereConjuncts;
+ }
+
+ public static class FromExpr {
+ final String datasetName;
+ final String alias;
+
+ public FromExpr(String datasetName, String alias) {
+ this.datasetName = datasetName;
+ this.alias = alias;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("%s AS %s", datasetName, alias);
+ }
+ }
+
+ public static class UnnestStep {
+ final String sourceAlias;
+ final String arrayField;
+ final String alias;
+
+ public UnnestStep(String sourceAlias, String arrayField, String alias) {
+ this.sourceAlias = sourceAlias;
+ this.arrayField = arrayField;
+ this.alias = alias;
+ }
+
+ public String getNamedExpr() {
+ return sourceAlias + "." + arrayField;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("UNNEST %s AS %s", getNamedExpr(), alias);
+ }
+ }
+
+ public static class SelectExpr {
+ final String expr;
+ final String alias;
+
+ public SelectExpr(String expr, String alias) {
+ this.expr = expr;
+ this.alias = alias;
+ }
+
+ public String asKeyValueString() {
+ return String.format("\"%s\":%s", alias, expr);
+ }
+
+ @Override
+ public String toString() {
+ return String.format("%s AS %s", expr, alias);
+ }
+ }
+
+ public interface Conjunct {
+ // Note: this is just a marker interface.
+ String toString();
+ }
+
+ public static class SimpleConjunct implements Conjunct {
+ final String expressionOne;
+ final String expressionTwo;
+ final String expressionThree;
+ final String operator;
+ final String annotation;
+
+ public SimpleConjunct(String e1, String e2, String e3, String operator, String annotation) {
+ this.expressionOne = e1;
+ this.expressionTwo = e2;
+ this.expressionThree = e3;
+ this.operator = operator;
+ this.annotation = annotation;
+ }
+
+ public SimpleConjunct(String e1, String e2, String operator, String annotation) {
+ this.expressionOne = e1;
+ this.expressionTwo = e2;
+ this.expressionThree = null;
+ this.operator = operator;
+ this.annotation = annotation;
+ }
+
+ @Override
+ public String toString() {
+ String a = (annotation == null) ? "" : (" " + annotation);
+ if (operator.equals("BETWEEN")) {
+ return String.format("( %s%s BETWEEN %s AND %s )", expressionOne, a, expressionTwo, expressionThree);
+
+ } else {
+ return String.format("( %s%s %s %s )", expressionOne, a, operator, expressionTwo);
+ }
+ }
+ }
+
+ public static class QuantifiedConjunct implements Conjunct {
+ final List<String> arraysToQuantify;
+ final List<String> quantifiedVars;
+ final List<Conjunct> conjuncts;
+ final String quantificationType;
+
+ public QuantifiedConjunct(List<String> arraysToQuantify, List<String> quantifiedVars, List<Conjunct> conjuncts,
+ String quantificationType) {
+ this.arraysToQuantify = arraysToQuantify;
+ this.quantifiedVars = quantifiedVars;
+ this.conjuncts = conjuncts;
+ this.quantificationType = quantificationType;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < arraysToQuantify.size(); i++) {
+ if (i != 0) {
+ sb.append(", ");
+ }
+ sb.append(quantifiedVars.get(i)).append(" IN ").append(arraysToQuantify.get(i));
+ }
+ String quantifiedBuildExpr = sb.toString();
+
+ return String.format("( %s %s SATISFIES %s )", quantificationType, quantifiedBuildExpr,
+ conjuncts.stream().map(Conjunct::toString).collect(Collectors.joining(" AND ")));
+ }
+ }
+
+ public static class ExistsConjunct implements Conjunct {
+ final FromExpr fromExpr;
+ final List<UnnestStep> unnestSteps;
+ final List<SimpleConjunct> conjuncts;
+
+ public ExistsConjunct(FromExpr fromExpr, List<UnnestStep> unnestSteps, List<SimpleConjunct> conjuncts) {
+ this.fromExpr = fromExpr;
+ this.unnestSteps = unnestSteps;
+ this.conjuncts = conjuncts;
+ }
+
+ @Override
+ public String toString() {
+ String unnestClause = unnestSteps.isEmpty() ? ""
+ : unnestSteps.stream().map(UnnestStep::toString).collect(Collectors.joining(" ")) + " ";
+ return String.format("EXISTS ( FROM %s %sWHERE %s SELECT 1 )", fromExpr.toString(), unnestClause,
+ conjuncts.stream().map(Conjunct::toString).collect(Collectors.joining(" AND ")));
+ }
+ }
+
+ public static class JoinStep {
+ final FromExpr subqueryFromExpr;
+ final List<UnnestStep> subqueryUnnestSteps;
+ final List<SelectExpr> subquerySelectExprs;
+ final List<SimpleConjunct> onExprConjuncts;
+ final String joinType;
+ final String joinAlias;
+
+ public JoinStep(List<UnnestStep> subqueryUnnestSteps, List<SelectExpr> subquerySelectExprs,
+ FromExpr subqueryFromExpr, List<SimpleConjunct> onExprConjuncts, String joinType, String joinAlias) {
+ this.subqueryUnnestSteps = subqueryUnnestSteps;
+ this.subquerySelectExprs = subquerySelectExprs;
+ this.subqueryFromExpr = subqueryFromExpr;
+ this.onExprConjuncts = onExprConjuncts;
+ this.joinType = joinType;
+ this.joinAlias = joinAlias;
+ }
+
+ public String getJoinExpr() {
+ String keyValuePairs = subquerySelectExprs.stream().map(SelectExpr::asKeyValueString)
+ .collect(Collectors.joining(",\n\t\t"));
+ return String.format("(\n\tFROM %s\n\t%s\n\tSELECT VALUE { \n\t\t%s\n\t} )", subqueryFromExpr,
+ subqueryUnnestSteps.stream().map(UnnestStep::toString).collect(Collectors.joining("\n\t")),
+ keyValuePairs);
+ }
+
+ @Override
+ public String toString() {
+ String onExpr = onExprConjuncts.stream().map(SimpleConjunct::toString).collect(Collectors.joining(" AND "));
+ return String.format("%s JOIN %s AS %s\nON %s", joinType, getJoinExpr(), joinAlias, onExpr);
+ }
+ }
+
+ @SuppressWarnings("rawtypes")
+ public static class Builder {
+ public static abstract class ValueSupplier {
+ public abstract List<UnnestStep> getExtraUnnestSteps(String alias);
+
+ public abstract String getJoinType();
+
+ public abstract boolean getIsConditionOnUnnest();
+
+ public abstract boolean getIsConstantConditionWithJoins();
+
+ public abstract boolean getIsCrossProductOnProbes();
+
+ public abstract String getOperatorForJoin();
+
+ public abstract String getOperatorForConstant();
+
+ public abstract Pair<String, String> getRangeFromDomain(BaseWisconsinTable.Domain domain);
+
+ public abstract boolean getIsBetweenConjunct();
+
+ public abstract String getQuantifier();
+
+ public abstract List<Conjunct> getExtraConjuncts(String alias);
+
+ public abstract List<Conjunct> getExtraQuantifiersAndConjuncts(List<String> arraysToQuantify,
+ List<String> quantifiedVars, String alias);
+
+ public abstract boolean getIsUseNestedQuantification();
+ }
+
+ private int limitCount;
+ private int unnestStepDepth;
+ private int numberOfJoins;
+ private int numberOfExplicitJoins;
+
+ private final ArrayIndex arrayIndex;
+ private final ArrayQuery arrayQuery;
+ private String workingUnnestAlias;
+ private ValueSupplier valueSupplier;
+
+ public Builder(ArrayIndex arrayIndex) {
+ this.arrayQuery = new ArrayQuery();
+ this.arrayIndex = arrayIndex;
+ }
+
+ public void setUnnestStepDepth(int unnestStepDepth) {
+ this.unnestStepDepth = unnestStepDepth;
+ }
+
+ public void setNumberOfJoins(int numberOfJoins) {
+ this.numberOfJoins = numberOfJoins;
+ }
+
+ public void setNumberOfExplicitJoins(int numberOfExplicitJoins) {
+ this.numberOfExplicitJoins = numberOfExplicitJoins;
+ }
+
+ @SuppressWarnings("SameParameterValue")
+ public void setLimitCount(int limitCount) {
+ this.limitCount = limitCount;
+ }
+
+ public void setValueSupplier(ValueSupplier valueSupplier) {
+ this.valueSupplier = valueSupplier;
+ }
+
+ public ArrayQuery build() {
+ StringBuilder sb = new StringBuilder();
+
+ // Start with our FROM clause. These will include all datasets to join with.
+ buildStartingFromClause(sb);
+
+ // Add the UNNEST steps expressions. These will always follow after the FROM expressions.
+ buildUnnestSteps(sb);
+
+ // Add the JOIN steps expressions. These will always follow after the UNNEST step expressions.
+ buildJoinSteps(sb);
+
+ // Add the WHERE clauses next. This will include implicit JOINs and quantified expressions.
+ buildWhereClause(sb);
+
+ // Add the SELECT clause. This consists of the primary keys from all involved datasets.
+ buildSelectClause(sb);
+
+ // We will ORDER BY all fields in our SELECT clause, and LIMIT accordingly.
+ buildQuerySuffix(sb);
+
+ arrayQuery.queryString = sb.toString();
+ return arrayQuery;
+ }
+
+ private void buildStartingFromClause(StringBuilder sb) {
+ for (int i = 0; i < numberOfJoins; i++) {
+ FromExpr fromExpr = new FromExpr("ProbeDataset" + (i + 1), "D" + (i + 2));
+ arrayQuery.fromExprs.add(fromExpr);
+ }
+ if (numberOfJoins == 0 || numberOfJoins != numberOfExplicitJoins) {
+ arrayQuery.fromExprs.add(new FromExpr(arrayIndex.getDatasetName(), "D1"));
+ }
+
+ sb.append("FROM ");
+ sb.append(arrayQuery.fromExprs.stream().map(FromExpr::toString).collect(Collectors.joining(",\n\t")));
+ sb.append('\n');
+ }
+
+ private void buildUnnestSteps(StringBuilder sb) {
+ if (unnestStepDepth == 0) {
+ workingUnnestAlias = "D1";
+
+ } else {
+ arrayQuery.unnestSteps.addAll(ArrayQueryUtil.createUnnestSteps(valueSupplier, "D1",
+ arrayIndex.getArrayPath(), unnestStepDepth, "P", "G", "G"));
+ workingUnnestAlias = arrayQuery.unnestSteps.get(arrayQuery.unnestSteps.size() - 1).alias;
+ String unnestSteps =
+ arrayQuery.unnestSteps.stream().map(UnnestStep::toString).collect(Collectors.joining("\n"));
+ sb.append(unnestSteps).append('\n');
+ }
+ }
+
+ private void buildJoinSteps(StringBuilder sb) {
+ if (numberOfExplicitJoins == 0) {
+ return;
+ }
+
+ // Build the expression to JOIN with. This will be a subquery that UNNESTs the array part that has
+ // not yet been UNNESTed.
+ List<UnnestStep> subqueryUnnestSteps = new ArrayList<>();
+ String prevSubqueryUnnestAlias = "D1";
+ for (int i = unnestStepDepth; i < arrayIndex.getArrayPath().size(); i++) {
+ String arrayField = String.join(".", arrayIndex.getArrayPath().get(i));
+ subqueryUnnestSteps.add(new UnnestStep(prevSubqueryUnnestAlias, arrayField, "G" + (i + 1)));
+ prevSubqueryUnnestAlias = "G" + (i + 1);
+ }
+ List<SelectExpr> subquerySelectExprs = new ArrayList<>();
+ String finalSubqueryUnnestAlias = prevSubqueryUnnestAlias;
+ ArrayQueryUtil.createFieldStream(arrayIndex.getElements()).forEach(e -> {
+ ArrayElement.TableField tableField = e.getRight();
+ String exprInSelect;
+ switch (e.getLeft()) {
+ case ATOMIC:
+ exprInSelect = "D1." + tableField.getFullFieldName();
+ subquerySelectExprs.add(new SelectExpr(exprInSelect, tableField.getLastFieldName()));
+ break;
+ case UNNEST_VALUE:
+ exprInSelect = finalSubqueryUnnestAlias;
+ subquerySelectExprs.add(new SelectExpr(exprInSelect, finalSubqueryUnnestAlias));
+ break;
+ case UNNEST_OBJECT:
+ exprInSelect = finalSubqueryUnnestAlias + "." + tableField.getFullFieldName();
+ subquerySelectExprs.add(new SelectExpr(exprInSelect, tableField.getLastFieldName()));
+ break;
+ }
+ });
+
+ // Now build each JOIN step.
+ for (int i = 0; i < numberOfExplicitJoins; i++) {
+ List<SimpleConjunct> onConjuncts = ArrayQueryUtil.createOnConjuncts(valueSupplier,
+ arrayIndex.getElements(), finalSubqueryUnnestAlias, i);
+ FromExpr indexedDataset = new FromExpr(arrayIndex.getDatasetName(), "D1");
+ arrayQuery.joinSteps.add(new JoinStep(subqueryUnnestSteps, subquerySelectExprs, indexedDataset,
+ onConjuncts, valueSupplier.getJoinType(), "J" + (i + 1)));
+ }
+ sb.append(arrayQuery.joinSteps.stream().map(JoinStep::toString).collect(Collectors.joining("\n")));
+ sb.append('\n');
+ }
+
+ private void buildWhereClause(StringBuilder sb) {
+ boolean isConditionOnUnnestPossible = unnestStepDepth == arrayIndex.getArrayPath().size();
+
+ // JOIN the remaining tables.
+ for (int i = arrayQuery.joinSteps.size(); i < numberOfJoins; i++) {
+ int joinPosition = i;
+ if (isConditionOnUnnestPossible && valueSupplier.getIsConditionOnUnnest()) {
+ // Having UNNESTed the indexed array(s), join with said array.
+ arrayQuery.whereConjuncts
+ .addAll(ArrayQueryUtil.createFieldStream(arrayIndex.getElements())
+ .map(e -> ArrayQueryUtil.createConjunctForUnnestJoins(valueSupplier,
+ workingUnnestAlias, e.getRight(), e.getLeft(), joinPosition))
+ .collect(Collectors.toList()));
+
+ } else {
+ // We are going to perform a quantified join. Join the atomic fields outside the quantification.
+ arrayQuery.whereConjuncts
+ .addAll(ArrayQueryUtil.createFieldStream(arrayIndex.getElements())
+ .filter(e -> e.getLeft() == ArrayElement.Kind.ATOMIC)
+ .map(e -> ArrayQueryUtil.createConjunctForUnnestJoins(valueSupplier,
+ workingUnnestAlias, e.getRight(), e.getLeft(), joinPosition))
+ .collect(Collectors.toList()));
+
+ // Create the quantified expression conjunct.
+ arrayQuery.whereConjuncts.add(ArrayQueryUtil.createConjunctForQuantifiedJoins(valueSupplier,
+ arrayIndex.getElements(), arrayIndex.getArrayPath(), "D1", joinPosition));
+ }
+ }
+
+ // If there are no JOINs, then we must condition on all constants in the index.
+ if (numberOfJoins == 0 || valueSupplier.getIsConstantConditionWithJoins()) {
+ // Condition on the atomic fields. Similar to joins, do not include these in the quantification.
+ arrayQuery.whereConjuncts.addAll(ArrayQueryUtil.createFieldStream(arrayIndex.getElements())
+ .filter(e -> e.getLeft() == ArrayElement.Kind.ATOMIC)
+ .map(e -> ArrayQueryUtil.createConjunctForUnnestNonJoins(valueSupplier,
+ "D1." + e.getRight().getFullFieldName(), e.getRight(), e.getLeft()))
+ .collect(Collectors.toList()));
+
+ if (isConditionOnUnnestPossible && valueSupplier.getIsConditionOnUnnest()) {
+ // Having UNNESTed the indexed array(s), condition on the items in said array.
+ arrayQuery.whereConjuncts
+ .addAll(ArrayQueryUtil.createFieldStream(arrayIndex.getElements())
+ .filter(e -> e.getLeft() != ArrayElement.Kind.ATOMIC)
+ .map(e -> ArrayQueryUtil.createConjunctForUnnestNonJoins(valueSupplier,
+ workingUnnestAlias, e.getRight(), e.getLeft()))
+ .collect(Collectors.toList()));
+
+ } else {
+ // Create the quantification expression conjunct.
+ arrayQuery.whereConjuncts.add(ArrayQueryUtil.createConjunctForQuantifiedNonJoins(valueSupplier,
+ arrayIndex.getElements(), arrayIndex.getArrayPath(), "D1"));
+ }
+ }
+
+ // If there is more than one probe, we must join each probe.
+ for (int i = 1; i < numberOfJoins; i++) {
+ String primaryKeyOne = "D" + (i + 1) + "._id";
+ String primaryKeyTwo = "D" + (i + 2) + "._id";
+ if (!valueSupplier.getIsCrossProductOnProbes()) {
+ arrayQuery.whereConjuncts.add(new SimpleConjunct(primaryKeyOne, primaryKeyTwo, "=", null));
+ }
+ }
+
+ if (!arrayQuery.whereConjuncts.isEmpty()) {
+ String d = " AND\n\t";
+ sb.append("WHERE ");
+ sb.append(arrayQuery.whereConjuncts.stream().map(Conjunct::toString).collect(Collectors.joining(d)));
+ sb.append('\n');
+ }
+ }
+
+ private void buildSelectClause(StringBuilder sb) {
+ sb.append("SELECT ");
+ if (numberOfJoins == 0 || numberOfJoins != numberOfExplicitJoins) {
+ arrayQuery.selectExprs.add(new SelectExpr("D1._id", "PK1"));
+ }
+ for (int i = 0; i < numberOfJoins; i++) {
+ arrayQuery.selectExprs.add(new SelectExpr("D" + (i + 2) + "._id", "PK" + (i + 2)));
+ }
+ sb.append(arrayQuery.selectExprs.stream().map(SelectExpr::toString).collect(Collectors.joining(", ")));
+ sb.append('\n');
+ }
+
+ private void buildQuerySuffix(StringBuilder sb) {
+ sb.append("ORDER BY ");
+ sb.append(arrayQuery.selectExprs.stream().map(e -> e.expr).collect(Collectors.joining(", "))).append('\n');
+ sb.append("LIMIT ").append(limitCount).append(";\n");
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayQueryUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayQueryUtil.java
new file mode 100644
index 0000000..3f2807f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ArrayQueryUtil.java
@@ -0,0 +1,366 @@
+/*
+ * 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.test.array;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.asterix.test.array.ArrayElement.Kind;
+import org.apache.asterix.test.array.ArrayElement.TableField;
+import org.apache.asterix.test.array.ArrayQuery.Builder.ValueSupplier;
+import org.apache.asterix.test.array.ArrayQuery.Conjunct;
+import org.apache.asterix.test.array.ArrayQuery.ExistsConjunct;
+import org.apache.asterix.test.array.ArrayQuery.FromExpr;
+import org.apache.asterix.test.array.ArrayQuery.QuantifiedConjunct;
+import org.apache.asterix.test.array.ArrayQuery.SimpleConjunct;
+import org.apache.asterix.test.array.ArrayQuery.UnnestStep;
+import org.apache.commons.lang3.tuple.Pair;
+
+class ArrayQueryUtil {
+ public static Stream<Pair<Kind, TableField>> createFieldStream(List<ArrayElement> elements) {
+ return elements.stream().map(e -> {
+ switch (e.kind) {
+ case ATOMIC:
+ TableField atomicField = e.projectList.get(0);
+ return Collections.singletonList(Pair.of(e.kind, atomicField));
+
+ case UNNEST_VALUE:
+ TableField lastUnnestField = e.unnestList.get(e.unnestList.size() - 1);
+ return Collections.singletonList(Pair.of(e.kind, lastUnnestField));
+
+ default: // UNNEST_OBJECT
+ return e.projectList.stream().map(p -> Pair.of(e.kind, p)).collect(Collectors.toUnmodifiableList());
+ }
+ }).flatMap(Collection::stream);
+ }
+
+ public static List<UnnestStep> createUnnestSteps(ValueSupplier valueSupplier, String startingAlias,
+ List<List<String>> arrayPath, int depthToUnnest, String prefixA, String prefixB, String prefixC) {
+ List<UnnestStep> unnestSteps = new ArrayList<>();
+ String workingAlias = startingAlias;
+ for (int i = 0; i < depthToUnnest; i++) {
+ List<UnnestStep> extraUnnestSteps = valueSupplier.getExtraUnnestSteps(workingAlias);
+ for (int j = 0; j < extraUnnestSteps.size(); j++) {
+ String aliasForExtraUnnest = prefixA.repeat(i + 1) + (j + 1);
+ UnnestStep extraUnnestStep = extraUnnestSteps.get(j);
+ UnnestStep extraUnnestStepWithAlias =
+ new UnnestStep(extraUnnestStep.sourceAlias, extraUnnestStep.arrayField, aliasForExtraUnnest);
+ unnestSteps.add(extraUnnestStepWithAlias);
+ }
+
+ String arrayField = String.join(".", arrayPath.get(i));
+ UnnestStep unnestStep = new UnnestStep(workingAlias, arrayField, prefixB + (i + 1));
+ unnestSteps.add(unnestStep);
+ workingAlias = prefixC + (i + 1);
+ }
+ return unnestSteps;
+ }
+
+ public static List<SimpleConjunct> createOnConjuncts(ValueSupplier valueSupplier,
+ List<ArrayElement> arrayIndexElements, String finalSubqueryUnnestAlias, int joinPosition) {
+ return ArrayQueryUtil.createFieldStream(arrayIndexElements).map(e -> {
+ TableField tableField = e.getRight();
+ String leftExpr = null;
+ switch (e.getLeft()) {
+ case ATOMIC:
+ case UNNEST_OBJECT:
+ leftExpr = "J" + (joinPosition + 1) + "." + tableField.getLastFieldName();
+ break;
+ case UNNEST_VALUE:
+ leftExpr = "J" + (joinPosition + 1) + "." + finalSubqueryUnnestAlias;
+ break;
+ }
+ String rightExpr = createProbeExpr(joinPosition, tableField);
+ String operator = valueSupplier.getOperatorForJoin();
+ return new SimpleConjunct(leftExpr, rightExpr, operator, "/* +indexnl */");
+ }).collect(Collectors.toList());
+ }
+
+ public static SimpleConjunct createConjunctForUnnestJoins(ValueSupplier valueSupplier, String unnestAlias,
+ ArrayElement.TableField tableField, ArrayElement.Kind fieldKind, int joinPosition) {
+ String leftExpr = null;
+ switch (fieldKind) {
+ case ATOMIC:
+ leftExpr = "D1." + tableField.getFullFieldName();
+ break;
+ case UNNEST_VALUE:
+ leftExpr = unnestAlias;
+ break;
+ case UNNEST_OBJECT:
+ leftExpr = unnestAlias + "." + tableField.getFullFieldName();
+ break;
+ }
+ String rightExpr = createProbeExpr(joinPosition, tableField);
+ return new SimpleConjunct(leftExpr, rightExpr, valueSupplier.getOperatorForJoin(), "/* +indexnl */");
+ }
+
+ public static QuantifiedConjunct createConjunctForQuantifiedJoins(ValueSupplier valueSupplier,
+ List<ArrayElement> arrayIndexElements, List<List<String>> workingArray, String prefix, int joinPosition) {
+ List<String> arraysToQuantify = new ArrayList<>();
+ List<String> quantifiedVars = new ArrayList<>();
+ List<Conjunct> conjuncts = new ArrayList<>();
+ int remainingDepthOfArray = workingArray.size();
+
+ if (remainingDepthOfArray > 1) {
+ String containingArray = String.join(".", workingArray.get(0));
+ arraysToQuantify.add(String.format("%s.%s", prefix, containingArray));
+ String itemVar = "V" + remainingDepthOfArray;
+ quantifiedVars.add(itemVar);
+
+ if (valueSupplier.getIsUseNestedQuantification()) {
+ // Recurse and create a nested quantification expression to quantify over the rest of our arrays.
+ conjuncts.add(createConjunctForQuantifiedJoins(valueSupplier, arrayIndexElements,
+ workingArray.subList(1, workingArray.size()), itemVar, joinPosition));
+
+ } else {
+ // We are now using an EXISTS clause to quantify over the rest of our arrays.
+ List<List<String>> remainingArray = workingArray.subList(1, workingArray.size());
+ conjuncts.add(createConjunctForExistsJoins(valueSupplier, arrayIndexElements, remainingArray, itemVar,
+ joinPosition));
+ }
+
+ } else {
+ conjuncts.addAll(createFieldStream(arrayIndexElements).filter(e -> e.getLeft() != Kind.ATOMIC)
+ .peek(e -> appendQuantifiedVars(workingArray, prefix, arraysToQuantify, quantifiedVars, e))
+ .map(e -> {
+ TableField tableField = e.getRight();
+ String operator = valueSupplier.getOperatorForJoin();
+ String leftExpr = null;
+ switch (e.getLeft()) {
+ case ATOMIC:
+ throw new IllegalStateException("Unexpected atomic element!");
+ case UNNEST_VALUE:
+ leftExpr = "V1";
+ break;
+ case UNNEST_OBJECT:
+ leftExpr = "V1." + tableField.getFullFieldName();
+ break;
+ }
+ String rightExpr = createProbeExpr(joinPosition, tableField);
+ return new SimpleConjunct(leftExpr, rightExpr, operator, "/* +indexnl */");
+ }).collect(Collectors.toList()));
+ }
+
+ return new QuantifiedConjunct(arraysToQuantify, quantifiedVars, conjuncts, valueSupplier.getQuantifier());
+ }
+
+ public static SimpleConjunct createConjunctForUnnestNonJoins(ValueSupplier valueSupplier, String unnestAlias,
+ ArrayElement.TableField tableField, ArrayElement.Kind fieldKind) {
+ String variableExpr = null;
+ switch (fieldKind) {
+ case ATOMIC:
+ variableExpr = "D1." + tableField.getFullFieldName();
+ break;
+ case UNNEST_VALUE:
+ variableExpr = unnestAlias;
+ break;
+ case UNNEST_OBJECT:
+ variableExpr = unnestAlias + "." + tableField.getFullFieldName();
+ break;
+ }
+
+ return createSimpleConjunctForNonJoins(valueSupplier, tableField, variableExpr);
+ }
+
+ public static QuantifiedConjunct createConjunctForQuantifiedNonJoins(ValueSupplier valueSupplier,
+ List<ArrayElement> indexElements, List<List<String>> workingArray, String prefix) {
+ List<String> arraysToQuantify = new ArrayList<>();
+ List<String> quantifiedVars = new ArrayList<>();
+ List<Conjunct> conjuncts = new ArrayList<>();
+ int remainingDepthOfArray = workingArray.size();
+
+ if (remainingDepthOfArray > 1) {
+ String containingArray = String.join(".", workingArray.get(0));
+ arraysToQuantify.add(String.format("%s.%s", prefix, containingArray));
+ String itemVar = "V" + remainingDepthOfArray;
+ quantifiedVars.add(itemVar);
+
+ if (valueSupplier.getIsUseNestedQuantification()) {
+ // Recurse and create a nested quantification expression to quantify over the rest of our arrays.
+ conjuncts.add(createConjunctForQuantifiedNonJoins(valueSupplier, indexElements,
+ workingArray.subList(1, workingArray.size()), itemVar));
+
+ } else {
+ // We are now using an EXISTS clause to quantify over the rest of our arrays.
+ List<List<String>> remainingArray = workingArray.subList(1, workingArray.size());
+ conjuncts.add(createConjunctForExistsNonJoins(valueSupplier, indexElements, remainingArray, itemVar));
+ }
+
+ } else {
+ conjuncts.addAll(createFieldStream(indexElements).filter(e -> e.getLeft() != Kind.ATOMIC)
+ .peek(e -> appendQuantifiedVars(workingArray, prefix, arraysToQuantify, quantifiedVars, e))
+ .map(e -> {
+ ArrayElement.TableField tableField = e.getRight();
+ String variableExpr = null;
+ switch (e.getLeft()) {
+ case ATOMIC:
+ throw new IllegalStateException("Unexpected atomic element!");
+ case UNNEST_VALUE:
+ variableExpr = "V1";
+ break;
+ case UNNEST_OBJECT:
+ variableExpr = "V1." + tableField.getFullFieldName();
+ break;
+ }
+ return createSimpleConjunctForNonJoins(valueSupplier, tableField, variableExpr);
+ }).collect(Collectors.toList()));
+ }
+
+ // Append any extra conjuncts from the indexed object.
+ if (ArrayQueryUtil.createFieldStream(indexElements).anyMatch(e -> e.getLeft() == Kind.UNNEST_OBJECT)) {
+ conjuncts.addAll(valueSupplier.getExtraConjuncts("V" + remainingDepthOfArray));
+ }
+
+ // Add any extra quantifiers.
+ conjuncts.addAll(valueSupplier.getExtraQuantifiersAndConjuncts(arraysToQuantify, quantifiedVars, prefix));
+
+ return new QuantifiedConjunct(arraysToQuantify, quantifiedVars, conjuncts, valueSupplier.getQuantifier());
+ }
+
+ private static void appendQuantifiedVars(List<List<String>> workingArray, String prefix,
+ List<String> arraysToQuantify, List<String> quantifiedVars, Pair<Kind, TableField> e) {
+ TableField tableField = e.getRight();
+ switch (e.getLeft()) {
+ case ATOMIC:
+ throw new IllegalStateException("Unexpected atomic element!");
+ case UNNEST_VALUE:
+ arraysToQuantify.add(String.format("%s.%s", prefix, tableField.getFullFieldName()));
+ quantifiedVars.add("V1");
+ break;
+ case UNNEST_OBJECT:
+ if (!quantifiedVars.contains("V1")) {
+ String containingArray = String.join(".", workingArray.get(0));
+ arraysToQuantify.add(String.format("%s.%s", prefix, containingArray));
+ quantifiedVars.add("V1");
+ }
+ break;
+ }
+ }
+
+ private static ExistsConjunct createConjunctForExistsJoins(ValueSupplier valueSupplier,
+ List<ArrayElement> arrayIndexElements, List<List<String>> arrayPath, String itemVar, int joinPosition) {
+ // Build our FROM and UNNEST clauses.
+ List<UnnestStep> unnestSteps = new ArrayList<>();
+ if (arrayPath.size() > 1) {
+ String prefixA = Character.toString('T' + joinPosition);
+ String prefixB = Character.toString('O' + joinPosition);
+ unnestSteps.addAll(ArrayQueryUtil.createUnnestSteps(valueSupplier, "GG1",
+ arrayPath.subList(1, arrayPath.size()), arrayPath.size() - 1, prefixA, prefixB, prefixB));
+ }
+ FromExpr fromExpr = new FromExpr(String.format("%s.%s", itemVar, String.join(".", arrayPath.get(0))), "GG1");
+
+ // Create the conjuncts for the JOIN.
+ String variableExpr = (unnestSteps.isEmpty()) ? "GG1" : unnestSteps.get(unnestSteps.size() - 1).alias;
+ List<SimpleConjunct> conjuncts =
+ createFieldStream(arrayIndexElements).filter(e -> e.getLeft() != Kind.ATOMIC).map(e -> {
+ TableField tableField = e.getRight();
+ String fieldName = tableField.getFullFieldName();
+ String operator = valueSupplier.getOperatorForJoin();
+ String leftExpr = null;
+ switch (e.getLeft()) {
+ case ATOMIC:
+ throw new IllegalStateException("Unexpected atomic element!");
+ case UNNEST_VALUE:
+ leftExpr = variableExpr;
+ break;
+ case UNNEST_OBJECT:
+ leftExpr = variableExpr + "." + fieldName;
+ break;
+ }
+ String rightExpr = createProbeExpr(joinPosition, tableField);
+ return new SimpleConjunct(leftExpr, rightExpr, operator, "/* +indexnl */");
+ }).collect(Collectors.toList());
+
+ return new ExistsConjunct(fromExpr, unnestSteps, conjuncts);
+ }
+
+ private static String createProbeExpr(int joinPosition, TableField tableField) {
+ String rightExpr = "D" + (joinPosition + 2) + "." + tableField.getSourceField().fieldName;
+ switch (tableField.getFieldType()) {
+ case BIGINT:
+ rightExpr = String.format("TO_BIGINT(%s)", rightExpr);
+ break;
+ case DOUBLE:
+ rightExpr = String.format("(TO_DOUBLE(%s) + 0.5)", rightExpr.replace("double", "integer"));
+ break;
+ case STRING:
+ rightExpr = String.format("CODEPOINT_TO_STRING([100 + %s])", rightExpr);
+ break;
+ }
+ return rightExpr;
+ }
+
+ private static SimpleConjunct createSimpleConjunctForNonJoins(ValueSupplier valueSupplier, TableField tableField,
+ String variableExpr) {
+ Pair<String, String> bounds = valueSupplier.getRangeFromDomain(tableField.getSourceField().domain);
+ if (valueSupplier.getIsBetweenConjunct()) {
+ return new SimpleConjunct(variableExpr, bounds.getLeft(), bounds.getRight(), "BETWEEN", null);
+
+ } else {
+ switch (valueSupplier.getOperatorForConstant()) {
+ case "=":
+ return new SimpleConjunct(variableExpr, bounds.getLeft(), "=", null);
+ case "<":
+ return new SimpleConjunct(variableExpr, bounds.getRight(), "<", null);
+ case ">":
+ return new SimpleConjunct(variableExpr, bounds.getLeft(), ">", null);
+ case "<=":
+ return new SimpleConjunct(variableExpr, bounds.getRight(), "<=", null);
+ default: // ">="
+ return new SimpleConjunct(variableExpr, bounds.getLeft(), ">=", null);
+ }
+ }
+ }
+
+ private static ExistsConjunct createConjunctForExistsNonJoins(ValueSupplier valueSupplier,
+ List<ArrayElement> arrayIndexElements, List<List<String>> arrayPath, String itemVar) {
+ // Build our FROM and UNNEST clauses.
+ List<UnnestStep> unnestSteps = new ArrayList<>();
+ if (arrayPath.size() > 1) {
+ unnestSteps.addAll(ArrayQueryUtil.createUnnestSteps(valueSupplier, "GG1",
+ arrayPath.subList(1, arrayPath.size()), arrayPath.size() - 1, "S", "N", "N"));
+ }
+ FromExpr fromExpr = new FromExpr(String.format("%s.%s", itemVar, String.join(".", arrayPath.get(0))), "GG1");
+
+ // Create the conjuncts for the JOIN.
+ String variableExpr = (unnestSteps.isEmpty()) ? "GG1" : unnestSteps.get(unnestSteps.size() - 1).alias;
+ List<SimpleConjunct> conjuncts =
+ createFieldStream(arrayIndexElements).filter(e -> e.getLeft() != Kind.ATOMIC).map(e -> {
+ TableField tableField = e.getRight();
+ String variableExprForConjunct = null;
+ switch (e.getLeft()) {
+ case ATOMIC:
+ throw new IllegalStateException("Unexpected atomic element!");
+ case UNNEST_VALUE:
+ variableExprForConjunct = variableExpr;
+ break;
+ case UNNEST_OBJECT:
+ variableExprForConjunct = variableExpr + "." + tableField.getFullFieldName();
+ break;
+ }
+ return createSimpleConjunctForNonJoins(valueSupplier, tableField, variableExprForConjunct);
+ }).collect(Collectors.toList());
+
+ return new ExistsConjunct(fromExpr, unnestSteps, conjuncts);
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/BaseWisconsinTable.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/BaseWisconsinTable.java
new file mode 100644
index 0000000..97c00bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/BaseWisconsinTable.java
@@ -0,0 +1,84 @@
+/*
+ * 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.test.array;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+public class BaseWisconsinTable {
+ public static final String TABLE_NAME = "BaseWisconsinTable";
+ public static final Path TABLE_FILE = Paths.get("data", "array-index.adm");
+
+ static final Field[] GROUPING_FIELDS =
+ new Field[] { Field.GROUPING_1000, Field.GROUPING_500, Field.GROUPING_250, Field.GROUPING_100 };
+ public static final int NUMBER_OF_GROUPING_FIELDS = GROUPING_FIELDS.length;
+ public static final int NUMBER_OF_NON_GROUPING_FIELDS = Field.values().length - NUMBER_OF_GROUPING_FIELDS;
+
+ @SuppressWarnings("rawtypes")
+ enum Field {
+ GROUPING_1000("grouping_1000", Type.BIGINT, new Domain<>(0, 999)),
+ GROUPING_500("grouping_500", Type.BIGINT, new Domain<>(0, 499)),
+ GROUPING_250("grouping_250", Type.BIGINT, new Domain<>(0, 249)),
+ GROUPING_100("grouping_100", Type.BIGINT, new Domain<>(0, 99)),
+
+ INTEGER_RAND_2000("integer_rand_2000", Type.BIGINT, new Domain<>(0, 1999)),
+ INTEGER_SEQ_2000("integer_seq_2000", Type.BIGINT, new Domain<>(0, 1999)),
+ INTEGER_SEQ_2("integer_rand_2", Type.BIGINT, new Domain<>(0, 1)),
+ INTEGER_SEQ_4("integer_rand_4", Type.BIGINT, new Domain<>(0, 3)),
+ INTEGER_SEQ_10("integer_rand_10", Type.BIGINT, new Domain<>(0, 9)),
+ INTEGER_SEQ_20("integer_rand_20", Type.BIGINT, new Domain<>(0, 19)),
+
+ DOUBLE_RAND_2000("double_rand_2000", Type.DOUBLE, new Domain<>(0.5, 1999.5)),
+ DOUBLE_SEQ_2000("double_seq_2000", Type.DOUBLE, new Domain<>(0.5, 1999.5)),
+ DOUBLE_SEQ_2("double_rand_2", Type.DOUBLE, new Domain<>(0.5, 1.5)),
+ DOUBLE_SEQ_4("double_rand_4", Type.DOUBLE, new Domain<>(0.5, 3.5)),
+ DOUBLE_SEQ_10("double_rand_10", Type.DOUBLE, new Domain<>(0.5, 9.5)),
+ DOUBLE_SEQ_20("double_rand_20", Type.DOUBLE, new Domain<>(0.5, 19.5)),
+
+ STRING_RAND_26_A("string_rand_26_a", Type.STRING, new Domain<>('A', 'Z')),
+ STRING_RAND_26_B("string_rand_26_b", Type.STRING, new Domain<>('A', 'Z')),
+ STRING_RAND_26_C("string_rand_26_c", Type.STRING, new Domain<>('A', 'Z'));
+
+ final String fieldName;
+ final Type fieldType;
+ final Domain domain;
+
+ Field(String fieldName, Type fieldType, Domain domain) {
+ this.fieldName = fieldName;
+ this.fieldType = fieldType;
+ this.domain = domain;
+ }
+
+ enum Type {
+ BIGINT,
+ DOUBLE,
+ STRING
+ }
+ }
+
+ static class Domain<T> {
+ final T minimum;
+ final T maximum;
+
+ Domain(T minimum, T maximum) {
+ this.minimum = minimum;
+ this.maximum = maximum;
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/SqlppArrayIndexRQGTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/SqlppArrayIndexRQGTest.java
new file mode 100644
index 0000000..070e29a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/SqlppArrayIndexRQGTest.java
@@ -0,0 +1,479 @@
+/*
+ * 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.test.array;
+
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.asterix.common.utils.Servlets;
+import org.apache.asterix.test.array.ArrayQuery.Conjunct;
+import org.apache.asterix.test.array.ArrayQuery.ExistsConjunct;
+import org.apache.asterix.test.array.ArrayQuery.QuantifiedConjunct;
+import org.apache.asterix.test.array.ArrayQuery.SimpleConjunct;
+import org.apache.asterix.test.array.ArrayQuery.UnnestStep;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.runtime.LangExecutionUtil;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
+import org.apache.commons.math3.distribution.ExponentialDistribution;
+import org.apache.commons.math3.random.MersenneTwister;
+import org.apache.commons.math3.random.RandomGenerator;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.JsonNode;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectReader;
+import org.testcontainers.shaded.com.fasterxml.jackson.databind.node.ObjectNode;
+
+/**
+ * This test will perform the following, using a flat schema that is defined ahead of time:
+ * <ol>
+ * <li>Generate a random schema for a single dataset using fields from the predefined flat schema, composed of
+ * various nested objects and an array of scalars or objects (whose contents can themselves contain scalars, arrays of
+ * scalars or arrays of objects following the same definition this is defined in).</li>
+ * <li>Using the randomly generated schema, create a query that will build the corresponding dataset from the flat
+ * schema using a series of CTEs and GROUP BY GROUP AS clauses.</li>
+ * <li>Create the array index DDL using the randomly generated schema.</li>
+ * <li>Randomly generate a query that will utilize the aforementioned array index, exploring the parameter space:<ul>
+ * <li>The number of non-indexed field terms defined in the query.</li>
+ * <li>The number of joins to perform for the query.</li>
+ * <li>The type of query.<ul>
+ *
+ * <li>For quantification queries...<ul>
+ * <li>The type of quantification (SOME AND EVERY | SOME).</li>
+ * <li>The presence of multiple quantifiers.</li>
+ * <li>Whether to use specify multiple levels of nesting with EXISTs or another quantification expression.</li>
+ * </ul></li>
+ *
+ * <li>For UNNEST queries...<ul>
+ * <li>The number of unrelated UNNESTs.</li>
+ * </ul></li>
+ *
+ * <li>For JOIN queries...<ul>
+ * <li>The type of JOIN (INNER | LEFT OUTER | Quantified).</li>
+ * <li><i>All of the options from the previous two items.</i></li>
+ * </ul></li>
+ * </ul></li>
+ * </ul>
+ * <li>Execute the query to build the dataset of our random schema and then execute the array index DDL, OR execute
+ * the array index DDL then build the dataset of our random schema. This decision is made randomly.</li>
+ * <li>Execute the randomly generated query with array index optimization enabled and without array index optimization
+ * enabled. Verify that the two result sets are exactly the same AND verify that the array index is being utilized.</li>
+ * <li>Repeat this entire process (from step 1) {@code ArrayIndexRQGIT.limit} amount of times.</li>
+ * </ol>
+ */
+@RunWith(Parameterized.class)
+public class SqlppArrayIndexRQGTest {
+ private static final Logger LOGGER = LogManager.getLogger(SqlppArrayIndexRQGTest.class);
+ private static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc.conf";
+ private static final String CONF_PROPERTY_LIMIT = "ArrayIndexRQGIT.limit";
+ private static final String CONF_PROPERTY_SEED = "ArrayIndexRQGIT.seed";
+ private static final String CONF_PROPERTY_JOIN = "ArrayIndexRQGIT.join";
+ private static final long CONF_PROPERTY_SEED_DEFAULT = System.currentTimeMillis();
+ private static final int CONF_PROPERTY_LIMIT_DEFAULT = 50;
+ private static final int CONF_PROPERTY_JOIN_DEFAULT = 25;
+ private static TestExecutor testExecutor;
+
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ private static final ObjectReader OBJECT_READER = OBJECT_MAPPER.readerFor(ObjectNode.class);
+
+ private static final int GROUP_MEMORY_MB = 4;
+ private static final int JOIN_MEMORY_MB = 4;
+ private static final int PROBE_DOCUMENT_LIMIT = 50;
+ private static final int QUERY_RESULT_LIMIT = 100;
+ private static final int JOIN_COUNT_LIMIT = 3;
+ private final TestInstance testInstance;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ testExecutor = new TestExecutor();
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+
+ StringBuilder sb = new StringBuilder();
+ sb.append("DROP DATAVERSE TestDataverse IF EXISTS;\n");
+ sb.append("CREATE DATAVERSE TestDataverse;\n");
+ sb.append("USE TestDataverse;\n\n");
+ sb.append("CREATE TYPE TestType AS { _id: uuid };\n");
+ sb.append("CREATE DATASET ").append(BaseWisconsinTable.TABLE_NAME);
+ sb.append(" (TestType)\nPRIMARY KEY _id AUTOGENERATED;\n");
+ sb.append("LOAD DATASET ").append(BaseWisconsinTable.TABLE_NAME).append('\n');
+ sb.append("USING localfs").append(String.format("((\"path\"=\"%s\"),(\"format\"=\"adm\"));\n\n",
+ "asterix_nc1://" + BaseWisconsinTable.TABLE_FILE));
+ for (int i = 0; i < JOIN_COUNT_LIMIT; i++) {
+ sb.append("CREATE DATASET ProbeDataset").append(i + 1);
+ sb.append(" (TestType)\nPRIMARY KEY _id AUTOGENERATED;\n");
+ sb.append("INSERT INTO ProbeDataset").append(i + 1).append('\n');
+ sb.append("FROM ").append(BaseWisconsinTable.TABLE_NAME);
+ sb.append(" B\nSELECT VALUE B LIMIT ").append(PROBE_DOCUMENT_LIMIT).append(";\n");
+ }
+
+ LOGGER.debug("Now executing setup DDL:\n" + sb);
+ testExecutor.executeSqlppUpdateOrDdl(sb.toString(), TestCaseContext.OutputFormat.ADM);
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+ }
+
+ private static long getConfigurationProperty(String propertyName, long defValue) {
+ String textValue = System.getProperty(propertyName);
+ if (textValue != null) {
+ try {
+ return Long.parseLong(textValue);
+
+ } catch (NumberFormatException e) {
+ LOGGER.warn(String.format("Cannot parse configuration property: %s. Will use default value: %d",
+ propertyName, defValue));
+ }
+ }
+ return defValue;
+ }
+
+ @Parameterized.Parameters(name = "ArrayIndexRQGIT {index}: {3}")
+ public static Collection<TestInstance> tests() {
+ List<TestInstance> testCases = new ArrayList<>();
+
+ long seed = getConfigurationProperty(CONF_PROPERTY_SEED, CONF_PROPERTY_SEED_DEFAULT);
+ int limit = (int) getConfigurationProperty(CONF_PROPERTY_LIMIT, CONF_PROPERTY_LIMIT_DEFAULT);
+ int join = (int) getConfigurationProperty(CONF_PROPERTY_JOIN, CONF_PROPERTY_JOIN_DEFAULT);
+ LOGGER.info(String.format("Testsuite configuration: -D%s=%d -D%s=%d -D%s=%d", CONF_PROPERTY_SEED, seed,
+ CONF_PROPERTY_LIMIT, limit, CONF_PROPERTY_JOIN, join));
+
+ // Initialize our random number generators.
+ BuilderContext context = new BuilderContext();
+ context.randomGenerator = new MersenneTwister(seed);
+ context.distBelow05 = new ExponentialDistribution(context.randomGenerator, 0.25, 1e-9);
+ context.distBelow1 = new ExponentialDistribution(context.randomGenerator, 0.5, 1e-9);
+ context.distEqual1 = new ExponentialDistribution(context.randomGenerator, 1.0, 1e-9);
+ context.distAbove1 = new ExponentialDistribution(context.randomGenerator, 1.5, 1e-9);
+ context.valueSupplierFactory = new ValueSupplierFactory(context.distBelow1, context.distEqual1,
+ context.distAbove1, context.randomGenerator);
+
+ for (int i = 0; i < limit; i++) {
+ TestInstance testCase = new TestInstance(i, i < join, context.randomGenerator.nextBoolean());
+ context.isJoin = testCase.isJoin;
+
+ // Build the array index first (implicitly, the schema for our dataset).
+ testCase.arrayIndex = buildArrayIndex(context);
+
+ // Build the dataset that corresponds to the array index.
+ testCase.arrayDataset = buildArrayDataset(context);
+
+ // Build the query that corresponds to the array index.
+ testCase.arrayQuery = buildArrayQuery(context);
+
+ // Finally, add the test case to our test cases.
+ testCases.add(testCase);
+ }
+
+ return testCases;
+ }
+
+ private static ArrayIndex buildArrayIndex(BuilderContext context) {
+ ArrayIndex.Builder indexBuilder = new ArrayIndex.Builder("testIndex", "IndexedDataset");
+
+ // Determine the number of fields and the depth of our array.
+ int totalNumberOfFields, depthOfArray;
+ do {
+ int numberOfAtomicPrefixes, numberOfFieldsInArray, numberOfAtomicSuffixes;
+ if (context.isJoin) {
+ // TODO (GLENN): Avoid performing joins with composite-atomic indexes for now.
+ numberOfAtomicPrefixes = 0; // (int) Math.round(context.distBelow05.sample());
+ numberOfFieldsInArray = Math.max(1, (int) Math.round(context.distBelow1.sample()));
+ numberOfAtomicSuffixes = 0; // (int) Math.round(context.distBelow05.sample());
+ } else {
+ numberOfAtomicPrefixes = (int) Math.round(context.distBelow05.sample());
+ numberOfFieldsInArray = Math.max(1, (int) Math.round(context.distBelow1.sample()));
+ numberOfAtomicSuffixes = (int) Math.round(context.distBelow05.sample());
+ }
+ indexBuilder.setNumberOfAtomicPrefixes(numberOfAtomicPrefixes);
+ indexBuilder.setNumberOfFieldsInArray(numberOfFieldsInArray);
+ indexBuilder.setNumberOfAtomicSuffixes(numberOfAtomicSuffixes);
+ indexBuilder.setIsArrayOfScalars(numberOfFieldsInArray == 1 && context.randomGenerator.nextBoolean());
+ depthOfArray = Math.max(1, (int) Math.round(context.distBelow1.sample()));
+ indexBuilder.setDepthOfArray(depthOfArray);
+ totalNumberOfFields = numberOfAtomicPrefixes + numberOfFieldsInArray + numberOfAtomicSuffixes;
+ } while (totalNumberOfFields > BaseWisconsinTable.NUMBER_OF_NON_GROUPING_FIELDS
+ || depthOfArray > BaseWisconsinTable.NUMBER_OF_GROUPING_FIELDS);
+
+ // Characterize how fields are generated (uniformly random).
+ indexBuilder.setValueSupplier(context.valueSupplierFactory.getCompleteArrayIndexValueSupplier());
+ context.arrayIndex = indexBuilder.build();
+ return context.arrayIndex;
+ }
+
+ private static ArrayDataset buildArrayDataset(BuilderContext context) {
+ ArrayDataset.Builder datasetBuilder = new ArrayDataset.Builder();
+ context.arrayIndex.getElements().forEach(datasetBuilder::addElement);
+ return datasetBuilder.build();
+ }
+
+ private static ArrayQuery buildArrayQuery(BuilderContext context) {
+ ArrayQuery.Builder queryBuilder = new ArrayQuery.Builder(context.arrayIndex);
+ queryBuilder.setLimitCount(QUERY_RESULT_LIMIT);
+
+ // Determine constants that should only be generated once.
+ int joinCount = Math.min(JOIN_COUNT_LIMIT, 1 + (int) Math.round(context.distBelow1.sample()));
+ int depthOfArray = context.arrayIndex.getArrayPath().size();
+ if (context.isJoin) {
+ // TODO: This is to avoid performing UNNEST joins on composite-atomic indexes. Refer to ASTERIXDB-2964.
+ // queryBuilder.setUnnestStepDepth(Math.min(depthOfArray, (int) Math.round(context.distBelow1.sample())));
+ queryBuilder.setUnnestStepDepth(0);
+ queryBuilder.setNumberOfJoins(joinCount);
+ if (joinCount == 1) {
+ // TODO: This is to avoid performing multiple UNNEST style explicit joins (falls under ASTERIXDB-2964).
+ queryBuilder.setNumberOfExplicitJoins(context.randomGenerator.nextBoolean() ? 1 : 0);
+ } else {
+ queryBuilder.setNumberOfExplicitJoins(0);
+ }
+ } else {
+ // TODO: This is to avoid specifying extra UNNESTs. Refer to ASTERIXDB-2962.
+ // queryBuilder.setUnnestStepDepth(Math.min(depthOfArray, (int) Math.round(context.distBelow1.sample())));
+ queryBuilder.setUnnestStepDepth(depthOfArray);
+ queryBuilder.setNumberOfJoins(0);
+ queryBuilder.setNumberOfExplicitJoins(0);
+ }
+
+ // Characterize the randomness in areas that occur more than once.
+ queryBuilder.setValueSupplier(context.valueSupplierFactory.getWorkingArrayQueryValueSupplier());
+ return queryBuilder.build();
+ }
+
+ public SqlppArrayIndexRQGTest(TestInstance testInstance) {
+ this.testInstance = testInstance;
+ }
+
+ @Test
+ public void test() throws Exception {
+ LOGGER.info("\n" + testInstance);
+ LOGGER.debug("Now executing test setup:\n" + testInstance.getTestSetup());
+ testExecutor.executeSqlppUpdateOrDdl(testInstance.getTestSetup(), TestCaseContext.OutputFormat.ADM);
+
+ // In addition to the results, we want the query plan as well.
+ String nonOptimizedQueryPrefix = "SET `compiler.arrayindex` \"false\";\n";
+ String optimizedQueryPrefix = "SET `compiler.arrayindex` \"true\";\n";
+ TestCase.CompilationUnit.Parameter planParameter = new TestCase.CompilationUnit.Parameter();
+ planParameter.setName("optimized-logical-plan");
+ planParameter.setValue("true");
+ planParameter.setType(ParameterTypeEnum.STRING);
+
+ try (InputStream notOptimizedResultStream =
+ testExecutor.executeQueryService(nonOptimizedQueryPrefix + testInstance.getTestQuery(),
+ TestCaseContext.OutputFormat.ADM, testExecutor.getEndpoint(Servlets.QUERY_SERVICE),
+ Collections.singletonList(planParameter), true, StandardCharsets.UTF_8);
+ InputStream optimizedResultStream =
+ testExecutor.executeQueryService(optimizedQueryPrefix + testInstance.getTestQuery(),
+ TestCaseContext.OutputFormat.ADM, testExecutor.getEndpoint(Servlets.QUERY_SERVICE),
+ Collections.singletonList(planParameter), true, StandardCharsets.UTF_8)) {
+ LOGGER.debug("Now executing query:\n" + testInstance.getTestQuery());
+
+ // Verify that the optimized result is the same as the non optimized result.
+ ObjectNode r1 = OBJECT_READER.readValue(notOptimizedResultStream);
+ ObjectNode r2 = OBJECT_READER.readValue(optimizedResultStream);
+ boolean doesR1HaveErrors = (r1.get("errors") != null);
+ boolean doesR2HaveErrors = (r2.get("errors") != null);
+
+ if (r1.get("results") == null || r2.get("results") == null) {
+ LOGGER.error("Results not found. Errors are:\n"
+ + (doesR1HaveErrors ? ("Non Optimized Error: " + r1.get("errors") + "\n")
+ : ("No errors thrown from the non-optimized query!\n"))
+ + (doesR2HaveErrors ? ("Optimized Error: " + r2.get("errors") + "\n")
+ : ("No errors thrown from the optimized query!\n")));
+ if (!doesR1HaveErrors && doesR2HaveErrors) {
+ Assert.fail("Optimized query resulted in an error.");
+ } else {
+ LOGGER.error("Both queries have resulted in an error (not logging issue w/ query gen itself).");
+ }
+
+ } else if (!r1.get("results").equals(r2.get("results"))) {
+ LOGGER.error("Non optimized query returned: " + r1.get("results") + "\n");
+ LOGGER.error("Optimized query returned: " + r2.get("results") + "\n");
+ Assert.fail("Optimized result is not equal to the non-optimized result.");
+
+ } else {
+ Iterator<JsonNode> results = r1.get("results").elements();
+ if (!results.hasNext()) {
+ LOGGER.error("Query has produced no results!");
+ } else {
+ LOGGER.debug("First result of query is: " + results.next().toString());
+ }
+ }
+
+ // Verify that the array index is being used by the "optimized" query.
+ String indexName = testInstance.arrayIndex.getIndexName();
+ if (!doesR1HaveErrors && !doesR2HaveErrors
+ && !r2.get("plans").get("optimizedLogicalPlan").asText().contains(indexName)) {
+ LOGGER.error("Optimized query plan: \n" + r2.get("plans").get("optimizedLogicalPlan").asText());
+ Assert.fail("Index " + indexName + " is not being used.");
+ }
+ }
+ }
+
+ private static class TestInstance {
+ private final int testCaseID;
+ private final boolean isJoin;
+ private final boolean isLoadFirst;
+
+ private ArrayDataset arrayDataset;
+ private ArrayIndex arrayIndex;
+ private ArrayQuery arrayQuery;
+
+ private String getTestSetup() {
+ String compilerSetStmt = "SET `compiler.groupmemory` \"" + GROUP_MEMORY_MB + "MB\";\n"
+ + "SET `compiler.joinmemory` \"" + JOIN_MEMORY_MB + "MB\";\n";
+ String dropDatasetStmt = "DROP DATASET IndexedDataset IF EXISTS;\n";
+ String createDatasetStmt = "CREATE DATASET IndexedDataset (TestType)\nPRIMARY KEY _id AUTOGENERATED;\n";
+ String leadingStatements = compilerSetStmt + "USE TestDataverse;\n" + dropDatasetStmt + createDatasetStmt;
+ if (isLoadFirst) {
+ return leadingStatements + "INSERT INTO IndexedDataset (\n" + arrayDataset + "\n);\n" + arrayIndex;
+ } else {
+ return leadingStatements + arrayIndex + "INSERT INTO IndexedDataset (\n" + arrayDataset + "\n);\n";
+ }
+ }
+
+ private String getTestQuery() {
+ String groupMemStmt = "SET `compiler.groupmemory` \"" + GROUP_MEMORY_MB + "MB\";\n";
+ String joinMemStmt = "SET `compiler.joinmemory` \"" + JOIN_MEMORY_MB + "MB\";\n";
+ String dataverseUseStmt = "USE TestDataverse;\n";
+ return groupMemStmt + joinMemStmt + dataverseUseStmt + arrayQuery;
+ }
+
+ public TestInstance(int testCaseID, boolean isJoin, boolean isLoadFirst) {
+ this.testCaseID = testCaseID;
+ this.isJoin = isJoin;
+ this.isLoadFirst = isLoadFirst;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("-------------------------------------------------------\n");
+ sb.append("Test Case Name: #").append(testCaseID).append('\n');
+ sb.append("Test Type: ").append(isJoin ? "JOIN" : "NON-JOIN").append('\n');
+ sb.append("Test Order: ");
+ sb.append(isLoadFirst ? "INSERT -> CREATE INDEX" : "CREATE INDEX -> INSERT").append(" -> QUERY\n");
+ sb.append("Indexed Elements:\n");
+ arrayIndex.getElements().forEach(e -> {
+ sb.append("\tElement #").append(e.elementPosition).append(":\n");
+ sb.append("\t\tKind: ").append(e.kind).append('\n');
+ e.unnestList.forEach(f -> sb.append("\t\tUNNEST Field: ").append(f).append('\n'));
+ e.projectList.forEach(f -> sb.append("\t\tPROJECT Field: ").append(f).append('\n'));
+ });
+ sb.append("Indexed Query: \n");
+ arrayQuery.getFromExprs().forEach(e -> {
+ sb.append("\tStarting FROM Expr:\n");
+ sb.append("\t\tDataset Name: ").append(e.datasetName).append('\n');
+ sb.append("\t\tAs Alias: ").append(e.alias).append('\n');
+ });
+ arrayQuery.getUnnestSteps().forEach(e -> {
+ sb.append("\tUNNEST Step:\n");
+ sb.append("\t\tSource Alias: ").append(e.sourceAlias).append('\n');
+ sb.append("\t\tArray Field: ").append(e.arrayField).append('\n');
+ sb.append("\t\tAs Alias: ").append(e.alias).append('\n');
+ });
+ arrayQuery.getJoinSteps().forEach(e -> {
+ sb.append("\tJoin Step:\n");
+ sb.append("\t\tSubquery FROM Expr:\n");
+ sb.append("\t\t\tDataset Name: ").append(e.subqueryFromExpr.datasetName).append('\n');
+ sb.append("\t\t\tAs Alias: ").append(e.subqueryFromExpr.alias).append('\n');
+ for (UnnestStep unnestStep : e.subqueryUnnestSteps) {
+ sb.append("\t\tSubquery UNNEST Step:\n");
+ sb.append("\t\t\tSource Alias: ").append(unnestStep.sourceAlias).append('\n');
+ sb.append("\t\t\tArray Field: ").append(unnestStep.arrayField).append('\n');
+ sb.append("\t\t\tAs Alias: ").append(unnestStep.alias).append('\n');
+ }
+ });
+ arrayQuery.getWhereConjuncts().forEach(e -> printConjunct(sb, e, 1));
+ arrayQuery.getSelectExprs().forEach(e -> {
+ sb.append("\tSelect Expr:\n");
+ sb.append("\t\tExpr: ").append(e.expr).append('\n');
+ sb.append("\t\tAs Alias: ").append(e.alias).append('\n');
+ });
+ sb.append("-------------------------------------------------------\n");
+ return sb.toString();
+ }
+
+ private static void printConjunct(StringBuilder sb, Conjunct conjunct, int depth) {
+ String localTabPrefix = "\t".repeat(depth);
+ if (conjunct instanceof SimpleConjunct) {
+ SimpleConjunct simpleConjunct = (SimpleConjunct) conjunct;
+ sb.append(localTabPrefix).append("WHERE Conjunct (Simple):\n");
+ localTabPrefix = localTabPrefix + "\t";
+ sb.append(localTabPrefix).append("Expression 1: ").append(simpleConjunct.expressionOne).append('\n');
+ sb.append(localTabPrefix).append("Expression 2: ").append(simpleConjunct.expressionTwo).append('\n');
+ sb.append(localTabPrefix).append("Expression 3: ").append(simpleConjunct.expressionThree).append('\n');
+ sb.append(localTabPrefix).append("Operator: ").append(simpleConjunct.operator).append('\n');
+ sb.append(localTabPrefix).append("Annotation: ").append(simpleConjunct.annotation).append('\n');
+
+ } else if (conjunct instanceof QuantifiedConjunct) {
+ QuantifiedConjunct quantifiedConjunct = (QuantifiedConjunct) conjunct;
+ sb.append(localTabPrefix).append("WHERE Conjunct (Quantified):\n");
+ localTabPrefix = localTabPrefix + "\t";
+ for (int i = 0; i < quantifiedConjunct.arraysToQuantify.size(); i++) {
+ String arrayToQuantify = quantifiedConjunct.arraysToQuantify.get(i);
+ String quantifiedVar = quantifiedConjunct.quantifiedVars.get(i);
+ sb.append(localTabPrefix).append("Quantified Array: ").append(arrayToQuantify).append('\n');
+ sb.append(localTabPrefix).append('\t').append("As Alias: ").append(quantifiedVar).append('\n');
+ }
+ quantifiedConjunct.conjuncts.forEach(e -> printConjunct(sb, e, depth + 1));
+
+ } else {
+ ExistsConjunct existsConjunct = (ExistsConjunct) conjunct;
+ ArrayQuery.FromExpr fromExpr = existsConjunct.fromExpr;
+ sb.append(localTabPrefix).append("WHERE Conjunct (EXISTS):\n");
+ localTabPrefix = localTabPrefix + "\t";
+ sb.append(localTabPrefix).append("Subquery FROM Expr:\n");
+ sb.append(localTabPrefix).append("\t").append("FROM Expr: ").append(fromExpr.datasetName).append('\n');
+ sb.append(localTabPrefix).append("\t").append("As Alias: ").append(fromExpr.alias).append('\n');
+ for (UnnestStep e : existsConjunct.unnestSteps) {
+ sb.append(localTabPrefix).append("Subquery UNNEST Steps:\n");
+ sb.append(localTabPrefix).append('\t').append("Source Alias: ").append(e.sourceAlias).append('\n');
+ sb.append(localTabPrefix).append('\t').append("Array Field: ").append(e.arrayField).append('\n');
+ sb.append(localTabPrefix).append('\t').append("As Alias: ").append(e.alias).append('\n');
+ }
+ sb.append(localTabPrefix).append("Subquery WHERE Conjuncts:\n");
+ existsConjunct.conjuncts.forEach(e -> printConjunct(sb, e, depth + 1));
+ }
+ }
+ }
+
+ private static class BuilderContext {
+ ValueSupplierFactory valueSupplierFactory;
+ ExponentialDistribution distBelow05;
+ ExponentialDistribution distBelow1;
+ ExponentialDistribution distEqual1;
+ ExponentialDistribution distAbove1;
+ RandomGenerator randomGenerator;
+ ArrayIndex arrayIndex;
+ boolean isJoin;
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ValueSupplierFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ValueSupplierFactory.java
new file mode 100644
index 0000000..0d63f12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/array/ValueSupplierFactory.java
@@ -0,0 +1,283 @@
+/*
+ * 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.test.array;
+
+import java.util.ArrayList;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.test.array.ArrayQuery.Builder.ValueSupplier;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.commons.math3.distribution.ExponentialDistribution;
+import org.apache.commons.math3.random.RandomGenerator;
+
+public class ValueSupplierFactory {
+ private final ExponentialDistribution distBelow1;
+ private final ExponentialDistribution distEqual1;
+ private final ExponentialDistribution distAbove1;
+ private final RandomGenerator randomGenerator;
+
+ public ValueSupplierFactory(ExponentialDistribution distBelow1, ExponentialDistribution distEqual1,
+ ExponentialDistribution distAbove1, RandomGenerator randomGenerator) {
+ this.distBelow1 = distBelow1;
+ this.distEqual1 = distEqual1;
+ this.distAbove1 = distAbove1;
+ this.randomGenerator = randomGenerator;
+ }
+
+ private class BaseArrayIndexValueSupplier extends ArrayIndex.Builder.ValueSupplier {
+ private final Set<BaseWisconsinTable.Field> consumedFields = new LinkedHashSet<>();
+ private final Set<String> consumedContainedNames = new LinkedHashSet<>();
+
+ @Override
+ public BaseWisconsinTable.Field getAtomicBaseField() {
+ BaseWisconsinTable.Field field;
+ do {
+ int index = BaseWisconsinTable.NUMBER_OF_GROUPING_FIELDS
+ + randomGenerator.nextInt(BaseWisconsinTable.NUMBER_OF_NON_GROUPING_FIELDS);
+ field = BaseWisconsinTable.Field.values()[index];
+ } while (consumedFields.contains(field) || field.fieldType != BaseWisconsinTable.Field.Type.BIGINT);
+ consumedFields.add(field);
+ return field;
+ }
+
+ @Override
+ public BaseWisconsinTable.Field getArrayBaseField() {
+ BaseWisconsinTable.Field field;
+ do {
+ int index = BaseWisconsinTable.NUMBER_OF_GROUPING_FIELDS
+ + randomGenerator.nextInt(BaseWisconsinTable.NUMBER_OF_NON_GROUPING_FIELDS);
+ field = BaseWisconsinTable.Field.values()[index];
+ } while (consumedFields.contains(field));
+ consumedFields.add(field);
+ return field;
+ }
+
+ @Override
+ public List<String> getFieldName(BaseWisconsinTable.Field baseField) {
+ List<String> fieldName = new ArrayList<>();
+ if (randomGenerator.nextBoolean()) {
+ int index = randomGenerator.nextInt(ArrayDataset.CONTAINER_OBJECT_NAMES.length);
+ fieldName.add(ArrayDataset.CONTAINER_OBJECT_NAMES[index]);
+ }
+ fieldName.add(baseField.fieldName);
+ if (randomGenerator.nextBoolean()) {
+ // Three tries to generate a unique contained name. Otherwise, we default to no contained name.
+ for (int i = 0; i < 3; i++) {
+ int index = randomGenerator.nextInt(ArrayDataset.CONTAINED_OBJECT_NAMES.length);
+ String containedName = ArrayDataset.CONTAINED_OBJECT_NAMES[index];
+ if (!consumedContainedNames.contains(containedName)) {
+ fieldName.add(containedName);
+ consumedContainedNames.add(containedName);
+ break;
+ }
+ }
+ }
+ return fieldName;
+ }
+
+ @Override
+ public List<String> getGroupFieldName(int nestingLevel) {
+ List<String> fieldName = new ArrayList<>();
+ if (randomGenerator.nextBoolean()) {
+ int index = randomGenerator.nextInt(ArrayDataset.CONTAINER_OBJECT_NAMES.length);
+ fieldName.add(ArrayDataset.CONTAINER_OBJECT_NAMES[index]);
+ }
+ fieldName.add("nesting_" + nestingLevel);
+ if (randomGenerator.nextBoolean()) {
+ int index = randomGenerator.nextInt(ArrayDataset.CONTAINED_OBJECT_NAMES.length);
+ fieldName.add(ArrayDataset.CONTAINED_OBJECT_NAMES[index]);
+ }
+ return fieldName;
+ }
+ }
+
+ private class BaseArrayQueryValueSupplier extends ValueSupplier {
+ @Override
+ public String getJoinType() {
+ return ((int) Math.round(distEqual1.sample()) == 0) ? "INNER" : "LEFT OUTER";
+ }
+
+ @Override
+ public String getOperatorForJoin() {
+ boolean isEquiJoin = randomGenerator.nextBoolean();
+ switch (isEquiJoin ? 0 : (1 + randomGenerator.nextInt(4))) {
+ case 0:
+ return "=";
+ case 1:
+ return "<";
+ case 2:
+ return ">";
+ case 3:
+ return "<=";
+ default:
+ return ">=";
+ }
+ }
+
+ @Override
+ public String getOperatorForConstant() {
+ boolean isEqualityPredicate = distBelow1.sample() > 2;
+ switch (isEqualityPredicate ? 0 : (1 + randomGenerator.nextInt(4))) {
+ case 0:
+ return "=";
+ case 1:
+ return "<";
+ case 2:
+ return ">";
+ case 3:
+ return "<=";
+ default:
+ return ">=";
+ }
+ }
+
+ @SuppressWarnings("rawtypes")
+ @Override
+ public Pair<String, String> getRangeFromDomain(BaseWisconsinTable.Domain domain) {
+ if (domain.minimum instanceof Integer && domain.maximum instanceof Integer) {
+ int domainMinimum = (int) domain.minimum, domainMaximum = (int) domain.maximum;
+ return Pair.of(String.valueOf(domainMinimum), String.valueOf(domainMaximum));
+
+ } else if (domain.minimum instanceof Character && domain.maximum instanceof Character) {
+ int domainMinimum = (char) domain.minimum, domainMaximum = (char) domain.maximum;
+ return Pair.of("\"" + (char) domainMinimum + "\"", "\"" + (char) domainMaximum + "\"");
+
+ } else if (domain.minimum instanceof Double && domain.maximum instanceof Double) {
+ int domainMinimum = (int) ((double) domain.minimum - 0.5);
+ int domainMaximum = (int) ((double) domain.maximum - 0.5);
+ return Pair.of(String.valueOf(domainMinimum), String.valueOf(domainMaximum));
+
+ } else {
+ throw new UnsupportedOperationException("Unknown / unsupported type for domain value.");
+ }
+ }
+
+ @Override
+ public boolean getIsBetweenConjunct() {
+ return randomGenerator.nextBoolean();
+ }
+
+ @Override
+ public String getQuantifier() {
+ return (distBelow1.sample() < 1) ? "SOME" : "SOME AND EVERY";
+ }
+
+ @Override
+ public boolean getIsConditionOnUnnest() {
+ return randomGenerator.nextBoolean();
+ }
+
+ @Override
+ public boolean getIsConstantConditionWithJoins() {
+ return distBelow1.sample() > 1;
+ }
+
+ @Override
+ public boolean getIsCrossProductOnProbes() {
+ return distBelow1.sample() > 1;
+ }
+
+ @Override
+ public boolean getIsUseNestedQuantification() {
+ return randomGenerator.nextBoolean();
+ }
+
+ @Override
+ public List<ArrayQuery.UnnestStep> getExtraUnnestSteps(String alias) {
+ List<ArrayQuery.UnnestStep> unnestSteps = new ArrayList<>();
+ int numberOfAdditionalGroups = ArrayDataset.ADDITIONAL_GROUPS.length;
+ for (int i = 0; i < randomGenerator.nextInt(numberOfAdditionalGroups + 1); i++) {
+ unnestSteps.add(new ArrayQuery.UnnestStep(alias, ArrayDataset.ADDITIONAL_GROUPS[i], null));
+ }
+ return unnestSteps;
+ }
+
+ @Override
+ public List<ArrayQuery.Conjunct> getExtraConjuncts(String alias) {
+ List<ArrayQuery.Conjunct> extraConjuncts = new ArrayList<>();
+ if (distBelow1.sample() > 1) {
+ String variableExpr = String.format("%s.%s", alias, ArrayDataset.ADDITIONAL_FIELDS[0]);
+ String value = ArrayDataset.VALUES_FOR_ADDITIONAL_FIELDS[0];
+ extraConjuncts.add(new ArrayQuery.SimpleConjunct(variableExpr, value, "=", null));
+ }
+ if (distBelow1.sample() > 1) {
+ String variableExpr = String.format("%s.%s", alias, ArrayDataset.ADDITIONAL_FIELDS[1]);
+ String value = ArrayDataset.VALUES_FOR_ADDITIONAL_FIELDS[1];
+ extraConjuncts.add(new ArrayQuery.SimpleConjunct(variableExpr, value, "=", null));
+ }
+ return extraConjuncts;
+ }
+
+ @Override
+ public List<ArrayQuery.Conjunct> getExtraQuantifiersAndConjuncts(List<String> arraysToQuantify,
+ List<String> quantifiedVars, String alias) {
+ List<ArrayQuery.Conjunct> extraConjuncts = new ArrayList<>();
+ if (distBelow1.sample() > 1) {
+ int i = randomGenerator.nextInt(ArrayDataset.VALUES_FOR_ADDITIONAL_GROUPS.length);
+ String valueExpr = ArrayDataset.VALUES_FOR_ADDITIONAL_GROUPS[i];
+ arraysToQuantify.add(String.format("%s.%s", alias, ArrayDataset.ADDITIONAL_GROUPS[0]));
+ quantifiedVars.add("W1");
+ extraConjuncts.add(new ArrayQuery.SimpleConjunct("W1", valueExpr, "=", null));
+ }
+ if (distBelow1.sample() > 1) {
+ int i = randomGenerator.nextInt(ArrayDataset.VALUES_FOR_ADDITIONAL_GROUPS.length);
+ String valueExpr = ArrayDataset.VALUES_FOR_ADDITIONAL_GROUPS[i];
+ arraysToQuantify.add(String.format("%s.%s", alias, ArrayDataset.ADDITIONAL_GROUPS[1]));
+ quantifiedVars.add("W2");
+ extraConjuncts.add(new ArrayQuery.SimpleConjunct("W2", valueExpr, "=", null));
+ }
+ return extraConjuncts;
+ }
+ }
+
+ public ArrayIndex.Builder.ValueSupplier getCompleteArrayIndexValueSupplier() {
+ return new BaseArrayIndexValueSupplier();
+ }
+
+ public ValueSupplier getCompleteArrayQueryValueSupplier() {
+ return new BaseArrayQueryValueSupplier();
+ }
+
+ /**
+ * @return Return a supplier that generates queries that get picked up and index-accelerated by the optimizer.
+ */
+ public ValueSupplier getWorkingArrayQueryValueSupplier() {
+ return new BaseArrayQueryValueSupplier() {
+ @Override
+ public boolean getIsConditionOnUnnest() {
+ // TODO: This is to avoid specifying extra UNNESTs. Refer to ASTERIXDB-2962.
+ return true;
+ }
+
+ @Override
+ public boolean getIsConstantConditionWithJoins() {
+ // TODO: This is to avoid specifying extra conjuncts in the presence of joins.
+ return false;
+ }
+
+ @Override
+ public boolean getIsCrossProductOnProbes() {
+ // TODO: This is to avoid performing a cross product with probes. Refer to ASTERIXDB-2966.
+ return false;
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
index db7ca7d..766a9d4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
@@ -19,10 +19,11 @@
package org.apache.asterix.test.common;
+import static org.apache.asterix.api.http.server.QueryServiceRequestParameters.Parameter.CLIENT_ID;
+
import java.io.InputStream;
import java.net.URI;
import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
import java.util.BitSet;
import java.util.Iterator;
import java.util.List;
@@ -40,8 +41,6 @@
import org.apache.asterix.testframework.xml.TestCase;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.http.HttpResponse;
-import org.apache.http.client.methods.HttpUriRequest;
-import org.apache.http.client.methods.RequestBuilder;
import org.apache.hyracks.api.util.ExceptionUtils;
import org.junit.Assert;
@@ -55,8 +54,8 @@
Predicate<Integer> responseCodeValidator, boolean cancellable) throws Exception {
cancellable = cancellable && !containsClientContextID(str);
String clientContextId = UUID.randomUUID().toString();
- final List<TestCase.CompilationUnit.Parameter> newParams = cancellable
- ? upsertParam(params, "client_context_id", ParameterTypeEnum.STRING, clientContextId) : params;
+ final List<TestCase.CompilationUnit.Parameter> newParams =
+ cancellable ? upsertParam(params, CLIENT_ID.str(), ParameterTypeEnum.STRING, clientContextId) : params;
Callable<InputStream> query = () -> {
try {
return CancellationTestExecutor.super.executeQueryService(str, fmt, uri,
@@ -89,21 +88,10 @@
// Cancels a submitted query through the cancellation REST API.
private int cancelQuery(URI uri, List<TestCase.CompilationUnit.Parameter> params) throws Exception {
- HttpUriRequest method = constructDeleteMethodUrl(uri, params);
- HttpResponse response = executeHttpRequest(method);
+ HttpResponse response = executeHttpRequest(constructDeleteMethod(uri, params));
return response.getStatusLine().getStatusCode();
}
- // Constructs a HTTP DELETE request.
- private HttpUriRequest constructDeleteMethodUrl(URI uri, List<TestCase.CompilationUnit.Parameter> otherParams) {
- RequestBuilder builder = RequestBuilder.delete(uri);
- for (TestCase.CompilationUnit.Parameter param : otherParams) {
- builder.addParameter(param.getName(), param.getValue());
- }
- builder.setCharset(StandardCharsets.UTF_8);
- return builder.build();
- }
-
@Override
protected boolean isUnExpected(Exception e, List<String> expectedErrors, int numOfErrors, MutableInt queryCount,
boolean expectedSourceLoc) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
index a1ed12b..8ba745b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
@@ -19,7 +19,6 @@
package org.apache.asterix.test.common;
import java.io.File;
-import java.util.BitSet;
import java.util.List;
import java.util.Map;
@@ -32,8 +31,7 @@
/**
* Execute the poll task
- *
- * @param testCaseCtx
+ * @param testCaseCtx
* @param ctx
* @param variableCtx
* @param statement
@@ -44,11 +42,9 @@
* @param expectedResultFileCtxs
* @param testFile
* @param actualPath
- * @param expectedWarnings
*/
void execute(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx, String statement,
boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit, MutableInt queryCount,
- List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath, BitSet expectedWarnings)
- throws Exception;
+ List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath) throws Exception;
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/RebalanceCancellationTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/RebalanceCancellationTestExecutor.java
index d35bfb7..758ba6f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/RebalanceCancellationTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/RebalanceCancellationTestExecutor.java
@@ -21,7 +21,6 @@
import java.io.File;
import java.net.URI;
-import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -35,8 +34,6 @@
import org.apache.asterix.testframework.xml.TestCase;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.http.HttpResponse;
-import org.apache.http.client.methods.HttpUriRequest;
-import org.apache.http.client.methods.RequestBuilder;
import org.apache.hyracks.api.util.ExceptionUtils;
import org.apache.logging.log4j.Level;
import org.junit.Assert;
@@ -102,18 +99,7 @@
// Cancels a submitted query through the cancellation REST API.
private int cancelQuery(URI uri, List<TestCase.CompilationUnit.Parameter> params) throws Exception {
- HttpUriRequest method = constructDeleteMethodUrl(uri, params);
- HttpResponse response = executeHttpRequest(method);
+ HttpResponse response = executeHttpRequest(constructDeleteMethod(uri, params));
return response.getStatusLine().getStatusCode();
}
-
- // Constructs a HTTP DELETE request.
- private HttpUriRequest constructDeleteMethodUrl(URI uri, List<TestCase.CompilationUnit.Parameter> otherParams) {
- RequestBuilder builder = RequestBuilder.delete(uri);
- for (TestCase.CompilationUnit.Parameter param : otherParams) {
- builder.addParameter(param.getName(), param.getValue());
- }
- builder.setCharset(StandardCharsets.UTF_8);
- return builder.build();
- }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index 5ab5b01..f83ddb2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -173,6 +173,10 @@
return extract(resultStream, EnumSet.of(ResultField.PLANS), resultCharset, OutputFormat.ADM, plans).getResult();
}
+ public static InputStream extractSignature(InputStream resultStream, Charset resultCharset) throws Exception {
+ return extract(resultStream, EnumSet.of(ResultField.SIGNATURE), resultCharset).getResult();
+ }
+
public static InputStream extractStatus(InputStream resultStream, Charset resultCharset) throws Exception {
return extract(resultStream, EnumSet.of(ResultField.STATUS), resultCharset).getResult();
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
index 86bffd2..f721aab 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestConstants.java
@@ -19,7 +19,7 @@
package org.apache.asterix.test.common;
public class TestConstants {
- // AWS S3 constants and place holders
+ // AWS S3 constants and placeholders
public static final String S3_ACCESS_KEY_ID_PLACEHOLDER = "%accessKeyId%";
public static final String S3_ACCESS_KEY_ID_DEFAULT = "dummyAccessKey";
public static final String S3_SECRET_ACCESS_KEY_PLACEHOLDER = "%secretAccessKey%";
@@ -35,42 +35,47 @@
+ "(\"secretAccessKey\"=\"" + S3_SECRET_ACCESS_KEY_DEFAULT + "\"),\n" + "(\"region\"=\"" + S3_REGION_DEFAULT
+ "\"),\n" + "(\"serviceEndpoint\"=\"" + S3_SERVICE_ENDPOINT_DEFAULT + "\")";
- // Azure blob storage constants and place holders
- // account name
- public static final String AZURE_ACCOUNT_NAME_PLACEHOLDER = "%azureblob-accountname%";
- public static final String AZURE_AZURITE_ACCOUNT_NAME_DEFAULT = "devstoreaccount1";
+ // Azure blob storage constants and placeholders
+ public static class Azure {
+ // account name
+ public static final String ACCOUNT_NAME_PLACEHOLDER = "%azure-accountname%";
+ public static final String AZURITE_ACCOUNT_NAME_DEFAULT = "devstoreaccount1";
- // account key
- public static final String AZURE_ACCOUNT_KEY_PLACEHOLDER = "%azureblob-accountkey%";
- public static final String AZURE_AZURITE_ACCOUNT_KEY_DEFAULT =
- "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==";
+ // account key
+ public static final String ACCOUNT_KEY_PLACEHOLDER = "%azure-accountkey%";
+ public static final String AZURITE_ACCOUNT_KEY_DEFAULT =
+ "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==";
- // SAS token: this is generated and assigned at runtime at the start of the test
- public static final String AZURE_SAS_TOKEN_PLACEHOLDER = "%azureblob-sas%";
- public static String sasToken = "";
+ // SAS token: this is generated and assigned at runtime at the start of the test
+ public static final String SAS_TOKEN_PLACEHOLDER = "%azure-sas%";
+ public static String sasToken = "";
- // blob endpoint
- public static final String AZURE_BLOB_ENDPOINT_PLACEHOLDER = "%azureblob-endpoint%";
- public static final String AZURE_BLOB_ENDPOINT_DEFAULT =
- "http://localhost:20000/" + AZURE_AZURITE_ACCOUNT_NAME_DEFAULT;
+ // blob endpoint
+ public static final String BLOB_ENDPOINT_PLACEHOLDER = "%azure-endpoint%";
+ public static final String BLOB_ENDPOINT_DEFAULT = "http://localhost:10000/" + AZURITE_ACCOUNT_NAME_DEFAULT;
- // connection string with account name & account key
- public static final String AZURE_CONNECTION_STRING_ACCOUNT_KEY_PLACEHOLDER =
- "%azureblob-connectionstringaccountkey%";
- public static final String AZURE_CONNECTION_STRING_ACCOUNT_KEY = "AccountName=" + AZURE_ACCOUNT_NAME_PLACEHOLDER
- + ";AccountKey=" + AZURE_ACCOUNT_KEY_PLACEHOLDER + ";BlobEndpoint=" + AZURE_BLOB_ENDPOINT_PLACEHOLDER;
+ public static final String MANAGED_IDENTITY_ID_PLACEHOLDER = "%azure-managedidentityid%";
+ public static final String MANAGED_IDENTITY_ID_DEFAULT = "myManagedIdentityId";
- // connection string with account name & sas token
- public static final String AZURE_CONNECTION_STRING_SAS_TOKEN_PLACEHOLDER = "%azureblob-connectionstringsas%";
- public static final String AZURE_CONNECTION_STRING_SAS_TOKEN =
- "AccountName=" + AZURE_ACCOUNT_NAME_PLACEHOLDER + ";SharedAccessSignature=" + AZURE_SAS_TOKEN_PLACEHOLDER
- + ";BlobEndpoint=" + AZURE_BLOB_ENDPOINT_PLACEHOLDER;
+ public static final String CLIENT_ID_PLACEHOLDER = "%azure-clientid%";
+ public static final String CLIENT_ID_DEFAULT = "myClientId";
- // azure template and default template
- public static final String AZURE_TEMPLATE = "(\"accountName\"=\"" + AZURE_AZURITE_ACCOUNT_NAME_DEFAULT + "\"),\n"
- + "(\"accountKey\"=\"" + AZURE_AZURITE_ACCOUNT_KEY_DEFAULT + "\"),\n" + "(\"blobEndpoint\"=\""
- + AZURE_BLOB_ENDPOINT_PLACEHOLDER + "\")";
- public static final String AZURE_TEMPLATE_DEFAULT = "(\"accountName\"=\"" + AZURE_AZURITE_ACCOUNT_NAME_DEFAULT
- + "\"),\n" + "(\"accountKey\"=\"" + AZURE_AZURITE_ACCOUNT_KEY_DEFAULT + "\"),\n" + "(\"blobEndpoint\"=\""
- + AZURE_BLOB_ENDPOINT_DEFAULT + "\")";
+ public static final String CLIENT_SECRET_PLACEHOLDER = "%azure-clientsecret%";
+ public static final String CLIENT_SECRET_DEFAULT = "myClientSecret";
+
+ public static final String CLIENT_CERTIFICATE_PLACEHOLDER = "%azure-clientcertificate%";
+ public static final String CLIENT_CERTIFICATE_DEFAULT = "myClientCertificate";
+
+ public static final String CLIENT_CERTIFICATE_PASSWORD_PLACEHOLDER = "%azure-clientcertificatepassword%";
+ public static final String CLIENT_CERTIFICATE_PASSWORD_DEFAULT = "myClientCertificatePassword";
+
+ public static final String TENANT_ID_PLACEHOLDER = "%azure-tenantid%";
+ public static final String TENANT_ID_DEFAULT = "myTenantId";
+
+ // azure template and default template
+ public static final String TEMPLATE = "(\"accountName\"=\"" + AZURITE_ACCOUNT_NAME_DEFAULT + "\"),\n"
+ + "(\"accountKey\"=\"" + AZURITE_ACCOUNT_KEY_DEFAULT + "\"),\n" + "(\"endpoint\"=\""
+ + BLOB_ENDPOINT_PLACEHOLDER + "\")";
+ public static final String TEMPLATE_DEFAULT = TEMPLATE;
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 40946d3..e2f50d3 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -19,6 +19,28 @@
package org.apache.asterix.test.common;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.asterix.test.common.TestConstants.Azure.ACCOUNT_KEY_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.ACCOUNT_NAME_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.AZURITE_ACCOUNT_KEY_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.AZURITE_ACCOUNT_NAME_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.BLOB_ENDPOINT_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.BLOB_ENDPOINT_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_CERTIFICATE_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_CERTIFICATE_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_ID_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_ID_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_SECRET_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.CLIENT_SECRET_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.MANAGED_IDENTITY_ID_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.MANAGED_IDENTITY_ID_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.SAS_TOKEN_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.TEMPLATE;
+import static org.apache.asterix.test.common.TestConstants.Azure.TEMPLATE_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.TENANT_ID_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.TENANT_ID_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.sasToken;
import static org.apache.hyracks.util.NetworkUtil.toHostPort;
import static org.apache.hyracks.util.file.FileUtil.canonicalize;
@@ -81,6 +103,7 @@
import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
import org.apache.asterix.app.external.IExternalUDFLibrarian;
import org.apache.asterix.common.api.Duration;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.common.utils.Servlets;
@@ -106,6 +129,7 @@
import org.apache.commons.io.output.ByteArrayOutputStream;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.http.Consts;
import org.apache.http.HttpResponse;
import org.apache.http.HttpStatus;
import org.apache.http.NameValuePair;
@@ -114,6 +138,7 @@
import org.apache.http.client.AuthCache;
import org.apache.http.client.CredentialsProvider;
import org.apache.http.client.HttpClient;
+import org.apache.http.client.entity.UrlEncodedFormEntity;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.client.methods.HttpUriRequest;
import org.apache.http.client.methods.RequestBuilder;
@@ -130,6 +155,7 @@
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
+import org.apache.http.message.BasicNameValuePair;
import org.apache.http.protocol.HttpContext;
import org.apache.http.util.EntityUtils;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -150,6 +176,8 @@
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.fasterxml.jackson.databind.util.RawValue;
+import io.netty.handler.codec.http.HttpMethod;
+
public class TestExecutor {
/*
@@ -216,6 +244,7 @@
private static final String METRICS_QUERY_TYPE = "metrics";
private static final String PROFILE_QUERY_TYPE = "profile";
private static final String PLANS_QUERY_TYPE = "plans";
+ private static final String SIGNATURE_QUERY_TYPE = "signature";
private static final HashMap<Integer, ITestServer> runningTestServers = new HashMap<>();
private static Map<String, InetSocketAddress> ncEndPoints;
@@ -841,7 +870,8 @@
}
private static boolean setFormatInAccept(OutputFormat fmt) {
- return fmt == OutputFormat.LOSSLESS_JSON || fmt == OutputFormat.CSV_HEADER;
+ return fmt == OutputFormat.LOSSLESS_JSON || fmt == OutputFormat.LOSSLESS_ADM_JSON
+ || fmt == OutputFormat.CSV_HEADER;
}
public void setAvailableCharsets(Charset... charsets) {
@@ -905,20 +935,6 @@
return result;
}
- private HttpUriRequest constructHttpMethod(String statement, URI uri, String stmtParam, boolean postStmtAsParam,
- List<Parameter> otherParams) {
- String stmtParamName = (postStmtAsParam ? stmtParam : null);
- return constructPostMethodUrl(statement, uri, stmtParamName, otherParams);
- }
-
- private HttpUriRequest constructGetMethod(URI endpoint, List<Parameter> params) {
- RequestBuilder builder = RequestBuilder.get(endpoint);
- for (Parameter param : params) {
- builder.addParameter(param.getName(), param.getValue());
- }
- return builder.build();
- }
-
private boolean isMultipart(Parameter p) {
return p != null && (ParameterTypeEnum.MULTIPART_TEXT == p.getType()
|| ParameterTypeEnum.MULTIPART_BINARY == p.getType());
@@ -942,16 +958,30 @@
|| p.getType() == ParameterTypeEnum.MULTIPART_TEXT)
? Optional.of(MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT))
: Optional.empty();
+ List<NameValuePair> parameters = new ArrayList<>();
for (Parameter param : params) {
if (isMultipart(param)) {
addMultipart(mPartBuilder.get(), param);
} else {
- builder.addParameter(param.getName(), param.getValue());
+ parameters.add(new BasicNameValuePair(param.getName(), param.getValue()));
}
}
builder.setCharset(UTF_8);
mPartBuilder.ifPresent(mpb -> builder.setEntity(mpb.build()));
- body.ifPresent(s -> builder.setEntity(new StringEntity(s, contentType)));
+ if (body.isPresent()) {
+ builder.addParameters(parameters.toArray(new NameValuePair[0]));
+ builder.setEntity(new StringEntity(body.get(), contentType));
+ } else if (mPartBuilder.isPresent()) {
+ builder.addParameters(parameters.toArray(new NameValuePair[0]));
+ builder.setEntity(mPartBuilder.get().build());
+ } else {
+ boolean formParams = HttpUtil.ignoreQueryParameters(HttpMethod.valueOf(method));
+ if (formParams) {
+ builder.setEntity(new UrlEncodedFormEntity(parameters, Consts.UTF_8));
+ } else {
+ builder.addParameters(parameters.toArray(new NameValuePair[0]));
+ }
+ }
return builder.build();
}
@@ -970,6 +1000,27 @@
return method;
}
+ public static HttpUriRequest constructGetMethod(URI endpoint, List<Parameter> params) {
+ RequestBuilder builder = RequestBuilder.get(endpoint);
+ for (Parameter param : params) {
+ builder.addParameter(param.getName(), param.getValue());
+ }
+ return builder.build();
+ }
+
+ public static HttpUriRequest constructDeleteMethod(URI uri, List<Parameter> params) {
+ List<NameValuePair> form = new ArrayList<>();
+ for (Parameter param : params) {
+ form.add(new BasicNameValuePair(param.getName(), param.getValue()));
+ }
+ return constructDeleteRequest(uri, form);
+ }
+
+ public static HttpUriRequest constructDeleteRequest(URI uri, List<NameValuePair> params) {
+ RequestBuilder builder = RequestBuilder.delete(uri);
+ return builder.setEntity(new UrlEncodedFormEntity(params, Consts.UTF_8)).setCharset(UTF_8).build();
+ }
+
private HttpUriRequest constructPostMethod(URI uri, List<Parameter> params) {
RequestBuilder builder = RequestBuilder.post(uri);
for (Parameter param : params) {
@@ -1041,12 +1092,6 @@
TEXT_PLAIN_UTF8);
}
- public InputStream executeJSON(OutputFormat fmt, String method, URI uri, Predicate<Integer> responseCodeValidator)
- throws Exception {
- return executeJSON(fmt, method, uri, Collections.emptyList(), responseCodeValidator, Optional.empty(),
- TEXT_PLAIN_UTF8);
- }
-
private InputStream executeJSON(OutputFormat fmt, String method, URI uri, List<Parameter> params,
Predicate<Integer> responseCodeValidator, Optional<String> body, ContentType contentType) throws Exception {
HttpUriRequest request = buildRequest(method, uri, fmt, params, body, contentType);
@@ -1143,8 +1188,8 @@
public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
- MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- BitSet expectedWarnings) throws Exception {
+ MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+ throws Exception {
InputStream resultStream;
File qbcFile;
boolean failed = false;
@@ -1152,7 +1197,7 @@
switch (ctx.getType()) {
case "ddl":
ExtractedResult ddlExtractedResult = executeSqlppUpdateOrDdl(statement, OutputFormat.CLEAN_JSON, cUnit);
- validateWarning(ddlExtractedResult, testCaseCtx, cUnit, testFile, expectedWarnings);
+ validateWarning(ddlExtractedResult, testCaseCtx, cUnit, testFile);
break;
case "update":
// isDmlRecoveryTest: set IP address
@@ -1166,11 +1211,11 @@
case "pollpost":
poll(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
expectedResultFileCtxs, testFile, actualPath, ctx.getType().substring("poll".length()),
- expectedWarnings, plainExecutor);
+ plainExecutor);
break;
case "polldynamic":
polldynamic(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
- expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+ expectedResultFileCtxs, testFile, actualPath);
break;
case "query":
case "async":
@@ -1179,6 +1224,7 @@
case "metrics":
case "profile":
case "plans":
+ case "signature":
// isDmlRecoveryTest: insert Crash and Recovery
if (isDmlRecoveryTest) {
executeScript(pb, pb.environment().get("SCRIPT_HOME") + File.separator + "dml_recovery"
@@ -1195,7 +1241,7 @@
variableCtx, ctx, expectedResultFile, actualResultFile, queryCount,
expectedResultFileCtxs.size(), cUnit.getParameter(), ComparisonEnum.TEXT);
- validateWarning(extractedResult, testCaseCtx, cUnit, testFile, expectedWarnings);
+ validateWarning(extractedResult, testCaseCtx, cUnit, testFile);
break;
case "store":
// This is a query that returns the expected output of a subsequent query
@@ -1466,25 +1512,24 @@
final String mimeReqType = extractHttpRequestType(statement);
final String saveResponseVar = getResultVariable(statement);
ContentType contentType = mimeReqType != null ? ContentType.create(mimeReqType, UTF_8) : TEXT_PLAIN_UTF8;
- if (!body.isPresent()) {
+ if (body.isEmpty()) {
body = getBodyFromReference(statement, variableCtx);
}
final Pair<String, String> credentials = extractCredentials(statement);
InputStream resultStream;
+ URI uri;
if ("http".equals(extension)) {
- if (credentials != null) {
- resultStream = executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body,
- contentType, credentials);
- } else {
- resultStream =
- executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body, contentType);
- }
+ uri = createEndpointURI(variablesReplaced);
} else if ("uri".equals(extension)) {
- resultStream = executeURI(reqType, URI.create(variablesReplaced), fmt, params, statusCodePredicate, body,
- contentType);
+ uri = URI.create(variablesReplaced);
} else {
throw new IllegalArgumentException("Unexpected format for method " + reqType + ": " + extension);
}
+ if (credentials != null) {
+ resultStream = executeURI(reqType, uri, fmt, params, statusCodePredicate, body, contentType, credentials);
+ } else {
+ resultStream = executeURI(reqType, uri, fmt, params, statusCodePredicate, body, contentType);
+ }
if (extracResult) {
resultStream = ResultExtractor.extract(resultStream, UTF_8).getResult();
} else if (extractStatus) {
@@ -1505,8 +1550,8 @@
LOGGER.info("Diagnostic output: {}", IOUtils.toString(resultStream, UTF_8));
} else {
LOGGER.info("Unexpected output: {}", IOUtils.toString(resultStream, UTF_8));
- Assert.fail("no result file for " + testFile.toString() + "; queryCount: " + queryCount
- + ", filectxs.size: " + numResultFiles);
+ Assert.fail("no result file for " + testFile + "; queryCount: " + queryCount + ", filectxs.size: "
+ + numResultFiles);
}
} else {
writeOutputToFile(actualResultFile, resultStream);
@@ -1554,6 +1599,9 @@
String[] plans = plans(statement);
resultStream = ResultExtractor.extractPlans(resultStream, responseCharset, plans);
break;
+ case SIGNATURE_QUERY_TYPE:
+ resultStream = ResultExtractor.extractSignature(resultStream, responseCharset);
+ break;
default:
extractedResult = ResultExtractor.extract(resultStream, responseCharset, fmt);
resultStream = extractedResult.getResult();
@@ -1610,18 +1658,18 @@
private void polldynamic(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
- MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- BitSet expectedWarnings) throws Exception {
+ MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+ throws Exception {
IExpectedResultPoller poller = getExpectedResultPoller(statement);
final String key = getKey(statement);
poll(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount, expectedResultFileCtxs,
- testFile, actualPath, "validate", expectedWarnings, new IPollTask() {
+ testFile, actualPath, "validate", new IPollTask() {
@Override
public void execute(TestCaseContext testCaseCtx, TestFileContext ctx,
Map<String, Object> variableCtx, String statement, boolean isDmlRecoveryTest,
ProcessBuilder pb, CompilationUnit cUnit, MutableInt queryCount,
- List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- BitSet expectedWarnings) throws Exception {
+ List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+ throws Exception {
File actualResultFile = new File(actualPath, testCaseCtx.getTestCase().getFilePath()
+ File.separatorChar + cUnit.getName() + '.' + ctx.getSeqNum() + ".polled.adm");
if (actualResultFile.exists() && !actualResultFile.delete()) {
@@ -1659,7 +1707,7 @@
private void poll(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- String newType, BitSet expectedWarnings, IPollTask pollTask) throws Exception {
+ String newType, IPollTask pollTask) throws Exception {
// polltimeoutsecs=nnn, polldelaysecs=nnn
int timeoutSecs = getTimeoutSecs(statement);
int retryDelaySecs = getRetryDelaySecs(statement);
@@ -1682,7 +1730,7 @@
try {
startSemaphore.release();
pollTask.execute(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
- queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+ queryCount, expectedResultFileCtxs, testFile, actualPath);
} finally {
endSemaphore.release();
}
@@ -1960,9 +2008,8 @@
}
public static Pair<String, String> extractCredentials(String statement) {
- List<Parameter> params = new ArrayList<>();
final Matcher m = HTTP_AUTH_PATTERN.matcher(statement);
- while (m.find()) {
+ if (m.find()) {
String username = m.group("username");
String password = m.group("password");
return new Pair<>(username, password);
@@ -2013,23 +2060,6 @@
}
}
- protected InputStream executeHttp(String ctxType, String endpoint, OutputFormat fmt, List<Parameter> params,
- Predicate<Integer> statusCodePredicate, Optional<String> body, ContentType contentType) throws Exception {
- URI uri = createEndpointURI(endpoint);
- return executeURI(ctxType, uri, fmt, params, statusCodePredicate, body, contentType);
- }
-
- private InputStream executeHttp(String ctxType, String endpoint, OutputFormat fmt, List<Parameter> params,
- Predicate<Integer> statusCodePredicate, Optional<String> body, ContentType contentType,
- Pair<String, String> credentials) throws Exception {
- URI uri = createEndpointURI(endpoint);
- return executeURI(ctxType, uri, fmt, params, statusCodePredicate, body, contentType, credentials);
- }
-
- private InputStream executeURI(String ctxType, URI uri, OutputFormat fmt, List<Parameter> params) throws Exception {
- return executeJSON(fmt, ctxType.toUpperCase(), uri, params);
- }
-
private InputStream executeURI(String ctxType, URI uri, OutputFormat fmt, List<Parameter> params,
Predicate<Integer> responseCodeValidator, Optional<String> body, ContentType contentType) throws Exception {
return executeJSON(fmt, ctxType.toUpperCase(), uri, params, responseCodeValidator, body, contentType);
@@ -2092,13 +2122,13 @@
public void executeTest(String actualPath, TestCaseContext testCaseCtx, ProcessBuilder pb,
boolean isDmlRecoveryTest, TestGroup failedGroup, TestGroup passedGroup) throws Exception {
MutableInt queryCount = new MutableInt(0);
- int numOfErrors = 0;
+ testCaseCtx.numOfErrors = 0;
int numOfFiles = 0;
List<CompilationUnit> cUnits = testCaseCtx.getTestCase().getCompilationUnit();
for (CompilationUnit cUnit : cUnits) {
- List<String> expectedErrors = cUnit.getExpectedError();
- BitSet expectedWarnings = new BitSet(cUnit.getExpectedWarn().size());
- expectedWarnings.set(0, cUnit.getExpectedWarn().size());
+ testCaseCtx.expectedErrors = cUnit.getExpectedError();
+ testCaseCtx.expectedWarnings = new BitSet(cUnit.getExpectedWarn().size());
+ testCaseCtx.expectedWarnings.set(0, cUnit.getExpectedWarn().size());
LOGGER.info(
"Starting [TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName() + " ... ");
Map<String, Object> variableCtx = new HashMap<>();
@@ -2120,7 +2150,7 @@
boolean loopCmd = testFile.getName().endsWith(".loop.cmd");
if (!testFile.getName().startsWith(DIAGNOSE)) {
executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
- queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+ queryCount, expectedResultFileCtxs, testFile, actualPath);
}
if (loopCmd) {
// this was a loop file and we have exited the loop; reset the loop iteration
@@ -2138,9 +2168,9 @@
}
loopIteration++;
} catch (Exception e) {
- numOfErrors++;
- boolean unexpected = isUnExpected(e, expectedErrors, numOfErrors, queryCount,
- testCaseCtx.isSourceLocationExpected(cUnit));
+ testCaseCtx.numOfErrors++;
+ boolean unexpected = isUnExpected(e, testCaseCtx.expectedErrors, testCaseCtx.numOfErrors,
+ queryCount, testCaseCtx.isSourceLocationExpected(cUnit));
if (unexpected) {
LOGGER.error("testFile {} raised an unexpected exception", testFile, e);
if (failedGroup != null) {
@@ -2152,12 +2182,12 @@
}
}
if (numOfFiles == testFileCtxs.size()) {
- if (numOfErrors < cUnit.getExpectedError().size()) {
+ if (testCaseCtx.numOfErrors < cUnit.getExpectedError().size()) {
LOGGER.error("Test {} failed to raise (an) expected exception(s)", cUnit.getName());
throw new Exception(
"Test \"" + cUnit.getName() + "\" FAILED; expected exception was not thrown...");
}
- ensureWarnings(expectedWarnings, cUnit);
+ ensureWarnings(testCaseCtx.expectedWarnings, cUnit);
LOGGER.info(
"[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName() + " PASSED ");
if (passedGroup != null) {
@@ -2300,21 +2330,22 @@
}
// This replaces specific external dataset placeholders
- str = str.replace(TestConstants.AZURE_CONNECTION_STRING_ACCOUNT_KEY_PLACEHOLDER,
- TestConstants.AZURE_CONNECTION_STRING_ACCOUNT_KEY);
- str = str.replace(TestConstants.AZURE_CONNECTION_STRING_SAS_TOKEN_PLACEHOLDER,
- TestConstants.AZURE_CONNECTION_STRING_SAS_TOKEN);
- str = str.replace(TestConstants.AZURE_ACCOUNT_NAME_PLACEHOLDER,
- TestConstants.AZURE_AZURITE_ACCOUNT_NAME_DEFAULT);
- str = str.replace(TestConstants.AZURE_ACCOUNT_KEY_PLACEHOLDER, TestConstants.AZURE_AZURITE_ACCOUNT_KEY_DEFAULT);
- str = str.replace(TestConstants.AZURE_SAS_TOKEN_PLACEHOLDER, TestConstants.sasToken);
+ str = str.replace(ACCOUNT_NAME_PLACEHOLDER, AZURITE_ACCOUNT_NAME_DEFAULT);
+ str = str.replace(ACCOUNT_KEY_PLACEHOLDER, AZURITE_ACCOUNT_KEY_DEFAULT);
+ str = str.replace(SAS_TOKEN_PLACEHOLDER, sasToken);
+ str = str.replace(MANAGED_IDENTITY_ID_PLACEHOLDER, MANAGED_IDENTITY_ID_DEFAULT);
+ str = str.replace(CLIENT_ID_PLACEHOLDER, CLIENT_ID_DEFAULT);
+ str = str.replace(CLIENT_SECRET_PLACEHOLDER, CLIENT_SECRET_DEFAULT);
+ str = str.replace(CLIENT_CERTIFICATE_PLACEHOLDER, CLIENT_CERTIFICATE_DEFAULT);
+ str = str.replace(CLIENT_CERTIFICATE_PASSWORD_PLACEHOLDER, CLIENT_CERTIFICATE_PASSWORD_DEFAULT);
+ str = str.replace(TENANT_ID_PLACEHOLDER, TENANT_ID_DEFAULT);
str = replaceExternalEndpoint(str);
return str;
}
protected String replaceExternalEndpoint(String str) {
- return str.replace(TestConstants.AZURE_BLOB_ENDPOINT_PLACEHOLDER, TestConstants.AZURE_BLOB_ENDPOINT_DEFAULT);
+ return str.replace(BLOB_ENDPOINT_PLACEHOLDER, BLOB_ENDPOINT_DEFAULT);
}
protected boolean noTemplateRequired(String str) {
@@ -2374,17 +2405,17 @@
protected String applyAzureSubstitution(String str, List<Placeholder> placeholders) {
boolean isReplaced = false;
- boolean hasBlobEndpoint = false;
+ boolean hasEndpoint = false;
for (Placeholder placeholder : placeholders) {
// Stop if all parameters are met
- if (hasBlobEndpoint) {
+ if (hasEndpoint) {
break;
- } else if (placeholder.getName().equals("blobEndpoint")) {
- hasBlobEndpoint = true;
+ } else if (placeholder.getName().equals("endpoint")) {
+ hasEndpoint = true;
isReplaced = true;
str = setAzureTemplate(str);
- str = str.replace(TestConstants.AZURE_BLOB_ENDPOINT_PLACEHOLDER, placeholder.getValue());
+ str = str.replace(BLOB_ENDPOINT_PLACEHOLDER, placeholder.getValue());
}
}
@@ -2397,11 +2428,11 @@
}
protected String setAzureTemplate(String str) {
- return str.replace("%template%", TestConstants.AZURE_TEMPLATE);
+ return str.replace("%template%", TEMPLATE);
}
protected String setAzureTemplateDefault(String str) {
- return str.replace("%template%", TestConstants.AZURE_TEMPLATE_DEFAULT);
+ return str.replace("%template%", TEMPLATE_DEFAULT);
}
protected void fail(boolean runDiagnostics, TestCaseContext testCaseCtx, CompilationUnit cUnit,
@@ -2416,7 +2447,7 @@
final File file = ctx.getFile();
final String statement = readTestFile(file);
executeTestFile(testCaseCtx, ctx, variableCtx, statement, false, pb, cUnit, new MutableInt(-1),
- Collections.emptyList(), file, null, new BitSet());
+ Collections.emptyList(), file, null);
}
}
} catch (Exception diagnosticFailure) {
@@ -2540,32 +2571,12 @@
public void cleanup(String testCase, List<String> badtestcases) throws Exception {
try {
List<DataverseName> toBeDropped = new ArrayList<>();
- InputStream resultStream = executeQueryService(
- "select dv.DataverseName from Metadata.`Dataverse` as dv order by dv.DataverseName;",
- getEndpoint(Servlets.QUERY_SERVICE), OutputFormat.CLEAN_JSON);
- JsonNode result = extractResult(IOUtils.toString(resultStream, UTF_8));
- for (int i = 0; i < result.size(); i++) {
- JsonNode json = result.get(i);
- if (json != null) {
- DataverseName dvName = DataverseName.createFromCanonicalForm(json.get("DataverseName").asText());
- if (!dvName.equals(MetadataConstants.METADATA_DATAVERSE_NAME)
- && !dvName.equals(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME)) {
- toBeDropped.add(dvName);
- }
- }
- }
+ listUserDefinedDataverses(toBeDropped);
if (!toBeDropped.isEmpty()) {
badtestcases.add(testCase);
LOGGER.info("Last test left some garbage. Dropping dataverses: " + StringUtils.join(toBeDropped, ','));
- StringBuilder dropStatement = new StringBuilder();
for (DataverseName dv : toBeDropped) {
- dropStatement.setLength(0);
- dropStatement.append("drop dataverse ");
- SqlppStatementUtil.encloseDataverseName(dropStatement, dv);
- dropStatement.append(";\n");
- resultStream = executeQueryService(dropStatement.toString(), getEndpoint(Servlets.QUERY_SERVICE),
- OutputFormat.CLEAN_JSON, UTF_8);
- ResultExtractor.extract(resultStream, UTF_8, OutputFormat.CLEAN_JSON);
+ dropDataverse(dv);
}
}
} catch (Throwable th) {
@@ -2574,6 +2585,54 @@
}
}
+ protected void listUserDefinedDataverses(List<DataverseName> outDataverses) throws Exception {
+ String query = "select dv.DataverseName from Metadata.`Dataverse` as dv order by dv.DataverseName";
+ InputStream resultStream =
+ executeQueryService(query, getEndpoint(Servlets.QUERY_SERVICE), OutputFormat.CLEAN_JSON);
+ JsonNode result = extractResult(IOUtils.toString(resultStream, UTF_8));
+ for (int i = 0; i < result.size(); i++) {
+ JsonNode json = result.get(i);
+ if (json != null) {
+ DataverseName dvName = DataverseName.createFromCanonicalForm(json.get("DataverseName").asText());
+ if (!dvName.equals(MetadataConstants.METADATA_DATAVERSE_NAME)
+ && !dvName.equals(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME)) {
+ outDataverses.add(dvName);
+ }
+ }
+ }
+ }
+
+ protected void dropDataverse(DataverseName dv) throws Exception {
+ StringBuilder dropStatement = new StringBuilder();
+ dropStatement.append("drop dataverse ");
+ SqlppStatementUtil.encloseDataverseName(dropStatement, dv);
+ dropStatement.append(";\n");
+ InputStream resultStream = executeQueryService(dropStatement.toString(), getEndpoint(Servlets.QUERY_SERVICE),
+ OutputFormat.CLEAN_JSON, UTF_8);
+ ResultExtractor.extract(resultStream, UTF_8, OutputFormat.CLEAN_JSON);
+ }
+
+ protected void listDatasets(DataverseName dataverseName, List<Pair<String, DatasetConfig.DatasetType>> outDatasets)
+ throws Exception {
+ String query = "select d.DatasetName, d.DatasetType from Metadata.`Dataset` d where d.DataverseName = '"
+ + dataverseName.getCanonicalForm() + "'";
+ InputStream resultStream = executeQueryService(query, getEndpoint(Servlets.QUERY_SERVICE),
+ TestCaseContext.OutputFormat.CLEAN_JSON);
+ JsonNode result = extractResult(IOUtils.toString(resultStream, UTF_8));
+ for (int i = 0; i < result.size(); i++) {
+ JsonNode json = result.get(i);
+ String datasetName = json.get("DatasetName").asText();
+ String datasetTypeText = json.get("DatasetType").asText();
+ DatasetConfig.DatasetType datasetType;
+ try {
+ datasetType = DatasetConfig.DatasetType.valueOf(datasetTypeText);
+ } catch (IllegalArgumentException e) {
+ throw new Exception("Unexpected dataset type: " + datasetTypeText);
+ }
+ outDatasets.add(new Pair<>(datasetName, datasetType));
+ }
+ }
+
private JsonNode extractResult(String jsonString) throws IOException {
try {
final JsonNode result = RESULT_NODE_READER.<ObjectNode> readValue(jsonString).get("results");
@@ -2655,8 +2714,8 @@
String partition = command[2];
String destNode = command[3];
final InetSocketAddress destAddress = getNcReplicationAddress(destNode);
- List<Parameter> parameters = new ArrayList<>(3);
- Stream.of("partition", "host", "port").forEach(arg -> {
+ List<Parameter> parameters = new ArrayList<>(4);
+ Stream.of("partition", "host", "port", "nodeId").forEach(arg -> {
Parameter p = new Parameter();
p.setName(arg);
p.setType(ParameterTypeEnum.STRING);
@@ -2665,6 +2724,7 @@
parameters.get(0).setValue(partition);
parameters.get(1).setValue(destAddress.getHostName());
parameters.get(2).setValue(String.valueOf(destAddress.getPort()));
+ parameters.get(3).setValue(destNode);
final HttpUriRequest httpUriRequest = constructPostMethod(endpoint, parameters);
final HttpResponse httpResponse = executeHttpRequest(httpUriRequest);
Assert.assertEquals(HttpStatus.SC_OK, httpResponse.getStatusLine().getStatusCode());
@@ -2794,11 +2854,11 @@
}
protected void validateWarning(ExtractedResult result, TestCaseContext testCaseCtx, CompilationUnit cUnit,
- File testFile, BitSet expectedWarnings) throws Exception {
+ File testFile) throws Exception {
if (testCaseCtx.getTestCase().isCheckWarnings()) {
boolean expectedSourceLoc = testCaseCtx.isSourceLocationExpected(cUnit);
- validateWarnings(result.getWarnings(), cUnit.getExpectedWarn(), expectedWarnings, expectedSourceLoc,
- testFile);
+ validateWarnings(result.getWarnings(), cUnit.getExpectedWarn(), testCaseCtx.expectedWarnings,
+ expectedSourceLoc, testFile);
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
index 24ca072..fcab213 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
@@ -41,6 +41,7 @@
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.testframework.xml.ParameterTypeEnum;
import org.apache.asterix.testframework.xml.TestCase;
+import org.apache.asterix.translator.SessionConfig;
import org.apache.commons.compress.utils.IOUtils;
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.util.file.FileUtil;
@@ -164,7 +165,8 @@
}
}
- return RequestParameters.deserializeParameterValues(RequestParameters.serializeParameterValues(stmtParams));
+ return RequestParameters.deserializeParameterValues(
+ RequestParameters.serializeParameterValues(stmtParams, SessionConfig.OutputFormat.CLEAN_JSON));
}
public static boolean equalJson(JsonNode expectedJson, JsonNode actualJson, boolean compareUnorderedArray,
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
index 1913233..3973dee 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@ -76,6 +76,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.apache.hyracks.util.OptionalBoolean;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -171,7 +172,7 @@
partitioningKeys, null, null, null, false, null, null),
null, DatasetType.INTERNAL, DATASET_ID, 0);
secondaryIndex = new Index(dvName, DATASET_NAME, INDEX_NAME, INDEX_TYPE, INDEX_FIELD_NAMES,
- INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0);
+ INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0, OptionalBoolean.of(false));
taskCtx = null;
primaryIndexDataflowHelper = null;
secondaryIndexDataflowHelper = null;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java
index 76dd035..e99e5d6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ConcurrentInsertTest.java
@@ -51,6 +51,7 @@
import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.util.OptionalBoolean;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@@ -128,7 +129,8 @@
Index secondaryIndexEntity = new Index(StorageTestUtils.DATASET.getDataverseName(),
StorageTestUtils.DATASET.getDatasetName(), "TestIndex", IndexType.BTREE,
Arrays.asList(Arrays.asList(StorageTestUtils.RECORD_TYPE.getFieldNames()[1])),
- Arrays.asList(Index.RECORD_INDICATOR), Arrays.asList(BuiltinType.AINT64), false, false, false, 0);
+ Arrays.asList(Index.RECORD_INDICATOR), Arrays.asList(BuiltinType.AINT64), false, false, false, 0,
+ OptionalBoolean.of(false));
SecondaryIndexInfo secondaryIndexInfo =
nc.createSecondaryIndex(primaryIndexInfo, secondaryIndexEntity, StorageTestUtils.STORAGE_MANAGER, 0);
@@ -142,7 +144,7 @@
Index primaryKeyIndexEntity = new Index(StorageTestUtils.DATASET.getDataverseName(),
StorageTestUtils.DATASET.getDatasetName(), "PrimaryKeyIndex", IndexType.BTREE, Arrays.asList(),
- Arrays.asList(), Arrays.asList(), false, false, false, 0);
+ Arrays.asList(), Arrays.asList(), false, false, false, 0, OptionalBoolean.empty());
SecondaryIndexInfo primaryKeyIndexInfo =
nc.createSecondaryIndex(primaryIndexInfo, primaryKeyIndexEntity, StorageTestUtils.STORAGE_MANAGER, 0);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index 2aad416..d24d081 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -70,6 +70,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.hyracks.util.OptionalBoolean;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.junit.After;
@@ -187,7 +188,7 @@
dataset = StorageTestUtils.DATASET;
secondaryIndexEntity = new Index(dataset.getDataverseName(), dataset.getDatasetName(), SECONDARY_INDEX_NAME,
SECONDARY_INDEX_TYPE, SECONDARY_INDEX_FIELD_NAMES, SECONDARY_INDEX_FIELD_INDICATORS,
- SECONDARY_INDEX_FIELD_TYPES, false, false, false, 0);
+ SECONDARY_INDEX_FIELD_TYPES, false, false, false, 0, OptionalBoolean.of(false));
primaryIndexInfos = new PrimaryIndexInfo[NUM_PARTITIONS];
secondaryIndexInfo = new SecondaryIndexInfo[NUM_PARTITIONS];
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index ef3a7cb..6f9bed4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -70,6 +70,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.apache.hyracks.util.OptionalBoolean;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -161,7 +162,7 @@
partitioningKeys, null, null, null, false, null, null),
null, DatasetType.INTERNAL, DATASET_ID, 0);
secondaryIndex = new Index(dvName, DATASET_NAME, INDEX_NAME, INDEX_TYPE, INDEX_FIELD_NAMES,
- INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0);
+ INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0, OptionalBoolean.of(false));
taskCtxs = new IHyracksTaskContext[NUM_PARTITIONS];
primaryIndexDataflowHelpers = new IIndexDataflowHelper[NUM_PARTITIONS];
secondaryIndexDataflowHelpers = new IIndexDataflowHelper[NUM_PARTITIONS];
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
index 24379a3..c5292d3 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
@@ -19,7 +19,6 @@
package org.apache.asterix.test.dataflow;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@@ -45,6 +44,7 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.operators.LSMPrimaryInsertOperatorNodePushable;
+import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorNodePushable;
import org.apache.asterix.test.common.TestHelper;
import org.apache.asterix.test.dataflow.StorageTestUtils.Searcher;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -53,18 +53,28 @@
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils;
import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeSearchCursor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.MultiComparator;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -84,7 +94,7 @@
private static final boolean[] UNIQUE_META_FIELDS = null;
private static final int[] KEY_INDEXES = { 0 };
private static final int[] KEY_INDICATORS = { Index.RECORD_INDICATOR };
- private static final List<Integer> KEY_INDICATORS_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
+ private static final List<Integer> KEY_INDICATORS_LIST = List.of(Index.RECORD_INDICATOR);
private static final int TOTAL_NUM_OF_RECORDS = 2000;
private static final int RECORDS_PER_COMPONENT = 1000;
private static final int DATASET_ID = 101;
@@ -102,6 +112,7 @@
private static IIndexDataflowHelper indexDataflowHelper;
private static ITransactionContext txnCtx;
private static LSMPrimaryInsertOperatorNodePushable insertOp;
+ private static LSMPrimaryUpsertOperatorNodePushable upsertOp;
@BeforeClass
public static void setUp() throws Exception {
@@ -143,6 +154,8 @@
new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
KEY_INDICATORS_LIST, storageManager, null, null).getLeft();
+ upsertOp = nc.getUpsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+ KEY_INDICATORS_LIST, storageManager, null, false).getLeft();
}
@After
@@ -202,6 +215,63 @@
}
@Test
+ public void testCursorSwitchSucceedWithNoDuplicates() {
+ try {
+ StorageTestUtils.allowAllOps(lsmBtree);
+ lsmBtree.clearSearchCallbacks();
+ RecordTupleGenerator tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES,
+ KEY_INDICATORS, RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+ VSizeFrame frame = new VSizeFrame(ctx);
+ FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
+ int totalNumRecords = LSMIndexSearchCursor.SWITCH_COMPONENT_CYCLE + 2;
+ ITupleReference[] upsertTuples = new ITupleReference[totalNumRecords];
+ for (int j = 0; j < totalNumRecords; j++) {
+ ITupleReference tuple = tupleGenerator.next();
+ upsertTuples[j] = TupleUtils.copyTuple(tuple);
+ }
+
+ // upsert and flush the tuples to create a disk component
+ upsert(tupleAppender, totalNumRecords, upsertTuples, true);
+ // upsert but don't flush the tuples to create a memory component
+ upsert(tupleAppender, totalNumRecords, upsertTuples, false);
+
+ // do the search operation
+ ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(lsmBtree.getHarness(),
+ lsmBtree.createOpContext(NoOpIndexAccessParameters.INSTANCE), LSMBTreeSearchCursor::new);
+ IIndexCursor searchCursor = accessor.createSearchCursor(false);
+ MultiComparator lowKeySearchCmp =
+ BTreeUtils.getSearchMultiComparator(lsmBtree.getComparatorFactories(), null);
+ MultiComparator highKeySearchCmp =
+ BTreeUtils.getSearchMultiComparator(lsmBtree.getComparatorFactories(), null);
+ RangePredicate rangePredicate =
+ new RangePredicate(null, null, true, true, lowKeySearchCmp, highKeySearchCmp, null, null);
+
+ accessor.search(searchCursor, rangePredicate);
+
+ int count = 0;
+ while (searchCursor.hasNext()) {
+ searchCursor.next();
+ count++;
+ // flush the memory component to disk so that we make the switch to it when we hit the switch cycle
+ if (count == 1) {
+ StorageTestUtils.flush(dsLifecycleMgr, lsmBtree, dataset, false);
+ }
+ }
+
+ Throwable failure = ResourceReleaseUtils.close(searchCursor, null);
+ failure = CleanupUtils.destroy(failure, searchCursor, accessor);
+ Assert.assertEquals("Records count not matching", totalNumRecords, count);
+ if (failure != null) {
+ Assert.fail(failure.getMessage());
+ }
+ nc.getTransactionManager().commitTransaction(txnCtx.getTxnId());
+ } catch (Throwable e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ @Test
public void testCursorSwitchFails() {
try {
// allow all operations
@@ -268,4 +338,17 @@
emptyTupleOp.close();
Assert.assertEquals(numOfRecords, countOp.getCount());
}
+
+ private void upsert(FrameTupleAppender tupleAppender, int totalNumRecords, ITupleReference[] upsertTuples,
+ boolean flush) throws Exception {
+ upsertOp.open();
+ for (int j = 0; j < totalNumRecords; j++) {
+ DataflowUtils.addTupleToFrame(tupleAppender, upsertTuples[j], upsertOp);
+ }
+ tupleAppender.write(upsertOp, true);
+ if (flush) {
+ StorageTestUtils.flush(dsLifecycleMgr, lsmBtree, dataset, false);
+ }
+ upsertOp.close();
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 9818538..2ee8fcf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -111,7 +111,7 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
@@ -148,7 +148,7 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryCmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/BinaryFileConverterUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/BinaryFileConverterUtil.java
new file mode 100644
index 0000000..d3865d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/BinaryFileConverterUtil.java
@@ -0,0 +1,83 @@
+/*
+ * 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.test.external_dataset;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.hadoop.fs.Path;
+import org.apache.hyracks.api.util.IoUtil;
+import org.kitesdk.data.spi.JsonUtil;
+import org.kitesdk.data.spi.filesystem.JSONFileReader;
+
+import parquet.avro.AvroParquetWriter;
+
+public class BinaryFileConverterUtil {
+ public static final String DEFAULT_PARQUET_SRC_PATH = "data/hdfs/parquet";
+ public static final String BINARY_GEN_BASEDIR = "target" + File.separatorChar + "generated_bin_files";
+
+ //How many records should the schema inference method inspect to infer the schema for parquet files
+ private static final int NUM_OF_RECORDS_SCHEMA = 20;
+
+ private BinaryFileConverterUtil() {
+ }
+
+ public static void cleanBinaryDirectory(File localDataRoot, String binaryFilesPath) throws IOException {
+ File destPath = new File(localDataRoot, binaryFilesPath);
+ //Delete old generated files
+ if (destPath.exists()) {
+ IoUtil.delete(destPath);
+ }
+ //Create new directory
+ Files.createDirectory(Paths.get(destPath.getAbsolutePath()));
+ }
+
+ public static void convertToParquet(File localDataRoot, String src, String dest) throws IOException {
+ File srcPath = new File(localDataRoot, src);
+ File destPath = new File(localDataRoot, dest);
+
+ //Write parquet files
+ File[] listOfFiles = srcPath.listFiles();
+ for (File jsonFile : listOfFiles) {
+ String fileName = jsonFile.getName().substring(0, jsonFile.getName().indexOf(".")) + ".parquet";
+ Path outputPath = new Path(destPath.getAbsolutePath(), fileName);
+ writeParquetFile(jsonFile, outputPath);
+ }
+ }
+
+ private static void writeParquetFile(File jsonInputPath, Path parquetOutputPath) throws IOException {
+ final FileInputStream schemaInputStream = new FileInputStream(jsonInputPath);
+ final FileInputStream jsonInputStream = new FileInputStream(jsonInputPath);
+ //Infer Avro schema
+ final Schema inputSchema = JsonUtil.inferSchema(schemaInputStream, "parquet_schema", NUM_OF_RECORDS_SCHEMA);
+ try (JSONFileReader<Record> reader = new JSONFileReader<>(jsonInputStream, inputSchema, Record.class)) {
+ reader.initialize();
+ try (AvroParquetWriter<Record> writer = new AvroParquetWriter<>(parquetOutputPath, inputSchema)) {
+ for (Record record : reader) {
+ writer.write(record);
+ }
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
new file mode 100644
index 0000000..094c1db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
@@ -0,0 +1,409 @@
+/*
+ * 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.test.external_dataset;
+
+import static org.apache.asterix.test.external_dataset.BinaryFileConverterUtil.BINARY_GEN_BASEDIR;
+import static org.apache.asterix.test.external_dataset.aws.AwsS3ExternalDatasetTest.BOM_FILE_CONTAINER;
+import static org.apache.asterix.test.external_dataset.aws.AwsS3ExternalDatasetTest.FIXED_DATA_CONTAINER;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class ExternalDatasetTestUtils {
+
+ protected static final Logger LOGGER = LogManager.getLogger();
+
+ // Base directory paths for data files
+ private static String JSON_DATA_PATH;
+ private static String CSV_DATA_PATH;
+ private static String TSV_DATA_PATH;
+
+ // IMPORTANT: The following values must be used in the AWS S3 test case
+ // Region, container and definitions
+ public static final String JSON_DEFINITION = "json-data/reviews/";
+ public static final String CSV_DEFINITION = "csv-data/reviews/";
+ public static final String TSV_DEFINITION = "tsv-data/reviews/";
+ public static final String MIXED_DEFINITION = "mixed-data/reviews/";
+ public static final String PARQUET_DEFINITION = "parquet-data/reviews/";
+
+ // This is used for a test to generate over 1000 number of files
+ public static final String OVER_1000_OBJECTS_PATH = "over-1000-objects";
+ public static final int OVER_1000_OBJECTS_COUNT = 2999;
+
+ private static Uploader playgroundDataLoader;
+ private static Uploader fixedDataLoader;
+ private static Uploader mixedDataLoader;
+ private static Uploader bomFileLoader;
+
+ protected TestCaseContext tcCtx;
+
+ public interface Uploader {
+ default void upload(String key, String content) {
+ upload(key, content, false, false);
+ }
+
+ void upload(String key, String content, boolean fromFile, boolean gzipped);
+ }
+
+ public ExternalDatasetTestUtils(TestCaseContext tcCtx) {
+ this.tcCtx = tcCtx;
+ }
+
+ /**
+ * Generate binary files (e.g., parquet files)
+ */
+ public static void createBinaryFiles(String parquetRawJsonDir) throws IOException {
+ //base path
+ File basePath = new File(".");
+ //clean the binary generated files' directory
+ BinaryFileConverterUtil.cleanBinaryDirectory(basePath, BINARY_GEN_BASEDIR);
+ //Convert files in DEFAULT_PARQUET_SRC_PATH to parquet
+ BinaryFileConverterUtil.convertToParquet(basePath, parquetRawJsonDir, BINARY_GEN_BASEDIR);
+ }
+
+ public static void setDataPaths(String jsonDataPath, String csvDataPath, String tsvDataPath) {
+ JSON_DATA_PATH = jsonDataPath;
+ CSV_DATA_PATH = csvDataPath;
+ TSV_DATA_PATH = tsvDataPath;
+ }
+
+ public static void setUploaders(Uploader playgroundDataLoader, Uploader fixedDataLoader, Uploader mixedDataLoader,
+ Uploader bomFileLoader) {
+ ExternalDatasetTestUtils.playgroundDataLoader = playgroundDataLoader;
+ ExternalDatasetTestUtils.fixedDataLoader = fixedDataLoader;
+ ExternalDatasetTestUtils.mixedDataLoader = mixedDataLoader;
+ ExternalDatasetTestUtils.bomFileLoader = bomFileLoader;
+ }
+
+ /**
+ * Creates a bucket and fills it with some files for testing purpose.
+ */
+ public static void preparePlaygroundContainer() {
+ LOGGER.info("Adding JSON files to the bucket");
+ loadJsonFiles();
+ LOGGER.info("JSON Files added successfully");
+
+ LOGGER.info("Adding CSV files to the bucket");
+ loadCsvFiles();
+ LOGGER.info("CSV Files added successfully");
+
+ LOGGER.info("Adding TSV files to the bucket");
+ loadTsvFiles();
+ LOGGER.info("TSV Files added successfully");
+
+ LOGGER.info("Adding a big JSON file");
+ loadBigJson();
+ LOGGER.info("JSON file added successfully");
+
+ LOGGER.info("Loading " + OVER_1000_OBJECTS_COUNT + " into " + OVER_1000_OBJECTS_PATH);
+ loadLargeNumberOfFiles();
+ LOGGER.info("Added " + OVER_1000_OBJECTS_COUNT + " files into " + OVER_1000_OBJECTS_PATH + " successfully");
+
+ LOGGER.info("Adding Parquet files to the bucket");
+ loadParquetFiles();
+ LOGGER.info("Parquet files added successfully");
+
+ LOGGER.info("Files added successfully");
+ }
+
+ /**
+ * This bucket is being filled by fixed data, a test is counting all records in this bucket. If this bucket is
+ * changed, the test case will fail and its result will need to be updated each time
+ */
+ public static void prepareFixedDataContainer() {
+ LOGGER.info("Loading fixed data to " + FIXED_DATA_CONTAINER);
+
+ // Files data
+ String path = Paths.get(JSON_DATA_PATH, "single-line", "20-records.json").toString();
+ fixedDataLoader.upload("1.json", path, true, false);
+ fixedDataLoader.upload("2.json", path, true, false);
+ fixedDataLoader.upload("lvl1/3.json", path, true, false);
+ fixedDataLoader.upload("lvl1/34.json", path, true, false);
+ fixedDataLoader.upload("lvl1/lvl2/5.json", path, true, false);
+ }
+
+ /**
+ * This bucket contains files that start with byte order mark (BOM): U+FEFF
+ */
+ public static void prepareBomFileContainer() {
+ LOGGER.info("Loading bom files data to " + BOM_FILE_CONTAINER);
+
+ // Files data
+ bomFileLoader.upload("1.json", "\uFEFF{\"id\": 1, \"age\": 1}", false, false);
+ bomFileLoader.upload("2.json", "\uFEFF{\"id\": 2, \"age\": 2}", false, false);
+ bomFileLoader.upload("3.json", "\uFEFF{\"id\": 3, \"age\": 3}", false, false);
+ bomFileLoader.upload("4.json", "\uFEFF{\"id\": 4, \"age\": 4}", false, false);
+ bomFileLoader.upload("5.json", "\uFEFF{\"id\": 5, \"age\": 5}", false, false);
+ bomFileLoader.upload("1.csv", "\uFEFF1,1", false, false);
+ bomFileLoader.upload("2.csv", "\uFEFF2,2", false, false);
+ bomFileLoader.upload("3.csv", "\uFEFF3,3", false, false);
+ bomFileLoader.upload("4.csv", "\uFEFF4,4", false, false);
+ bomFileLoader.upload("5.csv", "\uFEFF5,5", false, false);
+ bomFileLoader.upload("1.tsv", "\uFEFF1\t1", false, false);
+ bomFileLoader.upload("2.tsv", "\uFEFF2\t2", false, false);
+ bomFileLoader.upload("3.tsv", "\uFEFF3\t3", false, false);
+ bomFileLoader.upload("4.tsv", "\uFEFF4\t4", false, false);
+ bomFileLoader.upload("5.tsv", "\uFEFF5\t5", false, false);
+ }
+
+ public static void loadJsonFiles() {
+ String dataBasePath = JSON_DATA_PATH;
+ String definition = JSON_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "json";
+ loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+
+ definitionSegment = "json-array-of-objects";
+ loadData(dataBasePath, "single-line", "array_of_objects.json", "json-data/", definitionSegment, false, false);
+
+ // gz compressed format
+ definitionSegment = "gz";
+ loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+
+ // Mixed normal and gz compressed format
+ definitionSegment = "mixed";
+ loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+ loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+ }
+
+ private static void loadCsvFiles() {
+ String dataBasePath = CSV_DATA_PATH;
+ String definition = CSV_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "csv";
+ loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+
+ // gz compressed format
+ definitionSegment = "gz";
+ loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+
+ // Mixed normal and gz compressed format
+ definitionSegment = "mixed";
+ loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+ }
+
+ private static void loadTsvFiles() {
+ String dataBasePath = TSV_DATA_PATH;
+ String definition = TSV_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "tsv";
+ loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+
+ // gz compressed format
+ definitionSegment = "gz";
+ loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+
+ // Mixed normal and gz compressed format
+ definitionSegment = "mixed";
+ loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+ }
+
+ private static void loadParquetFiles() {
+ String dataBasePath = BINARY_GEN_BASEDIR;
+ String definition = PARQUET_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "";
+ loadData(dataBasePath, "", "dummy_tweet.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_age.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_age-string.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_name.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_name_comment.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "heterogeneous_1.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "heterogeneous_2.parquet", definition, definitionSegment, false, false);
+ }
+
+ private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
+ String definitionSegment, boolean removeExtension) {
+ loadData(fileBasePath, filePathSegment, filename, definition, definitionSegment, removeExtension, true);
+ }
+
+ private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
+ String definitionSegment, boolean removeExtension, boolean copyToSubLevels) {
+ // Files data
+ Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
+
+ // Keep or remove the file extension
+ Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
+ String finalFileName;
+ if (removeExtension) {
+ finalFileName = FilenameUtils.removeExtension(filename);
+ } else {
+ finalFileName = filename;
+ }
+
+ // Files base definition
+ filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
+ definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
+ String basePath = definition + filePathSegment + definitionSegment;
+
+ // Load the data
+ String path = filePath.toString();
+ playgroundDataLoader.upload(basePath + finalFileName, path, true, false);
+ if (copyToSubLevels) {
+ playgroundDataLoader.upload(basePath + "level1a/" + finalFileName, path, true, false);
+ playgroundDataLoader.upload(basePath + "level1b/" + finalFileName, path, true, false);
+ playgroundDataLoader.upload(basePath + "level1a/level2a/" + finalFileName, path, true, false);
+ playgroundDataLoader.upload(basePath + "level1a/level2b/" + finalFileName, path, true, false);
+ }
+ }
+
+ private static void loadGzData(String fileBasePath, String filePathSegment, String filename, String definition,
+ String definitionSegment, boolean removeExtension) {
+ // Keep or remove the file extension
+ Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
+ String finalFileName;
+ if (removeExtension) {
+ finalFileName = FilenameUtils.removeExtension(filename);
+ } else {
+ finalFileName = filename;
+ }
+ finalFileName += ".gz";
+
+ // Files base definition
+ filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
+ definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
+ String basePath = definition + filePathSegment + definitionSegment;
+
+ // Load the data
+ String path = Paths.get(fileBasePath, filePathSegment, filename).toString();
+ playgroundDataLoader.upload(basePath + finalFileName, path, true, true);
+ playgroundDataLoader.upload(basePath + "level1a/" + finalFileName, path, true, true);
+ playgroundDataLoader.upload(basePath + "level1b/" + finalFileName, path, true, true);
+ playgroundDataLoader.upload(basePath + "level1a/level2a/" + finalFileName, path, true, true);
+ playgroundDataLoader.upload(basePath + "level1a/level2b/" + finalFileName, path, true, true);
+ }
+
+ private static void loadBigJson() {
+ String fileName = "big_record.json";
+ int bufferSize = 4 * 1024 * 1024;
+ int maxSize = bufferSize * 9;
+ Path filePath = Paths.get("target", "rttest", "tmp", fileName);
+ try {
+ if (Files.notExists(filePath)) {
+ Files.createDirectories(filePath.getParent());
+ Files.createFile(filePath);
+ }
+ } catch (IOException ex) {
+ throw new IllegalStateException("File " + fileName + " not found");
+ }
+
+ try (FileWriter writer = new FileWriter(filePath.toFile(), false);
+ BufferedWriter bw = new BufferedWriter(writer, bufferSize)) {
+ bw.append("{ \"large_field\": \"");
+ for (int i = 0; i < maxSize; i++) {
+ bw.append('A');
+ }
+ bw.append("\" }");
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ String key = "big-json/" + fileName;
+ playgroundDataLoader.upload(key, filePath.toString(), true, false);
+ }
+
+ /**
+ * Generates over 1000 objects and upload them to S3 mock server, 1 record per object
+ */
+ private static void loadLargeNumberOfFiles() {
+ for (int i = 0; i < OVER_1000_OBJECTS_COUNT; i++) {
+ playgroundDataLoader.upload(OVER_1000_OBJECTS_PATH + "/" + i + ".json", "{\"id\":" + i + "}");
+ }
+ }
+
+ /**
+ * Loads a combination of different file formats in the same path
+ */
+ public static void prepareMixedDataContainer() {
+ // JSON
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/extension/" + "hello-world-2018.json", "{\"id\":" + 1 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/extension/" + "hello-world-2019.json", "{\"id\":" + 2 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/extension/" + "hello-world-2020.json", "{\"id\":" + 3 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/EXTENSION/" + "goodbye-world-2018.json", "{\"id\":" + 4 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/EXTENSION/" + "goodbye-world-2019.json", "{\"id\":" + 5 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/EXTENSION/" + "goodbye-world-2020.json", "{\"id\":" + 6 + "}");
+
+ // CSV
+ mixedDataLoader.upload(MIXED_DEFINITION + "csv/extension/" + "hello-world-2018.csv", "7,\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "csv/extension/" + "hello-world-2019.csv", "8,\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "csv/extension/" + "hello-world-2020.csv", "9,\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "csv/EXTENSION/" + "goodbye-world-2018.csv", "10,\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "csv/EXTENSION/" + "goodbye-world-2019.csv", "11,\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "csv/EXTENSION/" + "goodbye-world-2020.csv", "12,\"good\"");
+
+ // TSV
+ mixedDataLoader.upload(MIXED_DEFINITION + "tsv/extension/" + "hello-world-2018.tsv", "13\t\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "tsv/extension/" + "hello-world-2019.tsv", "14\t\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "tsv/extension/" + "hello-world-2020.tsv", "15\t\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "tsv/EXTENSION/" + "goodbye-world-2018.tsv", "16\t\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "tsv/EXTENSION/" + "goodbye-world-2019.tsv", "17\t\"good\"");
+ mixedDataLoader.upload(MIXED_DEFINITION + "tsv/EXTENSION/" + "goodbye-world-2020.tsv", "18\t\"good\"");
+
+ // JSON no extension
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/no-extension/" + "hello-world-2018", "{\"id\":" + 1 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/no-extension/" + "hello-world-2019", "{\"id\":" + 2 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/no-extension/" + "hello-world-2020", "{\"id\":" + 3 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/NO-EXTENSION/" + "goodbye-world-2018", "{\"id\":" + 4 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/NO-EXTENSION/" + "goodbye-world-2019", "{\"id\":" + 5 + "}");
+ mixedDataLoader.upload(MIXED_DEFINITION + "json/NO-EXTENSION/" + "goodbye-world-2020", "{\"id\":" + 6 + "}");
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetOnePartitionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetOnePartitionTest.java
index 8114873..6c07fab 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetOnePartitionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetOnePartitionTest.java
@@ -46,6 +46,7 @@
PREPARE_BUCKET = AwsS3ExternalDatasetOnePartitionTest::prepareS3Bucket;
PREPARE_FIXED_DATA_BUCKET = AwsS3ExternalDatasetOnePartitionTest::prepareFixedDataBucket;
PREPARE_MIXED_DATA_BUCKET = AwsS3ExternalDatasetOnePartitionTest::prepareMixedDataBucket;
+ PREPARE_BOM_FILE_BUCKET = AwsS3ExternalDatasetOnePartitionTest::prepareBomDataBucket;
return LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
}
@@ -57,4 +58,7 @@
private static void prepareMixedDataBucket() {
}
+
+ private static void prepareBomDataBucket() {
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
index 387f7f6..05b0d0b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
@@ -18,17 +18,20 @@
*/
package org.apache.asterix.test.external_dataset.aws;
+import static org.apache.asterix.test.external_dataset.BinaryFileConverterUtil.DEFAULT_PARQUET_SRC_PATH;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createBinaryFiles;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setDataPaths;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setUploaders;
import static org.apache.hyracks.util.file.FileUtil.joinPath;
import java.io.ByteArrayOutputStream;
import java.io.File;
+import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.URI;
import java.nio.file.Files;
-import java.nio.file.Path;
import java.nio.file.Paths;
-import java.util.BitSet;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
@@ -39,6 +42,7 @@
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils;
import org.apache.asterix.test.runtime.ExecutionTestUtil;
import org.apache.asterix.test.runtime.LangExecutionUtil;
import org.apache.asterix.testframework.context.TestCaseContext;
@@ -50,7 +54,6 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.junit.AfterClass;
-import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
@@ -86,12 +89,12 @@
static Runnable PREPARE_BUCKET;
static Runnable PREPARE_FIXED_DATA_BUCKET;
static Runnable PREPARE_MIXED_DATA_BUCKET;
+ static Runnable PREPARE_BOM_FILE_BUCKET;
// Base directory paths for data files
private static final String JSON_DATA_PATH = joinPath("data", "json");
private static final String CSV_DATA_PATH = joinPath("data", "csv");
private static final String TSV_DATA_PATH = joinPath("data", "tsv");
- private static final String MIXED_DATA_PATH = joinPath("data", "mixed");
// Service endpoint
private static final int MOCK_SERVER_PORT = 8001;
@@ -99,31 +102,30 @@
// Region, bucket and definitions
private static final String MOCK_SERVER_REGION = "us-west-2";
- private static final String MOCK_SERVER_BUCKET = "playground";
- private static final String FIXED_DATA_BUCKET = "fixed-data"; // Do not use, has fixed data
- private static final String INCLUDE_EXCLUDE_BUCKET = "include-exclude"; // include & exclude bucket
- private static final String JSON_DEFINITION = "json-data/reviews/"; // data resides here
- private static final String CSV_DEFINITION = "csv-data/reviews/"; // data resides here
- private static final String TSV_DEFINITION = "tsv-data/reviews/"; // data resides here
-
- // This is used for a test to generate over 1000 number of files
- private static final String OVER_1000_OBJECTS_PATH = "over-1000-objects";
- private static final int OVER_1000_OBJECTS_COUNT = 2999;
private static final Set<String> fileNames = new HashSet<>();
private static final CreateBucketRequest.Builder CREATE_BUCKET_BUILDER = CreateBucketRequest.builder();
private static final DeleteBucketRequest.Builder DELETE_BUCKET_BUILDER = DeleteBucketRequest.builder();
private static final PutObjectRequest.Builder PUT_OBJECT_BUILDER = PutObjectRequest.builder();
- // IMPORTANT: The following values must be used in the AWS S3 test case
private static S3Mock s3MockServer;
private static S3Client client;
- private static final PutObjectRequest.Builder builder = PutObjectRequest.builder().bucket(MOCK_SERVER_BUCKET);
- private static final PutObjectRequest.Builder includeExcludeBuilder =
- PutObjectRequest.builder().bucket(INCLUDE_EXCLUDE_BUCKET);
protected TestCaseContext tcCtx;
+ public static final String PLAYGROUND_CONTAINER = "playground";
+ public static final String FIXED_DATA_CONTAINER = "fixed-data"; // Do not use, has fixed data
+ public static final String INCLUDE_EXCLUDE_CONTAINER = "include-exclude";
+ public static final String BOM_FILE_CONTAINER = "bom-file-container";
+ public static final PutObjectRequest.Builder playgroundBuilder =
+ PutObjectRequest.builder().bucket(PLAYGROUND_CONTAINER);
+ public static final PutObjectRequest.Builder fixedDataBuilder =
+ PutObjectRequest.builder().bucket(FIXED_DATA_CONTAINER);
+ public static final PutObjectRequest.Builder includeExcludeBuilder =
+ PutObjectRequest.builder().bucket(INCLUDE_EXCLUDE_CONTAINER);
+ public static final PutObjectRequest.Builder bomFileContainerBuilder =
+ PutObjectRequest.builder().bucket(BOM_FILE_CONTAINER);
+
public AwsS3ExternalDatasetTest(TestCaseContext tcCtx) {
this.tcCtx = tcCtx;
}
@@ -132,6 +134,7 @@
public static void setUp() throws Exception {
final TestExecutor testExecutor = new AwsTestExecutor();
LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+ createBinaryFiles(DEFAULT_PARQUET_SRC_PATH);
setNcEndpoints(testExecutor);
startAwsS3MockServer();
}
@@ -156,9 +159,11 @@
SUITE_TESTS = "testsuite_external_dataset_s3.xml";
ONLY_TESTS = "only_external_dataset.xml";
TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
- PREPARE_BUCKET = AwsS3ExternalDatasetTest::prepareS3Bucket;
- PREPARE_FIXED_DATA_BUCKET = AwsS3ExternalDatasetTest::prepareFixedDataBucket;
- PREPARE_MIXED_DATA_BUCKET = AwsS3ExternalDatasetTest::prepareMixedDataBucket;
+ PREPARE_BUCKET = ExternalDatasetTestUtils::preparePlaygroundContainer;
+ PREPARE_FIXED_DATA_BUCKET = ExternalDatasetTestUtils::prepareFixedDataContainer;
+ PREPARE_MIXED_DATA_BUCKET = ExternalDatasetTestUtils::prepareMixedDataContainer;
+ PREPARE_BOM_FILE_BUCKET = ExternalDatasetTestUtils::prepareBomFileContainer;
+
return LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
}
@@ -197,320 +202,70 @@
builder.region(Region.of(MOCK_SERVER_REGION)).credentialsProvider(AnonymousCredentialsProvider.create())
.endpointOverride(endpoint);
client = builder.build();
+ client.createBucket(CreateBucketRequest.builder().bucket(PLAYGROUND_CONTAINER).build());
+ client.createBucket(CreateBucketRequest.builder().bucket(FIXED_DATA_CONTAINER).build());
+ client.createBucket(CreateBucketRequest.builder().bucket(INCLUDE_EXCLUDE_CONTAINER).build());
+ client.createBucket(CreateBucketRequest.builder().bucket(BOM_FILE_CONTAINER).build());
LOGGER.info("Client created successfully");
// Create the bucket and upload some json files
+ setDataPaths(JSON_DATA_PATH, CSV_DATA_PATH, TSV_DATA_PATH);
+ setUploaders(AwsS3ExternalDatasetTest::loadPlaygroundData, AwsS3ExternalDatasetTest::loadFixedData,
+ AwsS3ExternalDatasetTest::loadMixedData, AwsS3ExternalDatasetTest::loadBomData);
PREPARE_BUCKET.run();
PREPARE_FIXED_DATA_BUCKET.run();
PREPARE_MIXED_DATA_BUCKET.run();
+ PREPARE_BOM_FILE_BUCKET.run();
}
- /**
- * Creates a bucket and fills it with some files for testing purpose.
- */
- private static void prepareS3Bucket() {
- LOGGER.info("creating bucket " + MOCK_SERVER_BUCKET);
- client.createBucket(CreateBucketRequest.builder().bucket(MOCK_SERVER_BUCKET).build());
- LOGGER.info("bucket created successfully");
-
- LOGGER.info("Adding JSON files to the bucket");
- loadJsonFiles();
- LOGGER.info("JSON Files added successfully");
-
- LOGGER.info("Adding CSV files to the bucket");
- loadCsvFiles();
- LOGGER.info("CSV Files added successfully");
-
- LOGGER.info("Adding TSV files to the bucket");
- loadTsvFiles();
- LOGGER.info("TSV Files added successfully");
-
- LOGGER.info("Loading " + OVER_1000_OBJECTS_COUNT + " into " + OVER_1000_OBJECTS_PATH);
- loadLargeNumberOfFiles();
- LOGGER.info("Added " + OVER_1000_OBJECTS_COUNT + " files into " + OVER_1000_OBJECTS_PATH + " successfully");
+ private static void loadPlaygroundData(String key, String content, boolean fromFile, boolean gzipped) {
+ client.putObject(playgroundBuilder.key(key).build(), getRequestBody(content, fromFile, gzipped));
}
- /**
- * This bucket is being filled by fixed data, a test is counting all records in this bucket. If this bucket is
- * changed, the test case will fail and its result will need to be updated each time
- */
- private static void prepareFixedDataBucket() {
- LOGGER.info("creating bucket " + FIXED_DATA_BUCKET);
- client.createBucket(CreateBucketRequest.builder().bucket(FIXED_DATA_BUCKET).build());
- LOGGER.info("bucket " + FIXED_DATA_BUCKET + " created successfully");
-
- LOGGER.info("Loading fixed data to " + FIXED_DATA_BUCKET);
-
- // Files data
- RequestBody requestBody = RequestBody.fromFile(Paths.get(JSON_DATA_PATH, "single-line", "20-records.json"));
- client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("1.json").build(), requestBody);
- client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("2.json").build(), requestBody);
- client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/3.json").build(), requestBody);
- client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/4.json").build(), requestBody);
- client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/lvl2/5.json").build(), requestBody);
+ private static void loadFixedData(String key, String content, boolean fromFile, boolean gzipped) {
+ client.putObject(fixedDataBuilder.key(key).build(), getRequestBody(content, fromFile, gzipped));
}
- private static void loadJsonFiles() {
- String dataBasePath = JSON_DATA_PATH;
- String definition = JSON_DEFINITION;
-
- // Normal format
- String definitionSegment = "json";
- loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
- loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
- loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
- loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
- false);
-
- definitionSegment = "json-array-of-objects";
- loadData(dataBasePath, "single-line", "array_of_objects.json", "json-data/", definitionSegment, false, false);
-
- // gz compressed format
- definitionSegment = "gz";
- loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
- loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
- loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
- loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
- false);
-
- // Mixed normal and gz compressed format
- definitionSegment = "mixed";
- loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
- loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
- loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
- loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
- false);
- loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
- loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
- loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
- loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
- false);
+ private static void loadMixedData(String key, String content, boolean fromFile, boolean gzipped) {
+ client.putObject(includeExcludeBuilder.key(key).build(), getRequestBody(content, fromFile, gzipped));
}
- private static void loadCsvFiles() {
- String dataBasePath = CSV_DATA_PATH;
- String definition = CSV_DEFINITION;
-
- // Normal format
- String definitionSegment = "csv";
- loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
- loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
-
- // gz compressed format
- definitionSegment = "gz";
- loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
- loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
-
- // Mixed normal and gz compressed format
- definitionSegment = "mixed";
- loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
- loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
- loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
- loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+ private static void loadBomData(String key, String content, boolean fromFile, boolean gzipped) {
+ client.putObject(bomFileContainerBuilder.key(key).build(), getRequestBody(content, fromFile, gzipped));
}
- private static void loadTsvFiles() {
- String dataBasePath = TSV_DATA_PATH;
- String definition = TSV_DEFINITION;
-
- // Normal format
- String definitionSegment = "tsv";
- loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
- loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
-
- // gz compressed format
- definitionSegment = "gz";
- loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
- loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
-
- // Mixed normal and gz compressed format
- definitionSegment = "mixed";
- loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
- loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
- loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
- loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
- }
-
- private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
- String definitionSegment, boolean removeExtension) {
- loadData(fileBasePath, filePathSegment, filename, definition, definitionSegment, removeExtension, true);
- }
-
- private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
- String definitionSegment, boolean removeExtension, boolean copyToSubLevels) {
- // Files data
- Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
- RequestBody requestBody = RequestBody.fromFile(filePath);
-
- // Keep or remove the file extension
- Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
- String finalFileName;
- if (removeExtension) {
- finalFileName = FilenameUtils.removeExtension(filename);
+ private static RequestBody getRequestBody(String content, boolean fromFile, boolean gzipped) {
+ RequestBody body;
+ // Content is string
+ if (!fromFile) {
+ body = RequestBody.fromString(content);
} else {
- finalFileName = filename;
- }
-
- // Files base definition
- filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
- definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
- String basePath = definition + filePathSegment + definitionSegment;
-
- // Load the data
- client.putObject(builder.key(basePath + finalFileName).build(), requestBody);
- if (copyToSubLevels) {
- client.putObject(builder.key(basePath + "level1a/" + finalFileName).build(), requestBody);
- client.putObject(builder.key(basePath + "level1b/" + finalFileName).build(), requestBody);
- client.putObject(builder.key(basePath + "level1a/level2a/" + finalFileName).build(), requestBody);
- client.putObject(builder.key(basePath + "level1a/level2b/" + finalFileName).build(), requestBody);
- }
- }
-
- private static void loadGzData(String fileBasePath, String filePathSegment, String filename, String definition,
- String definitionSegment, boolean removeExtension) {
- try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
- GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) {
-
- // Files data
- Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
-
- // Get the compressed data
- gzipOutputStream.write(Files.readAllBytes(filePath));
- gzipOutputStream.close(); // Need to close or data will be invalid
- byte[] gzipBytes = byteArrayOutputStream.toByteArray();
- RequestBody requestBody = RequestBody.fromBytes(gzipBytes);
-
- // Keep or remove the file extension
- Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
- String finalFileName;
- if (removeExtension) {
- finalFileName = FilenameUtils.removeExtension(filename);
+ // Content is a file path
+ if (!gzipped) {
+ body = RequestBody.fromFile(Paths.get(content));
} else {
- finalFileName = filename;
+ try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) {
+ gzipOutputStream.write(Files.readAllBytes(Paths.get(content)));
+ gzipOutputStream.close(); // Need to close or data will be invalid
+ byte[] gzipBytes = byteArrayOutputStream.toByteArray();
+ body = RequestBody.fromBytes(gzipBytes);
+ } catch (IOException ex) {
+ throw new IllegalArgumentException(ex.toString());
+ }
}
- finalFileName += ".gz";
-
- // Files base definition
- filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
- definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
- String basePath = definition + filePathSegment + definitionSegment;
-
- // Load the data
- client.putObject(builder.key(basePath + finalFileName).build(), requestBody);
- client.putObject(builder.key(basePath + "level1a/" + finalFileName).build(), requestBody);
- client.putObject(builder.key(basePath + "level1b/" + finalFileName).build(), requestBody);
- client.putObject(builder.key(basePath + "level1a/level2a/" + finalFileName).build(), requestBody);
- client.putObject(builder.key(basePath + "level1a/level2b/" + finalFileName).build(), requestBody);
- } catch (Exception ex) {
- LOGGER.error(ex.getMessage());
}
- }
- /**
- * Generates over 1000 objects and upload them to S3 mock server, 1 record per object
- */
- private static void loadLargeNumberOfFiles() {
- for (int i = 0; i < OVER_1000_OBJECTS_COUNT; i++) {
- RequestBody body = RequestBody.fromString("{\"id\":" + i + "}");
- client.putObject(builder.key(OVER_1000_OBJECTS_PATH + "/" + i + ".json").build(), body);
- }
- }
-
- /**
- * Loads a combination of different file formats in the same path
- */
- private static void prepareMixedDataBucket() {
- LOGGER.info("creating bucket " + INCLUDE_EXCLUDE_BUCKET);
- client.createBucket(CreateBucketRequest.builder().bucket(INCLUDE_EXCLUDE_BUCKET).build());
- LOGGER.info("bucket " + INCLUDE_EXCLUDE_BUCKET + " created successfully");
-
- // JSON
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2018.json").build(),
- RequestBody.fromString("{\"id\":" + 1 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2019.json").build(),
- RequestBody.fromString("{\"id\":" + 2 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2020.json").build(),
- RequestBody.fromString("{\"id\":" + 3 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2018.json").build(),
- RequestBody.fromString("{\"id\":" + 4 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2019.json").build(),
- RequestBody.fromString("{\"id\":" + 5 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2020.json").build(),
- RequestBody.fromString("{\"id\":" + 6 + "}"));
-
- // CSV
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2018.csv").build(),
- RequestBody.fromString("7,\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2019.csv").build(),
- RequestBody.fromString("8,\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2020.csv").build(),
- RequestBody.fromString("{9,\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2018.csv").build(),
- RequestBody.fromString("10,\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2019.csv").build(),
- RequestBody.fromString("11,\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2020.csv").build(),
- RequestBody.fromString("12,\"good\""));
-
- // TSV
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2018.tsv").build(),
- RequestBody.fromString("13\t\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2019.tsv").build(),
- RequestBody.fromString("14\t\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2020.tsv").build(),
- RequestBody.fromString("15\t\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2018.tsv").build(),
- RequestBody.fromString("16\t\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2019.tsv").build(),
- RequestBody.fromString("17\t\"good\""));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2020.tsv").build(),
- RequestBody.fromString("18\t\"good\""));
-
- // JSON no extension
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2018").build(),
- RequestBody.fromString("{\"id\":" + 1 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2019").build(),
- RequestBody.fromString("{\"id\":" + 2 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2020").build(),
- RequestBody.fromString("{\"id\":" + 3 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2018").build(),
- RequestBody.fromString("{\"id\":" + 4 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2019").build(),
- RequestBody.fromString("{\"id\":" + 5 + "}"));
- client.putObject(
- includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2020").build(),
- RequestBody.fromString("{\"id\":" + 6 + "}"));
+ return body;
}
static class AwsTestExecutor extends TestExecutor {
+ @Override
public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, TestCase.CompilationUnit cUnit,
- MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- BitSet expectedWarnings) throws Exception {
+ MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+ throws Exception {
String[] lines;
switch (ctx.getType()) {
case "container":
@@ -526,7 +281,7 @@
break;
default:
super.executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
- queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+ queryCount, expectedResultFileCtxs, testFile, actualPath);
}
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
index 27a46c1..894b4bc 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
@@ -18,6 +18,12 @@
*/
package org.apache.asterix.test.external_dataset.microsoft;
+import static org.apache.asterix.test.common.TestConstants.Azure.AZURITE_ACCOUNT_KEY_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.AZURITE_ACCOUNT_NAME_DEFAULT;
+import static org.apache.asterix.test.common.TestConstants.Azure.BLOB_ENDPOINT_PLACEHOLDER;
+import static org.apache.asterix.test.common.TestConstants.Azure.sasToken;
+import static org.apache.asterix.test.external_dataset.BinaryFileConverterUtil.BINARY_GEN_BASEDIR;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.PARQUET_DEFINITION;
import static org.apache.hyracks.util.file.FileUtil.joinPath;
import java.io.ByteArrayInputStream;
@@ -31,7 +37,6 @@
import java.nio.file.Paths;
import java.time.OffsetDateTime;
import java.time.temporal.ChronoUnit;
-import java.util.BitSet;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
@@ -41,8 +46,8 @@
import java.util.zip.GZIPOutputStream;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.test.common.TestConstants;
import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils;
import org.apache.asterix.test.runtime.ExecutionTestUtil;
import org.apache.asterix.test.runtime.LangExecutionUtil;
import org.apache.asterix.testframework.context.TestCaseContext;
@@ -67,11 +72,14 @@
import com.azure.storage.blob.BlobContainerClient;
import com.azure.storage.blob.BlobServiceClient;
import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.blob.models.PublicAccessType;
+import com.azure.storage.common.StorageSharedKeyCredential;
import com.azure.storage.common.sas.AccountSasPermission;
import com.azure.storage.common.sas.AccountSasResourceType;
import com.azure.storage.common.sas.AccountSasService;
import com.azure.storage.common.sas.AccountSasSignatureValues;
+// TODO(Hussain): Need to run the test manually to ensure new tests (anonymous access) are working fine
@Ignore
@RunWith(Parameterized.class)
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
@@ -92,15 +100,17 @@
private static final String CSV_DATA_PATH = joinPath("data", "csv");
private static final String TSV_DATA_PATH = joinPath("data", "tsv");
private static final String MIXED_DATA_PATH = joinPath("data", "mixed");
+ private static final String PARQUET_RAW_DATA_PATH = joinPath("data", "hdfs", "parquet");
// Service endpoint
- private static final int BLOB_SERVICE_PORT = 20000;
- private static final String BLOB_SERVICE_ENDPOINT = "http://localhost:" + BLOB_SERVICE_PORT;
+ private static final int BLOB_SERVICE_PORT = 10000;
+ private static final String BLOB_SERVICE_ENDPOINT = "http://192.168.0.100:" + BLOB_SERVICE_PORT;
// Region, container and definitions
private static final String PLAYGROUND_CONTAINER = "playground";
private static final String FIXED_DATA_CONTAINER = "fixed-data"; // Do not use, has fixed data
private static final String INCLUDE_EXCLUDE_CONTAINER = "include-exclude";
+ private static final String PUBLIC_ACCESS_CONTAINER = "public-access-container"; // requires no authentication
private static final String JSON_DEFINITION = "json-data/reviews/";
private static final String CSV_DEFINITION = "csv-data/reviews/";
private static final String TSV_DEFINITION = "tsv-data/reviews/";
@@ -112,11 +122,9 @@
private static final Set<String> fileNames = new HashSet<>();
// Create a BlobServiceClient object which will be used to create a container client
- private static final String connectionString = "AccountName=devstoreaccount1;"
- + "AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;"
- + "BlobEndpoint=" + BLOB_SERVICE_ENDPOINT + "/devstoreaccount1;";
private static BlobServiceClient blobServiceClient;
private static BlobContainerClient playgroundContainer;
+ private static BlobContainerClient publicAccessContainer;
protected TestCaseContext tcCtx;
@@ -127,6 +135,7 @@
@BeforeClass
public static void setUp() throws Exception {
final TestExecutor testExecutor = new AzureTestExecutor();
+ ExternalDatasetTestUtils.createBinaryFiles(PARQUET_RAW_DATA_PATH);
LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
setNcEndpoints(testExecutor);
createBlobServiceClient();
@@ -168,11 +177,14 @@
private static void createBlobServiceClient() {
LOGGER.info("Creating Azurite Blob Service client");
- blobServiceClient = new BlobServiceClientBuilder().connectionString(connectionString).buildClient();
+ BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
+ builder.credential(new StorageSharedKeyCredential(AZURITE_ACCOUNT_NAME_DEFAULT, AZURITE_ACCOUNT_KEY_DEFAULT));
+ builder.endpoint(BLOB_ENDPOINT_PLACEHOLDER);
+ blobServiceClient = builder.buildClient();
LOGGER.info("Azurite Blob Service client created successfully");
// Generate the SAS token for the SAS test cases
- TestConstants.sasToken = generateSasToken();
+ sasToken = generateSasToken();
// Create the container and upload some json files
PREPARE_PLAYGROUND_CONTAINER.run();
@@ -196,6 +208,8 @@
LOGGER.info("creating container " + PLAYGROUND_CONTAINER);
playgroundContainer = blobServiceClient.createBlobContainer(PLAYGROUND_CONTAINER);
+ publicAccessContainer = blobServiceClient.createBlobContainer(PUBLIC_ACCESS_CONTAINER);
+ publicAccessContainer.setAccessPolicy(PublicAccessType.CONTAINER, null);
LOGGER.info("container " + PLAYGROUND_CONTAINER + " created successfully");
LOGGER.info("Adding JSON files");
@@ -213,6 +227,10 @@
LOGGER.info("Loading " + OVER_1000_OBJECTS_COUNT + " into " + OVER_1000_OBJECTS_PATH);
loadLargeNumberOfFiles();
LOGGER.info("Added " + OVER_1000_OBJECTS_COUNT + " files into " + OVER_1000_OBJECTS_PATH + " successfully");
+
+ LOGGER.info("Adding Parquet files to the bucket");
+ loadParquetFiles();
+ LOGGER.info("Parquet files added successfully");
}
/**
@@ -317,6 +335,21 @@
loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
}
+ private static void loadParquetFiles() {
+ String dataBasePath = BINARY_GEN_BASEDIR;
+ String definition = PARQUET_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "";
+ loadData(dataBasePath, "", "dummy_tweet.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_age.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_age-string.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_name.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "id_name_comment.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "heterogeneous_1.parquet", definition, definitionSegment, false, false);
+ loadData(dataBasePath, "", "heterogeneous_2.parquet", definition, definitionSegment, false, false);
+ }
+
private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
String definitionSegment, boolean removeExtension) {
loadData(fileBasePath, filePathSegment, filename, definition, definitionSegment, removeExtension, true);
@@ -343,6 +376,7 @@
// Load the data
playgroundContainer.getBlobClient(basePath + finalFileName).uploadFromFile(filePath.toString());
+ publicAccessContainer.getBlobClient(basePath + finalFileName).uploadFromFile(filePath.toString());
if (copyToSubLevels) {
playgroundContainer.getBlobClient(basePath + "level1a/" + finalFileName)
.uploadFromFile(filePath.toString());
@@ -508,10 +542,11 @@
static class AzureTestExecutor extends TestExecutor {
+ @Override
public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, TestCase.CompilationUnit cUnit,
- MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- BitSet expectedWarnings) throws Exception {
+ MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
+ throws Exception {
String[] lines;
switch (ctx.getType()) {
case "container":
@@ -527,7 +562,7 @@
break;
default:
super.executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
- queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+ queryCount, expectedResultFileCtxs, testFile, actualPath);
}
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
index 28fca9c..65cbb9e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
@@ -26,38 +26,22 @@
import java.io.PrintWriter;
import java.io.StringReader;
import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
import java.util.Map;
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.api.java.AsterixJavaClient;
-import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
-import org.apache.asterix.common.config.GlobalConfig;
-import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.IdentitiyResolverFactory;
-import org.apache.asterix.file.StorageComponentProvider;
import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.test.base.AsterixTestHelper;
-import org.apache.asterix.test.common.TestHelper;
-import org.apache.asterix.test.runtime.HDFSCluster;
+import org.apache.asterix.test.optimizer.AbstractOptimizerTest;
import org.apache.asterix.translator.ExecutionPlans;
-import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.asterix.translator.SessionConfig.PlanFormat;
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.junit.AfterClass;
-import org.junit.Assume;
-import org.junit.BeforeClass;
import org.junit.Test;
-import org.junit.internal.AssumptionViolatedException;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
@@ -67,177 +51,82 @@
import com.fasterxml.jackson.databind.ObjectMapper;
@RunWith(Parameterized.class)
-public class JsonLogicalPlanTest {
+public class JsonLogicalPlanTest extends AbstractOptimizerTest {
- private static final Logger LOGGER = LogManager.getLogger();
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
static {
OBJECT_MAPPER.enable(DeserializationFeature.FAIL_ON_READING_DUP_TREE_KEY);
OBJECT_MAPPER.enable(DeserializationFeature.FAIL_ON_TRAILING_TOKENS);
}
- protected static final String SEPARATOR = File.separator;
- private static final String EXTENSION_SQLPP = "sqlpp";
- private static final String EXTENSION_RESULT = "plan.json";
- private static final String FILENAME_IGNORE = "ignore.txt";
- private static final String FILENAME_ONLY = "only.txt";
- private static final String PATH_BASE =
- "src" + SEPARATOR + "test" + SEPARATOR + "resources" + SEPARATOR + "optimizerts" + SEPARATOR;
- private static final String PATH_QUERIES = PATH_BASE + "queries" + SEPARATOR;
- protected static String PATH_ACTUAL = "target" + File.separator + "jplantest" + SEPARATOR;
protected static boolean optimized = false;
- private static final ArrayList<String> ignore = AsterixTestHelper.readTestListFile(FILENAME_IGNORE, PATH_BASE);
- private static final ArrayList<String> only = AsterixTestHelper.readTestListFile(FILENAME_ONLY, PATH_BASE);
- protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
- private static final ILangCompilationProvider sqlppCompilationProvider = new SqlppCompilationProvider();
- protected static ILangCompilationProvider extensionLangCompilationProvider = null;
- protected static IStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
- protected static IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
-
- protected static AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
-
- @BeforeClass
- public static void setUp() throws Exception {
- System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
- final File outdir = new File(PATH_ACTUAL);
- outdir.mkdirs();
-
- HDFSCluster.getInstance().setup();
-
- integrationUtil.init(true, TEST_CONFIG_FILE_NAME);
- // Set the node resolver to be the identity resolver that expects node names
- // to be node controller ids; a valid assumption in test environment.
- System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
- IdentitiyResolverFactory.class.getName());
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- File outdir = new File(PATH_ACTUAL);
- File[] files = outdir.listFiles();
- if (files == null || files.length == 0) {
- outdir.delete();
- }
-
- HDFSCluster.getInstance().cleanup();
-
- integrationUtil.deinit(true);
- }
-
- private static void suiteBuildPerFile(File file, Collection<Object[]> testArgs, String path) {
- if (file.isDirectory() && !file.getName().startsWith(".")) {
- for (File innerfile : file.listFiles()) {
- String subdir = innerfile.isDirectory() ? path + innerfile.getName() + SEPARATOR : path;
- suiteBuildPerFile(innerfile, testArgs, subdir);
- }
- }
- if (file.isFile() && file.getName().endsWith(EXTENSION_SQLPP)) {
- String resultFileName = AsterixTestHelper.extToResExt(file.getName(), EXTENSION_RESULT);
- File actualFile = new File(PATH_ACTUAL + SEPARATOR + path + resultFileName);
- testArgs.add(new Object[] { file, actualFile });
- }
+ static {
+ EXTENSION_RESULT = "plan.json";
+ PATH_ACTUAL = "target" + File.separator + "jplantest" + SEPARATOR;
}
@Parameters(name = "JsonLogicalPlanTest {index}: {0}")
public static Collection<Object[]> tests() {
- Collection<Object[]> testArgs = new ArrayList<>();
- if (only.isEmpty()) {
- suiteBuildPerFile(new File(PATH_QUERIES), testArgs, "");
- } else {
- for (String path : only) {
- suiteBuildPerFile(new File(PATH_QUERIES + path), testArgs,
- path.lastIndexOf(SEPARATOR) < 0 ? "" : path.substring(0, path.lastIndexOf(SEPARATOR) + 1));
- }
- }
- return testArgs;
+ return AbstractOptimizerTest.tests();
}
- private final File actualFile;
- private final File queryFile;
-
- public JsonLogicalPlanTest(final File queryFile, final File actualFile) {
- this.queryFile = queryFile;
- this.actualFile = actualFile;
+ public JsonLogicalPlanTest(final File queryFile, final File expectedFile, final File actualFile) {
+ super(queryFile, expectedFile, actualFile);
}
@Test
public void test() throws Exception {
- try {
- String queryFileShort =
- queryFile.getPath().substring(PATH_QUERIES.length()).replace(SEPARATOR.charAt(0), '/');
- if (!only.isEmpty()) {
- boolean toRun = TestHelper.isInPrefixList(only, queryFileShort);
- if (!toRun) {
- LOGGER.info("SKIP TEST: \"" + queryFile.getPath()
- + "\" \"only.txt\" not empty and not in \"only.txt\".");
+ super.test();
+ }
+
+ @Override
+ protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
+ IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+ FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
+ String planStr;
+ try (PrintWriter plan = new PrintWriter(actualFile)) {
+ AsterixJavaClient asterix = new AsterixJavaClient(
+ (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc, new StringReader(query),
+ plan, provider, statementExecutorFactory, storageComponentProvider);
+ asterix.setStatementParameters(queryParams);
+ asterix.compile(true, false, !optimized, optimized, false, false, false, PlanFormat.JSON);
+ ExecutionPlans executionPlans = asterix.getExecutionPlans();
+ planStr = optimized ? executionPlans.getOptimizedLogicalPlan() : executionPlans.getLogicalPlan();
+ plan.write(planStr);
+ } catch (AsterixException e) {
+ throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
+ }
+
+ BufferedReader readerActual =
+ new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));
+ String lineActual, objectActual = "";
+ boolean firstPlan = false;
+ while ((lineActual = readerActual.readLine()) != null) {
+ if (lineActual.contains("--")) {
+ if (firstPlan) {
+ break;
}
- Assume.assumeTrue(toRun);
- }
- boolean skipped = TestHelper.isInPrefixList(ignore, queryFileShort);
- if (skipped) {
- LOGGER.info("SKIP TEST: \"" + queryFile.getPath() + "\" in \"ignore.txt\".");
- }
- Assume.assumeTrue(!skipped);
+ firstPlan = true;
- LOGGER.info("RUN TEST: \"" + queryFile.getPath() + "\"");
- String query = FileUtils.readFileToString(queryFile, StandardCharsets.UTF_8);
- Map<String, IAObject> queryParams = TestHelper.readStatementParameters(query);
-
- // Forces the creation of actualFile.
- actualFile.getParentFile().mkdirs();
-
- ILangCompilationProvider provider = sqlppCompilationProvider;
- if (extensionLangCompilationProvider != null) {
- provider = extensionLangCompilationProvider;
- }
- IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
- String planStr;
- try (PrintWriter plan = new PrintWriter(actualFile)) {
- AsterixJavaClient asterix = new AsterixJavaClient(
- (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
- new StringReader(query), plan, provider, statementExecutorFactory, storageComponentProvider);
- asterix.setStatementParameters(queryParams);
- asterix.compile(true, false, !optimized, optimized, false, false, false, PlanFormat.JSON);
- ExecutionPlans executionPlans = asterix.getExecutionPlans();
- planStr = optimized ? executionPlans.getOptimizedLogicalPlan() : executionPlans.getLogicalPlan();
- plan.write(planStr);
- } catch (AsterixException e) {
- throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
- }
-
- BufferedReader readerActual =
- new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));
- String lineActual, objectActual = "";
- boolean firstPlan = false;
- while ((lineActual = readerActual.readLine()) != null) {
- if (lineActual.contains("--")) {
- if (firstPlan) {
- break;
- }
- firstPlan = true;
-
- } else {
- objectActual = objectActual + lineActual;
- }
- }
-
- try {
- JsonNode jsonNode = OBJECT_MAPPER.readTree(planStr);
- if (jsonNode == null || !jsonNode.isObject()) {
- throw new Exception("ERROR: No JSON plan or plan is malformed!");
- }
- } finally {
- readerActual.close();
- }
-
- } catch (Exception e) {
- if (!(e instanceof AssumptionViolatedException)) {
- LOGGER.error("Test \"" + queryFile.getPath() + "\" FAILED!");
- throw new Exception("Test \"" + queryFile.getPath() + "\" FAILED!", e);
} else {
- throw e;
+ objectActual = objectActual + lineActual;
}
}
+
+ try {
+ JsonNode jsonNode = OBJECT_MAPPER.readTree(planStr);
+ if (jsonNode == null || !jsonNode.isObject()) {
+ throw new Exception("ERROR: No JSON plan or plan is malformed!");
+ }
+ } finally {
+ readerActual.close();
+ }
+ }
+
+ @Override
+ protected List<String> getExpectedLines() {
+ // this test only checks the produced result is valid, so no expected results
+ return Collections.emptyList();
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
index b1fe529..7f78743 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonOptimizedLogicalPlanTest.java
@@ -20,16 +20,27 @@
package org.apache.asterix.test.jsonplan;
import java.io.File;
+import java.util.Collection;
+import org.apache.asterix.test.optimizer.AbstractOptimizerTest;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class JsonOptimizedLogicalPlanTest extends JsonLogicalPlanTest {
- public JsonOptimizedLogicalPlanTest(File queryFile, File actualFile) {
- super(queryFile, actualFile);
+ static {
+ EXTENSION_RESULT = "plan.json";
optimized = true;
PATH_ACTUAL = "target" + File.separator + "joptplantest" + SEPARATOR;
}
+
+ public JsonOptimizedLogicalPlanTest(File queryFile, File expectedFile, File actualFile) {
+ super(queryFile, expectedFile, actualFile);
+ }
+
+ @Parameterized.Parameters(name = "JsonOptimizedLogicalPlanTest {index}: {0}")
+ public static Collection<Object[]> tests() {
+ return AbstractOptimizerTest.tests();
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
new file mode 100644
index 0000000..0fba54b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/AbstractOptimizerTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.test.optimizer;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
+import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.IdentitiyResolverFactory;
+import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.test.base.AsterixTestHelper;
+import org.apache.asterix.test.common.TestHelper;
+import org.apache.asterix.test.runtime.HDFSCluster;
+import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.commons.io.FileUtils;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.internal.AssumptionViolatedException;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public abstract class AbstractOptimizerTest {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ protected static final String SEPARATOR = File.separator;
+ protected static final String EXTENSION_SQLPP = "sqlpp";
+ protected static String EXTENSION_RESULT;
+ protected static final String FILENAME_IGNORE = "ignore.txt";
+ protected static final String FILENAME_ONLY = "only.txt";
+ protected static final String PATH_BASE =
+ "src" + SEPARATOR + "test" + SEPARATOR + "resources" + SEPARATOR + "optimizerts" + SEPARATOR;
+ protected static final String PATH_QUERIES = PATH_BASE + "queries" + SEPARATOR;
+ protected static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
+ protected static String PATH_ACTUAL;
+
+ protected static final ArrayList<String> ignore = AsterixTestHelper.readTestListFile(FILENAME_IGNORE, PATH_BASE);
+ protected static final ArrayList<String> only = AsterixTestHelper.readTestListFile(FILENAME_ONLY, PATH_BASE);
+ protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+ protected static final ILangCompilationProvider sqlppCompilationProvider = new SqlppCompilationProvider();
+ protected static ILangCompilationProvider extensionLangCompilationProvider = null;
+ protected static IStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
+ protected static IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
+ protected static AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
+
+ private static final String REPEAT_EXEC = "repeat_exec:(.*)";
+ private static final Pattern PATTERN_REPEAT_EXEC = Pattern.compile(REPEAT_EXEC);
+ private static final ObjectMapper MAPPER = new ObjectMapper();
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
+ final File outdir = new File(PATH_ACTUAL);
+ outdir.mkdirs();
+
+ HDFSCluster.getInstance().setup();
+
+ integrationUtil.init(true, TEST_CONFIG_FILE_NAME);
+ // Set the node resolver to be the identity resolver that expects node names
+ // to be node controller ids; a valid assumption in test environment.
+ System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
+ IdentitiyResolverFactory.class.getName());
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ File outdir = new File(PATH_ACTUAL);
+ File[] files = outdir.listFiles();
+ if (files == null || files.length == 0) {
+ outdir.delete();
+ }
+
+ HDFSCluster.getInstance().cleanup();
+
+ integrationUtil.deinit(true);
+ }
+
+ private static void suiteBuildPerFile(File file, Collection<Object[]> testArgs, String path) {
+ if (file.isDirectory() && !file.getName().startsWith(".")) {
+ File[] files = file.listFiles();
+ Arrays.sort(files);
+ for (File innerfile : files) {
+ String subdir = innerfile.isDirectory() ? path + innerfile.getName() + SEPARATOR : path;
+ suiteBuildPerFile(innerfile, testArgs, subdir);
+ }
+ }
+ if (file.isFile() && file.getName().endsWith(EXTENSION_SQLPP)) {
+ String resultFileName = AsterixTestHelper.extToResExt(file.getName(), EXTENSION_RESULT);
+ File expectedFile = new File(PATH_EXPECTED + path + resultFileName);
+ File actualFile = new File(PATH_ACTUAL + SEPARATOR + path + resultFileName);
+ testArgs.add(new Object[] { file, expectedFile, actualFile });
+ }
+ }
+
+ protected static Collection<Object[]> tests() {
+ Collection<Object[]> testArgs = new ArrayList<>();
+ if (only.isEmpty()) {
+ suiteBuildPerFile(new File(PATH_QUERIES), testArgs, "");
+ } else {
+ for (String path : only) {
+ suiteBuildPerFile(new File(PATH_QUERIES + path), testArgs,
+ path.lastIndexOf(SEPARATOR) < 0 ? "" : path.substring(0, path.lastIndexOf(SEPARATOR) + 1));
+ }
+ }
+ return testArgs;
+ }
+
+ protected final File actualFile;
+ protected final File expectedFile;
+ protected final File queryFile;
+
+ public AbstractOptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
+ this.queryFile = queryFile;
+ this.expectedFile = expectedFile;
+ this.actualFile = actualFile;
+ }
+
+ protected abstract void runAndCompare(String query, ILangCompilationProvider provider,
+ Map<String, IAObject> queryParams, IHyracksClientConnection hcc, List<String> linesExpected)
+ throws Exception;
+
+ protected abstract List<String> getExpectedLines() throws IOException;
+
+ @Test
+ public void test() throws Exception {
+ try {
+ String queryFileShort =
+ queryFile.getPath().substring(PATH_QUERIES.length()).replace(SEPARATOR.charAt(0), '/');
+ if (!only.isEmpty()) {
+ boolean toRun = TestHelper.isInPrefixList(only, queryFileShort);
+ if (!toRun) {
+ LOGGER.info("SKIP TEST: \"" + queryFile.getPath()
+ + "\" \"only.txt\" not empty and not in \"only.txt\".");
+ }
+ Assume.assumeTrue(toRun);
+ }
+ boolean skipped = TestHelper.isInPrefixList(ignore, queryFileShort);
+ if (skipped) {
+ LOGGER.info("SKIP TEST: \"" + queryFile.getPath() + "\" in \"ignore.txt\".");
+ }
+ Assume.assumeTrue(!skipped);
+
+ LOGGER.info("RUN TEST: \"" + queryFile.getPath() + "\"");
+ String query = FileUtils.readFileToString(queryFile, StandardCharsets.UTF_8);
+ Map<String, IAObject> queryParams = TestHelper.readStatementParameters(query);
+ Matcher matcher = PATTERN_REPEAT_EXEC.matcher(query);
+ boolean repeat = false;
+ String placeholder = null;
+ JsonNode substitutions = null;
+ if (matcher.find()) {
+ repeat = true;
+ String placeholderSubstitutions = matcher.group(1);
+ String[] split = placeholderSubstitutions.split("=", 2);
+ placeholder = split[0].trim();
+ substitutions = MAPPER.readTree(split[1]);
+ }
+
+ LOGGER.info("ACTUAL RESULT FILE: " + actualFile.getAbsolutePath());
+
+ // Forces the creation of actualFile.
+ actualFile.getParentFile().mkdirs();
+
+ ILangCompilationProvider provider = sqlppCompilationProvider;
+ if (extensionLangCompilationProvider != null) {
+ provider = extensionLangCompilationProvider;
+ }
+ IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
+ List<String> linesExpected = getExpectedLines();
+ if (repeat) {
+ runAndRepeat(placeholder, substitutions, query, provider, queryParams, hcc, linesExpected);
+ } else {
+ runAndCompare(query, provider, queryParams, hcc, linesExpected);
+ }
+
+ LOGGER.info("Test \"" + queryFile.getPath() + "\" PASSED!");
+ actualFile.delete();
+ } catch (Exception e) {
+ if (!(e instanceof AssumptionViolatedException)) {
+ LOGGER.error("Test \"" + queryFile.getPath() + "\" FAILED!");
+ throw new Exception("Test \"" + queryFile.getPath() + "\" FAILED!", e);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ private void runAndRepeat(String placeholder, JsonNode substitutions, String query,
+ ILangCompilationProvider provider, Map<String, IAObject> queryParams, IHyracksClientConnection hcc,
+ List<String> linesExpected) throws Exception {
+ if (substitutions.isArray()) {
+ for (int i = 0, size = substitutions.size(); i < size; i++) {
+ String substitute = substitutions.get(i).asText();
+ String newQuery = query.replaceAll(placeholder, substitute);
+ runAndCompare(newQuery, provider, queryParams, hcc, linesExpected);
+ }
+ } else {
+ runAndCompare(query, provider, queryParams, hcc, linesExpected);
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index dbf8f2c..a89585d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -19,12 +19,11 @@
package org.apache.asterix.test.optimizer;
import java.io.File;
+import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringReader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
-import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
@@ -32,31 +31,14 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.api.java.AsterixJavaClient;
-import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
-import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.IdentitiyResolverFactory;
-import org.apache.asterix.file.StorageComponentProvider;
import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.test.base.AsterixTestHelper;
-import org.apache.asterix.test.common.TestHelper;
-import org.apache.asterix.test.runtime.HDFSCluster;
-import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.junit.AfterClass;
-import org.junit.Assume;
-import org.junit.BeforeClass;
import org.junit.Test;
-import org.junit.internal.AssumptionViolatedException;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
@@ -67,183 +49,76 @@
* query by setting the property (append the test case name with "_ps")
*/
@RunWith(Parameterized.class)
-public class OptimizerTest {
-
- private static final Logger LOGGER = LogManager.getLogger();
-
- private static final String SEPARATOR = File.separator;
- private static final String EXTENSION_SQLPP = "sqlpp";
- private static final String EXTENSION_RESULT = "plan";
- private static final String FILENAME_IGNORE = "ignore.txt";
- private static final String FILENAME_ONLY = "only.txt";
- private static final String PATH_BASE =
- "src" + SEPARATOR + "test" + SEPARATOR + "resources" + SEPARATOR + "optimizerts" + SEPARATOR;
- private static final String PATH_QUERIES = PATH_BASE + "queries" + SEPARATOR;
- private static final String PATH_EXPECTED = PATH_BASE + "results" + SEPARATOR;
- protected static final String PATH_ACTUAL = "target" + File.separator + "opttest" + SEPARATOR;
-
- private static final ArrayList<String> ignore = AsterixTestHelper.readTestListFile(FILENAME_IGNORE, PATH_BASE);
- private static final ArrayList<String> only = AsterixTestHelper.readTestListFile(FILENAME_ONLY, PATH_BASE);
- protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
- private static final ILangCompilationProvider sqlppCompilationProvider = new SqlppCompilationProvider();
- protected static ILangCompilationProvider extensionLangCompilationProvider = null;
- protected static IStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
- protected static IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
-
- protected static AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
+public class OptimizerTest extends AbstractOptimizerTest {
private static final String PATTERN_VAR_ID_PREFIX = "\\$\\$";
private static final Pattern PATTERN_VAR_ID = Pattern.compile(PATTERN_VAR_ID_PREFIX + "(\\d+)");
- @BeforeClass
- public static void setUp() throws Exception {
- final File outdir = new File(PATH_ACTUAL);
- outdir.mkdirs();
-
- HDFSCluster.getInstance().setup();
-
- integrationUtil.init(true, TEST_CONFIG_FILE_NAME);
- // Set the node resolver to be the identity resolver that expects node names
- // to be node controller ids; a valid assumption in test environment.
- System.setProperty(ExternalDataConstants.NODE_RESOLVER_FACTORY_PROPERTY,
- IdentitiyResolverFactory.class.getName());
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- File outdir = new File(PATH_ACTUAL);
- File[] files = outdir.listFiles();
- if (files == null || files.length == 0) {
- outdir.delete();
- }
-
- HDFSCluster.getInstance().cleanup();
-
- integrationUtil.deinit(true);
- }
-
- private static void suiteBuildPerFile(File file, Collection<Object[]> testArgs, String path) {
- if (file.isDirectory() && !file.getName().startsWith(".")) {
- File[] files = file.listFiles();
- Arrays.sort(files);
- for (File innerfile : files) {
- String subdir = innerfile.isDirectory() ? path + innerfile.getName() + SEPARATOR : path;
- suiteBuildPerFile(innerfile, testArgs, subdir);
- }
- }
- if (file.isFile() && file.getName().endsWith(EXTENSION_SQLPP)) {
- String resultFileName = AsterixTestHelper.extToResExt(file.getName(), EXTENSION_RESULT);
- File expectedFile = new File(PATH_EXPECTED + path + resultFileName);
- File actualFile = new File(PATH_ACTUAL + SEPARATOR + path + resultFileName);
- testArgs.add(new Object[] { file, expectedFile, actualFile });
- }
+ static {
+ EXTENSION_RESULT = "plan";
+ PATH_ACTUAL = "target" + File.separator + "opttest" + SEPARATOR;
}
@Parameters(name = "OptimizerTest {index}: {0}")
public static Collection<Object[]> tests() {
- Collection<Object[]> testArgs = new ArrayList<>();
- if (only.isEmpty()) {
- suiteBuildPerFile(new File(PATH_QUERIES), testArgs, "");
- } else {
- for (String path : only) {
- suiteBuildPerFile(new File(PATH_QUERIES + path), testArgs,
- path.lastIndexOf(SEPARATOR) < 0 ? "" : path.substring(0, path.lastIndexOf(SEPARATOR) + 1));
- }
- }
- return testArgs;
+ return AbstractOptimizerTest.tests();
}
- private final File actualFile;
- private final File expectedFile;
- private final File queryFile;
-
public OptimizerTest(final File queryFile, final File expectedFile, final File actualFile) {
- this.queryFile = queryFile;
- this.expectedFile = expectedFile;
- this.actualFile = actualFile;
+ super(queryFile, expectedFile, actualFile);
}
@Test
public void test() throws Exception {
- try {
- String queryFileShort =
- queryFile.getPath().substring(PATH_QUERIES.length()).replace(SEPARATOR.charAt(0), '/');
- if (!only.isEmpty()) {
- boolean toRun = TestHelper.isInPrefixList(only, queryFileShort);
- if (!toRun) {
- LOGGER.info("SKIP TEST: \"" + queryFile.getPath()
- + "\" \"only.txt\" not empty and not in \"only.txt\".");
- }
- Assume.assumeTrue(toRun);
- }
- boolean skipped = TestHelper.isInPrefixList(ignore, queryFileShort);
- if (skipped) {
- LOGGER.info("SKIP TEST: \"" + queryFile.getPath() + "\" in \"ignore.txt\".");
- }
- Assume.assumeTrue(!skipped);
+ super.test();
+ }
- LOGGER.info("RUN TEST: \"" + queryFile.getPath() + "\"");
- String query = FileUtils.readFileToString(queryFile, StandardCharsets.UTF_8);
- Map<String, IAObject> queryParams = TestHelper.readStatementParameters(query);
-
- LOGGER.info("ACTUAL RESULT FILE: " + actualFile.getAbsolutePath());
-
- // Forces the creation of actualFile.
- actualFile.getParentFile().mkdirs();
-
- ILangCompilationProvider provider = sqlppCompilationProvider;
- if (extensionLangCompilationProvider != null) {
- provider = extensionLangCompilationProvider;
- }
- IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
- try (PrintWriter plan = new PrintWriter(actualFile)) {
- AsterixJavaClient asterix = new AsterixJavaClient(
- (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
- new StringReader(query), plan, provider, statementExecutorFactory, storageComponentProvider);
- asterix.setStatementParameters(queryParams);
- asterix.compile(true, false, false, true, true, false, false);
- } catch (AlgebricksException e) {
- throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
- }
-
- List<String> linesExpected = Files.readAllLines(expectedFile.toPath(), StandardCharsets.UTF_8);
- List<String> linesActual = Files.readAllLines(actualFile.toPath(), StandardCharsets.UTF_8);
-
- int varBaseExpected = findBaseVarId(linesExpected);
- int varBaseActual = findBaseVarId(linesActual);
-
- Iterator<String> readerExpected = linesExpected.iterator();
- Iterator<String> readerActual = linesActual.iterator();
- String lineExpected, lineActual;
- int num = 1;
- while (readerExpected.hasNext()) {
- lineExpected = readerExpected.next();
- if (!readerActual.hasNext()) {
- throw new Exception(
- "Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected + "\n> ");
- }
- lineActual = readerActual.next();
-
- if (!planLineEquals(lineExpected, varBaseExpected, lineActual, varBaseActual)) {
- throw new Exception("Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected
- + "\n> " + lineActual);
- }
- ++num;
- }
- if (readerActual.hasNext()) {
- throw new Exception(
- "Result for " + queryFile + " changed at line " + num + ":\n< \n> " + readerActual.next());
- }
- LOGGER.info("Test \"" + queryFile.getPath() + "\" PASSED!");
- actualFile.delete();
- } catch (Exception e) {
- if (!(e instanceof AssumptionViolatedException)) {
- LOGGER.error("Test \"" + queryFile.getPath() + "\" FAILED!");
- throw new Exception("Test \"" + queryFile.getPath() + "\" FAILED!", e);
- } else {
- throw e;
- }
+ @Override
+ protected void runAndCompare(String query, ILangCompilationProvider provider, Map<String, IAObject> queryParams,
+ IHyracksClientConnection hcc, List<String> linesExpected) throws Exception {
+ FileUtils.writeStringToFile(actualFile, "", StandardCharsets.UTF_8);
+ try (PrintWriter plan = new PrintWriter(actualFile)) {
+ AsterixJavaClient asterix = new AsterixJavaClient(
+ (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc, new StringReader(query),
+ plan, provider, statementExecutorFactory, storageComponentProvider);
+ asterix.setStatementParameters(queryParams);
+ asterix.compile(true, false, false, true, true, false, false);
+ } catch (AlgebricksException e) {
+ throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
}
+
+ List<String> linesActual = Files.readAllLines(actualFile.toPath(), StandardCharsets.UTF_8);
+
+ int varBaseExpected = findBaseVarId(linesExpected);
+ int varBaseActual = findBaseVarId(linesActual);
+
+ Iterator<String> readerExpected = linesExpected.iterator();
+ Iterator<String> readerActual = linesActual.iterator();
+ String lineExpected, lineActual;
+ int num = 1;
+ while (readerExpected.hasNext()) {
+ lineExpected = readerExpected.next();
+ if (!readerActual.hasNext()) {
+ throw new Exception(
+ "Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected + "\n> ");
+ }
+ lineActual = readerActual.next();
+
+ if (!planLineEquals(lineExpected, varBaseExpected, lineActual, varBaseActual)) {
+ throw new Exception("Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected
+ + "\n> " + lineActual);
+ }
+ ++num;
+ }
+ if (readerActual.hasNext()) {
+ throw new Exception(
+ "Result for " + queryFile + " changed at line " + num + ":\n< \n> " + readerActual.next());
+ }
+ }
+
+ @Override
+ protected List<String> getExpectedLines() throws IOException {
+ return Files.readAllLines(expectedFile.toPath(), StandardCharsets.UTF_8);
}
private boolean planLineEquals(String lineExpected, int varIdBaseExpected, String lineActual, int varIdBaseActual) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index 2328bf4..ccdf620 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -73,11 +73,10 @@
}
integrationUtil.init(cleanup, configFile);
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("initializing HDFS");
- }
-
if (startHdfs) {
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("initializing HDFS");
+ }
HDFSCluster.getInstance().setup();
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
index f5c51b4..4c30d2b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
@@ -18,15 +18,14 @@
*/
package org.apache.asterix.test.runtime;
+import static org.apache.asterix.test.external_dataset.BinaryFileConverterUtil.BINARY_GEN_BASEDIR;
+import static org.apache.asterix.test.external_dataset.BinaryFileConverterUtil.DEFAULT_PARQUET_SRC_PATH;
+
import java.io.File;
-import java.io.FileInputStream;
import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
import org.apache.asterix.external.dataset.adapter.GenericAdapter;
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData.Record;
+import org.apache.asterix.test.external_dataset.BinaryFileConverterUtil;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -36,11 +35,6 @@
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hyracks.api.util.IoUtil;
-import org.kitesdk.data.spi.JsonUtil;
-import org.kitesdk.data.spi.filesystem.JSONFileReader;
-
-import parquet.avro.AvroParquetWriter;
/**
* Manages a Mini (local VM) HDFS cluster with a configured number of datanodes.
@@ -53,8 +47,6 @@
private static final String DATA_PATH = "data/hdfs";
private static final String HDFS_PATH = "/asterix";
private static final HDFSCluster INSTANCE = new HDFSCluster();
- //Temporary folder that holds generated binary files
- private static final String BINARY_GEN_BASEDIR = "target" + File.separatorChar + "generated_bin_files";
//How many records should the schema inference method inspect to infer the schema for parquet files
private static final int NUM_OF_RECORDS_SCHEMA = 20;
@@ -93,8 +85,10 @@
build.startupOption(StartupOption.REGULAR);
dfsCluster = build.build();
dfs = FileSystem.get(conf);
- //Generate binary files from JSON files (e.g., parquet files)
- generateBinaryFiles(basePath);
+ //clean the binary generated files' directory
+ BinaryFileConverterUtil.cleanBinaryDirectory(basePath, BINARY_GEN_BASEDIR);
+ //Convert files in DEFAULT_PARQUET_SRC_PATH to parquet
+ BinaryFileConverterUtil.convertToParquet(basePath, DEFAULT_PARQUET_SRC_PATH, BINARY_GEN_BASEDIR);
//Load JSON/ADM files to HDFS
loadData(basePath, DATA_PATH);
//Load generated binary files (e.g., parquet files) to HDFS
@@ -123,16 +117,6 @@
System.setProperty("hadoop.log.dir", "logs");
}
- private void generateBinaryFiles(File localDataRoot) throws IOException {
- File srcPath = new File(localDataRoot, DATA_PATH);
- File destPath = new File(localDataRoot, BINARY_GEN_BASEDIR);
- //Delete old generated files
- IoUtil.delete(destPath);
- Files.createDirectory(Paths.get(destPath.getAbsolutePath()));
- //Write parquet files
- writeParquetDir(new File(srcPath, "parquet"), destPath);
- }
-
public void cleanup() throws Exception {
if (dfsCluster != null) {
dfsCluster.shutdown();
@@ -161,28 +145,4 @@
conf.set("mapred.input.format.class", TextInputFormat.class.getName());
return conf;
}
-
- private void writeParquetDir(File parquetSrcDir, File destPath) throws IOException {
- File[] listOfFiles = parquetSrcDir.listFiles();
- for (File jsonFile : listOfFiles) {
- String fileName = jsonFile.getName().substring(0, jsonFile.getName().indexOf(".")) + ".parquet";
- Path outputPath = new Path(destPath.getAbsolutePath(), fileName);
- writeParquetFile(jsonFile, outputPath);
- }
- }
-
- public void writeParquetFile(File jsonInputPath, Path parquetOutputPath) throws IOException {
- final FileInputStream schemaInputStream = new FileInputStream(jsonInputPath);
- final FileInputStream jsonInputStream = new FileInputStream(jsonInputPath);
- //Infer Avro schema
- final Schema inputSchema = JsonUtil.inferSchema(schemaInputStream, "parquet_schema", NUM_OF_RECORDS_SCHEMA);
- try (JSONFileReader<Record> reader = new JSONFileReader<>(jsonInputStream, inputSchema, Record.class)) {
- reader.initialize();
- try (AvroParquetWriter<Record> writer = new AvroParquetWriter<>(parquetOutputPath, inputSchema)) {
- for (Record record : reader) {
- writer.write(record);
- }
- }
- }
- }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingExceptionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingExceptionTest.java
index 735f8c5..034dd22 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingExceptionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingExceptionTest.java
@@ -77,6 +77,7 @@
// We test all functions except record and cast functions, which requires type settings (we test them
// in runtime tests).
+ // TODO(ali): ASTERIXDB-2982 do it in a proper way so that it does not exclude classes inadvertently
if (!className.contains("record") && !className.contains("Cast")) {
tests.add(new Object[] { getTestName(functionDescriptor.getClass()), functionDescriptor });
} else {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
index 8cd02ed..30cb6bd 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
@@ -20,6 +20,10 @@
package org.apache.asterix.test.runtime;
+import static org.apache.asterix.common.annotations.MissingNullInOutFunction.MissingNullType.MISSING;
+import static org.apache.asterix.common.annotations.MissingNullInOutFunction.MissingNullType.NULL;
+import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_MISSING_TYPE_TAG;
+import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_NULL_TYPE_TAG;
import static org.mockito.Mockito.mock;
import java.util.ArrayList;
@@ -33,7 +37,6 @@
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -89,6 +92,7 @@
// We test all functions except record, cast and full-text contains functions,
// which requires type settings or argument settings.
// Instead, we test them in runtime tests.
+ // TODO(ali): ASTERIXDB-2982 do it in a proper way so that it does not exclude classes inadvertently
if (!className.contains("record") && !className.contains("Cast")
&& !className.contains("FullTextContains")) {
tests.add(new Object[] { getTestName(functionDescriptor.getClass()), functionDescriptor });
@@ -128,6 +132,9 @@
int inputArity = funcDesc.getIdentifier().getArity();
Iterator<Pair<IScalarEvaluatorFactory[], IAType[]>> argEvalFactoryIterator = getArgCombinations(inputArity);
int index = 0;
+ MissingNullInOutFunction annot = functionDescriptor.getClass().getAnnotation(MissingNullInOutFunction.class);
+ byte missingOut = annot.onMissing() == MISSING ? SERIALIZED_MISSING_TYPE_TAG : SERIALIZED_NULL_TYPE_TAG;
+ byte nullOut = annot.onNull() == NULL ? SERIALIZED_NULL_TYPE_TAG : SERIALIZED_MISSING_TYPE_TAG;
// Test is happening here
while (argEvalFactoryIterator.hasNext()) {
@@ -147,11 +154,9 @@
// Result checks
if (index != 0) {
- Assert.assertEquals(ATypeTag.SERIALIZED_MISSING_TYPE_TAG,
- resultPointable.getByteArray()[resultPointable.getStartOffset()]);
+ Assert.assertEquals(missingOut, resultPointable.getByteArray()[resultPointable.getStartOffset()]);
} else {
- Assert.assertEquals(ATypeTag.SERIALIZED_NULL_TYPE_TAG,
- resultPointable.getByteArray()[resultPointable.getStartOffset()]);
+ Assert.assertEquals(nullOut, resultPointable.getByteArray()[resultPointable.getStartOffset()]);
}
++index;
}
@@ -178,11 +183,10 @@
if ((index & (1 << j)) != 0) {
argumentTypes[j] = BuiltinType.AMISSING;
scalarEvaluatorFactories[j] =
- new ConstantEvalFactory(new byte[] { ATypeTag.SERIALIZED_MISSING_TYPE_TAG });
+ new ConstantEvalFactory(new byte[] { SERIALIZED_MISSING_TYPE_TAG });
} else {
argumentTypes[j] = BuiltinType.ANULL;
- scalarEvaluatorFactories[j] =
- new ConstantEvalFactory(new byte[] { ATypeTag.SERIALIZED_NULL_TYPE_TAG });
+ scalarEvaluatorFactories[j] = new ConstantEvalFactory(new byte[] { SERIALIZED_NULL_TYPE_TAG });
}
}
++index;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHashJoinRQJTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHashJoinRQJTest.java
new file mode 100644
index 0000000..40ee38e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppHashJoinRQJTest.java
@@ -0,0 +1,442 @@
+/*
+ * 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.test.runtime;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.nio.IntBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
+import org.apache.asterix.common.utils.Servlets;
+import org.apache.asterix.lang.sqlpp.parser.SqlppHint;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.math3.random.MersenneTwister;
+import org.apache.commons.math3.random.RandomGenerator;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+/**
+ * RQG testsuite for hash joins.
+ * Tests:
+ * <ul>
+ * <li> Fields with / without NULL or MISSING values
+ * <li> Inner / Left Outer joins</li>
+ * <li> Repartitioning / Broadcast joins </li>
+ * </ul>
+ */
+@RunWith(Parameterized.class)
+public class SqlppHashJoinRQJTest {
+
+ static final Logger LOGGER = LogManager.getLogger(SqlppHashJoinRQJTest.class);
+
+ static final String CONF_PROPERTY_SEED =
+ SqlppRQGTestBase.getConfigurationPropertyName(SqlppHashJoinRQJTest.class, "seed");
+ static final long CONF_PROPERTY_SEED_DEFAULT = System.currentTimeMillis();
+
+ static final String CONF_PROPERTY_LIMIT =
+ SqlppRQGTestBase.getConfigurationPropertyName(SqlppHashJoinRQJTest.class, "limit");
+ static final int CONF_PROPERTY_LIMIT_DEFAULT = 40;
+
+ static final String CONF_PROPERTY_OFFSET =
+ SqlppRQGTestBase.getConfigurationPropertyName(SqlppHashJoinRQJTest.class, "offset");
+ static final int CONF_PROPERTY_OFFSET_DEFAULT = 0;
+
+ static final Path OUTPUT_DIR = Paths.get("target", SqlppHashJoinRQJTest.class.getSimpleName());
+
+ static final String DATAVERSE_NAME = "dvTest";
+ static final String[] DATASET_NAMES = new String[] { "ds1", "ds2" };
+ static final String ID_COLUMN_NAME = "id";
+ static final String BASE_COLUMN_NAME = "i";
+ static final List<Integer> DATASET_ROWS = Arrays.asList(20000, 40000);
+ static final List<Integer> DATASET_COLUMNS = Arrays.asList(4, 10, 100, 1000, 10000);
+ static final int DATASET_COLUMN_LENGTH_MIN =
+ String.valueOf(DATASET_COLUMNS.stream().mapToInt(Integer::intValue).max().orElse(0)).length();
+ static final int DATASET_COLUMN_LENGTH_MAX = Math.max(20, DATASET_COLUMN_LENGTH_MIN);
+ static final int NULLABLE_COLUMN_RATIO = 2;
+ static final int OUTER_JOIN_RATIO = 3;
+ static final int BROADCAST_RATIO = 4;
+
+ static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ static final ObjectReader OBJECT_READER = OBJECT_MAPPER.readerFor(ObjectNode.class);
+
+ static long datasetRowCount;
+ static int datasetColumnLength;
+ static TestExecutor testExecutor;
+
+ final TestInstance testInstance;
+
+ public SqlppHashJoinRQJTest(TestInstance testInstance) {
+ this.testInstance = testInstance;
+ }
+
+ @Parameterized.Parameters(name = "SqlppHashJoinRQJTest {index}: {0}")
+ public static Collection<TestInstance> tests() {
+ long seed = SqlppRQGTestBase.getLongConfigurationProperty(CONF_PROPERTY_SEED, CONF_PROPERTY_SEED_DEFAULT);
+ int limit =
+ (int) SqlppRQGTestBase.getLongConfigurationProperty(CONF_PROPERTY_LIMIT, CONF_PROPERTY_LIMIT_DEFAULT);
+ int testOffset =
+ (int) SqlppRQGTestBase.getLongConfigurationProperty(CONF_PROPERTY_OFFSET, CONF_PROPERTY_OFFSET_DEFAULT);
+
+ LOGGER.info(String.format("Testsuite configuration: -D%s=%d -D%s=%d -D%s=%d", CONF_PROPERTY_SEED, seed,
+ CONF_PROPERTY_LIMIT, limit, CONF_PROPERTY_OFFSET, testOffset));
+
+ RandomGenerator random = new MersenneTwister(seed);
+ datasetRowCount = randomElement(DATASET_ROWS, random);
+ datasetColumnLength =
+ DATASET_COLUMN_LENGTH_MIN + random.nextInt(DATASET_COLUMN_LENGTH_MAX - DATASET_COLUMN_LENGTH_MIN);
+
+ LOGGER.info(String.format("Dataset row count=%d, column length=%d", datasetRowCount, datasetColumnLength));
+
+ LinkedHashMap<IntBuffer, TestInstance> testCases = new LinkedHashMap<>();
+ int i = 0;
+ while (i < limit) {
+ int c0 = randomElement(DATASET_COLUMNS, random);
+ boolean c0nullable = random.nextInt(NULLABLE_COLUMN_RATIO) == 0;
+ int c1 = randomElement(DATASET_COLUMNS, random);
+ boolean c1nullable = random.nextInt(NULLABLE_COLUMN_RATIO) == 0;
+ boolean outerJoin = random.nextInt(OUTER_JOIN_RATIO) == 0;
+ boolean broadcast = random.nextInt(BROADCAST_RATIO) == 0;
+ TestInstance test = new TestInstance(i, c0, c0nullable, c1, c1nullable, outerJoin, broadcast);
+ IntBuffer testSignature = test.signature();
+ if (testCases.containsKey(testSignature)) {
+ continue;
+ }
+ if (i >= testOffset) {
+ testCases.put(testSignature, test);
+ }
+ i++;
+ }
+ return testCases.values();
+ }
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ testExecutor = new TestExecutor();
+ LangExecutionUtil.setUp(SqlppRQGTestBase.TEST_CONFIG_FILE_NAME, testExecutor, false);
+
+ FileUtils.forceMkdir(OUTPUT_DIR.toFile());
+ for (String datasetName : DATASET_NAMES) {
+ Path datasetFilePath = OUTPUT_DIR.resolve(datasetName + ".adm");
+ LOGGER.info("Writing data file: " + datasetFilePath.toAbsolutePath());
+ try (PrintWriter pw = new PrintWriter(datasetFilePath.toFile())) {
+ for (int i = 0; i < datasetRowCount; i++) {
+ writeRecord(pw, datasetName, i);
+ }
+ }
+ }
+
+ StringBuilder sb = new StringBuilder(2048);
+ addDropDataverse(sb, DATAVERSE_NAME);
+ addCreateDataverse(sb, DATAVERSE_NAME);
+ for (String datasetName : DATASET_NAMES) {
+ addCreateDataset(sb, DATAVERSE_NAME, datasetName);
+ addLoadDataset(sb, DATAVERSE_NAME, datasetName);
+ }
+ executeUpdateOrDdl(sb.toString());
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+ }
+
+ @Test
+ public void test() throws Exception {
+ LOGGER.info(testInstance);
+ testInstance.execute();
+ }
+
+ private static void addDropDataverse(StringBuilder sb, String dataverseName) {
+ sb.append(String.format("DROP DATAVERSE %s IF EXISTS;\n", dataverseName));
+ }
+
+ private static void addCreateDataverse(StringBuilder sb, String dataverseName) {
+ sb.append(String.format("CREATE DATAVERSE %s;\n", dataverseName));
+ }
+
+ private static void addCreateDataset(StringBuilder sb, String dataverseName, String datasetName) {
+ sb.append("CREATE DATASET ").append(dataverseName).append('.').append(datasetName);
+ sb.append(" (").append(ID_COLUMN_NAME).append(" string");
+ sb.append(") ");
+ sb.append("OPEN TYPE PRIMARY KEY ").append(ID_COLUMN_NAME).append(";\n");
+ }
+
+ private static void addLoadDataset(StringBuilder sb, String dataverseName, String datasetName) {
+ sb.append(String.format(
+ "LOAD DATASET %s.%s USING localfs((`path`=`asterix_nc1://%s/%s.adm`),(`format`=`adm`));%n",
+ dataverseName, datasetName, OUTPUT_DIR, datasetName));
+ }
+
+ private static void writeRecord(PrintWriter pw, String datasetName, int id) throws IOException {
+ pw.print("{");
+ pw.print(String.format("\"%s\": \"%s:%d\"", ID_COLUMN_NAME, datasetName, id));
+ int nColumns = DATASET_COLUMNS.size();
+ for (int i = 0; i < nColumns; i++) {
+ long c = DATASET_COLUMNS.get(i);
+ writeColumn(pw, c, false, id); // no NULL/MISSING
+ writeColumn(pw, c, true, id); // with NULL/MISSING
+ }
+ pw.println("}");
+ }
+
+ private static String getColumnName(long c, boolean nullable) {
+ return BASE_COLUMN_NAME + c + (nullable ? "n" : "");
+ }
+
+ private static void writeColumn(Appendable out, long c, boolean nullable, long id) throws IOException {
+ String columnName = getColumnName(c, nullable);
+ boolean isNull = false;
+ long v;
+ if (nullable) {
+ long r = id % (2 * c);
+ if (r < c) {
+ v = r + 1;
+ } else if (r % 2 == 0) {
+ v = 0;
+ isNull = true;
+ } else {
+ // MISSING -> nothing to do
+ return;
+ }
+ } else {
+ long r = id % c;
+ v = r + 1;
+ }
+ String text;
+ if (isNull) {
+ text = "null";
+ } else {
+ int cLen = datasetColumnLength;
+ StringBuilder textBuilder = new StringBuilder(cLen + 2);
+ textBuilder.append('"').append(v);
+ int pad = cLen - (textBuilder.length() - 1);
+ for (int i = 0; i < pad; i++) {
+ textBuilder.append(' ');
+ }
+ textBuilder.append('"');
+ text = textBuilder.toString();
+ }
+ out.append(String.format(",\"%s\":%s", columnName, text));
+ }
+
+ private static void executeUpdateOrDdl(String statement) throws Exception {
+ LOGGER.debug("Executing: " + statement);
+ testExecutor.executeSqlppUpdateOrDdl(statement, TestCaseContext.OutputFormat.CLEAN_JSON);
+ }
+
+ private static Pair<ArrayNode, String> executeQuery(String query, boolean fetchPlan) throws Exception {
+ LOGGER.debug("Executing: " + query);
+
+ List<TestCase.CompilationUnit.Parameter> params;
+ if (fetchPlan) {
+ TestCase.CompilationUnit.Parameter planParameter = new TestCase.CompilationUnit.Parameter();
+ planParameter.setName(QueryServiceRequestParameters.Parameter.OPTIMIZED_LOGICAL_PLAN.str());
+ planParameter.setValue(Boolean.TRUE.toString());
+ planParameter.setType(ParameterTypeEnum.STRING);
+ params = Collections.singletonList(planParameter);
+ } else {
+ params = Collections.emptyList();
+ }
+
+ try (InputStream resultStream = testExecutor.executeQueryService(query, TestCaseContext.OutputFormat.CLEAN_JSON,
+ testExecutor.getEndpoint(Servlets.QUERY_SERVICE), params, true, StandardCharsets.UTF_8)) {
+ JsonNode r = OBJECT_READER.readTree(resultStream);
+ JsonNode errors = r.get("errors");
+ if (errors != null) {
+ Assert.fail("Query failed: " + errors);
+ }
+ JsonNode results = r.get("results");
+ if (!results.isArray()) {
+ Assert.fail("Expected array result, got: " + results);
+ }
+ ArrayNode resultsArray = (ArrayNode) results;
+ String plan = fetchPlan ? r.get("plans").get("optimizedLogicalPlan").asText() : null;
+ return new Pair<>(resultsArray, plan);
+ }
+ }
+
+ private static <T> T randomElement(List<T> list, RandomGenerator randomGenerator) {
+ return list.get(randomGenerator.nextInt(list.size()));
+ }
+
+ private static class TestInstance {
+
+ private final int id;
+
+ private final int c0;
+ private final int c1;
+ private final boolean c0nullable;
+ private final boolean c1nullable;
+ private final String col0;
+ private final String col1;
+ private final boolean outerJoin;
+ private final boolean broadcastJoin;
+
+ public TestInstance(int id, int c0, boolean c0nullable, int c1, boolean c1nullable, boolean outerJoin,
+ boolean broadcastJoin) {
+ this.id = id;
+ this.outerJoin = outerJoin;
+ this.c0 = c0;
+ this.c1 = c1;
+ this.c0nullable = c0nullable;
+ this.c1nullable = c1nullable;
+ this.broadcastJoin = broadcastJoin;
+ this.col0 = getColumnName(c0, c0nullable);
+ this.col1 = getColumnName(c1, c1nullable);
+ }
+
+ IntBuffer signature() {
+ return IntBuffer.wrap(
+ new int[] { c0, toInt(c0nullable), c1, toInt(c1nullable), toInt(outerJoin), toInt(broadcastJoin) });
+ }
+
+ void execute() throws Exception {
+ String query = createQuery();
+ Pair<ArrayNode, String> res = executeQuery(query, true);
+ String plan = res.second;
+ if (!plan.contains(PhysicalOperatorTag.HYBRID_HASH_JOIN.toString())) {
+ Assert.fail(PhysicalOperatorTag.HYBRID_HASH_JOIN + " operator was not used in query plan " + plan);
+ }
+ if (broadcastJoin && !plan.contains(PhysicalOperatorTag.BROADCAST_EXCHANGE.toString())) {
+ Assert.fail(PhysicalOperatorTag.BROADCAST_EXCHANGE + " operator was not used in query plan " + plan);
+ }
+ ArrayNode resultArray = res.first;
+
+ long expectedRowCount;
+ long expectedRowCountInnerJoin = Math.min(c0, c1);
+ if (outerJoin) {
+ expectedRowCount = expectedRowCountInnerJoin + (c0nullable ? 2 : 0) + Math.max(0, c0 - c1);
+ } else {
+ expectedRowCount = expectedRowCountInnerJoin;
+ }
+
+ long expectedAggCountInnerJoin = (datasetRowCount * datasetRowCount) / (((long) c0) * c1)
+ / (c0nullable ? 2 : 1) / (c1nullable ? 2 : 1);
+
+ int actualRowCount = resultArray.size();
+ if (actualRowCount != expectedRowCount) {
+ String commentHash = String.format("%s;%s", this, query);
+ File fHash = SqlppRQGTestBase.writeResult(OUTPUT_DIR, resultArray, id, "hash", commentHash);
+ Assert.fail(String.format("Unexpected row count %d for query #%d [%s]. Expected row count: %d %n %s ",
+ actualRowCount, id, this, expectedRowCount, fHash.getAbsolutePath()));
+ }
+
+ String col0Alias = String.format("%s_%s", DATASET_NAMES[0], col0);
+
+ for (int i = 0; i < actualRowCount; i++) {
+ JsonNode resultRecord = resultArray.get(i);
+ long actualAggCount = resultRecord.get("cnt").longValue();
+
+ long expectedAggCount;
+ if (outerJoin) {
+ JsonNode col0Node = resultRecord.get(col0Alias);
+ if (col0Node == null || col0Node.isNull()) {
+ expectedAggCount = datasetRowCount / 4;
+ } else {
+ if (getValueAsLong(col0Node) > c1) {
+ expectedAggCount = datasetRowCount / (c0nullable ? 2 : 1) / c0;
+ } else {
+ expectedAggCount = expectedAggCountInnerJoin;
+ }
+ }
+ } else {
+ expectedAggCount = expectedAggCountInnerJoin;
+ }
+
+ if (actualAggCount != expectedAggCount) {
+ String commentHash = String.format("%s;%s", this, query);
+ File fHash = SqlppRQGTestBase.writeResult(OUTPUT_DIR, resultArray, id, "hash", commentHash);
+ Assert.fail(String.format(
+ "Unexpected agg count %d in row %d for query #%d [%s]. Expected agg count: %d %n %s ",
+ actualAggCount, i, id, this, expectedAggCount, fHash.getAbsolutePath()));
+ }
+ }
+ }
+
+ private long getValueAsLong(JsonNode node) throws Exception {
+ String text = node.textValue().trim();
+ if (text.isEmpty()) {
+ throw new Exception("Unexpected result value: " + node);
+ }
+ try {
+ return Long.parseLong(text);
+ } catch (NumberFormatException e) {
+ throw new Exception("Unexpected result value: " + node);
+ }
+ }
+
+ String createQuery() {
+ return String.format(
+ "USE %s; SELECT t1.%s AS %s_%s, t2.%s AS %s_%s, count(*) AS cnt FROM %s t1 %s JOIN %s t2 ON t1.%s /*%s*/ = t2.%s /*%s*/ GROUP BY t1.%s, t2.%s ORDER BY t1.%s, t2.%s",
+ DATAVERSE_NAME, col0, DATASET_NAMES[0], col0, col1, DATASET_NAMES[1], col1, DATASET_NAMES[0],
+ getJoinKind(), DATASET_NAMES[1], col0, getJoinHint(), col1, getGroupByHint(), col0, col1, col0,
+ col1);
+ }
+
+ private String getJoinKind() {
+ return outerJoin ? "LEFT OUTER" : "INNER";
+ }
+
+ private String getJoinHint() {
+ return broadcastJoin ? "+" + SqlppHint.HASH_BROADCAST_JOIN_HINT.getIdentifier() : "";
+ }
+
+ private String getGroupByHint() {
+ return "";
+ }
+
+ @Override
+ public String toString() {
+ return String.format("%s ON %s=%s %s", getJoinKind(), col0, col1, getJoinHint());
+ }
+
+ static int toInt(boolean b) {
+ return b ? 1 : 0;
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppNumericIndexRQGTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppNumericIndexRQGTest.java
new file mode 100644
index 0000000..e97a389
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppNumericIndexRQGTest.java
@@ -0,0 +1,609 @@
+/*
+ * 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.test.runtime;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.common.utils.Servlets;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.sqlpp.parser.SqlppHint;
+import org.apache.asterix.metadata.utils.TypeUtil;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
+import org.apache.commons.math3.random.MersenneTwister;
+import org.apache.commons.math3.random.RandomGenerator;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+/**
+ * RQG testsuite for indexes on numeric fields
+ */
+@RunWith(Parameterized.class)
+public class SqlppNumericIndexRQGTest {
+
+ static final Logger LOGGER = LogManager.getLogger(SqlppNumericIndexRQGTest.class);
+
+ static final String CONF_PROPERTY_SEED =
+ SqlppRQGTestBase.getConfigurationPropertyName(SqlppNumericIndexRQGTest.class, "seed");
+ static final long CONF_PROPERTY_SEED_DEFAULT = System.currentTimeMillis();
+
+ static final String CONF_PROPERTY_LIMIT =
+ SqlppRQGTestBase.getConfigurationPropertyName(SqlppNumericIndexRQGTest.class, "limit");
+ static final int CONF_PROPERTY_LIMIT_DEFAULT = 50;
+
+ static final String CONF_PROPERTY_OFFSET =
+ SqlppRQGTestBase.getConfigurationPropertyName(SqlppNumericIndexRQGTest.class, "offset");
+ static final int CONF_PROPERTY_OFFSET_DEFAULT = 0;
+
+ static final String CONF_PROPERTY_OFFSET_QUERY =
+ SqlppRQGTestBase.getConfigurationPropertyName(SqlppNumericIndexRQGTest.class, "offset.query");
+ static final int CONF_PROPERTY_OFFSET_QUERY_DEFAULT = 0;
+
+ static final Path OUTPUT_DIR = Paths.get("target", SqlppNumericIndexRQGTest.class.getSimpleName());
+
+ static final String DATAVERSE_NAME = "dvTest";
+ static final String DATASET_NAME_TYPED = "dsTyped";
+ static final String DATASET_NAME_UNTYPED = "dsUntyped";
+ static final String COMPILER_OPTION_FORMAT = "set `%s` '%s';";
+ static final String ID_COLUMN_NAME = "id";
+
+ static final List<AlgebricksBuiltinFunctions.ComparisonKind> CMP_KINDS =
+ Arrays.asList(AlgebricksBuiltinFunctions.ComparisonKind.GT, AlgebricksBuiltinFunctions.ComparisonKind.GE,
+ AlgebricksBuiltinFunctions.ComparisonKind.LT, AlgebricksBuiltinFunctions.ComparisonKind.LE,
+ AlgebricksBuiltinFunctions.ComparisonKind.EQ);
+
+ static final List<BuiltinType> FIELD_TYPES = Arrays.asList(BuiltinType.ANY, BuiltinType.AINT8, BuiltinType.AINT16,
+ BuiltinType.AINT32, BuiltinType.AINT64, BuiltinType.AFLOAT, BuiltinType.ADOUBLE);
+
+ static final double MIN_VALUE = -3.5;
+ static final double MAX_VALUE = 3.5;
+ static final double PROBE_STEP = 1.0 / 4;
+ static final double DATA_STEP = PROBE_STEP / 2;
+ static final int NON_NUMERIC_DATA_RATIO = 8;
+ static final int INDEX_ONLY_ENABLED_RATIO = 3;
+ static final int INDEXED_AS_IS_RATIO = 4;
+ static final int JOIN_RATIO = 8;
+
+ static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ static final ObjectReader OBJECT_READER = OBJECT_MAPPER.readerFor(ObjectNode.class);
+
+ static TestExecutor testExecutor;
+
+ private final TestInstance testInstance;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ testExecutor = new TestExecutor();
+ LangExecutionUtil.setUp(SqlppRQGTestBase.TEST_CONFIG_FILE_NAME, testExecutor, false);
+
+ StringBuilder sb = new StringBuilder(2048);
+ addDropDataverse(sb, DATAVERSE_NAME);
+ addCreateDataverse(sb, DATAVERSE_NAME);
+ addCreateDataset(sb, DATAVERSE_NAME, DATASET_NAME_TYPED, false);
+ addLoadDataset(sb, DATAVERSE_NAME, DATASET_NAME_TYPED);
+ addCreateDataset(sb, DATAVERSE_NAME, DATASET_NAME_UNTYPED, true);
+ addLoadDataset(sb, DATAVERSE_NAME, DATASET_NAME_UNTYPED);
+ executeUpdateOrDdl(sb.toString());
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+ }
+
+ @Parameterized.Parameters(name = "SqlppNumericIndexRQGTest {index}: {0}")
+ public static Collection<TestInstance> tests() {
+
+ long seed = SqlppRQGTestBase.getLongConfigurationProperty(CONF_PROPERTY_SEED, CONF_PROPERTY_SEED_DEFAULT);
+ int limit =
+ (int) SqlppRQGTestBase.getLongConfigurationProperty(CONF_PROPERTY_LIMIT, CONF_PROPERTY_LIMIT_DEFAULT);
+ int testOffset =
+ (int) SqlppRQGTestBase.getLongConfigurationProperty(CONF_PROPERTY_OFFSET, CONF_PROPERTY_OFFSET_DEFAULT);
+ int queryOffset = (int) SqlppRQGTestBase.getLongConfigurationProperty(CONF_PROPERTY_OFFSET_QUERY,
+ CONF_PROPERTY_OFFSET_QUERY_DEFAULT);
+
+ LOGGER.info(String.format("Testsuite configuration: -D%s=%d -D%s=%d -D%s=%d -D%s=%d", CONF_PROPERTY_SEED, seed,
+ CONF_PROPERTY_LIMIT, limit, CONF_PROPERTY_OFFSET, testOffset, CONF_PROPERTY_OFFSET_QUERY, queryOffset));
+
+ String indexName = "idx_" + seed;
+
+ List<TestInstance> testCases = new ArrayList<>(limit);
+ RandomGenerator random = new MersenneTwister(seed);
+
+ int i = 0;
+ while (i < limit) {
+ BuiltinType fieldType = randomElement(FIELD_TYPES, random);
+
+ BuiltinType indexedType;
+ if (random.nextInt(INDEXED_AS_IS_RATIO) == 0) {
+ if (fieldType.getTypeTag() == ATypeTag.ANY) {
+ continue;
+ }
+ indexedType = null;
+ } else {
+ indexedType = randomElement(FIELD_TYPES, random);
+ if (indexedType.getTypeTag() == ATypeTag.ANY) {
+ continue;
+ }
+ }
+
+ boolean join = random.nextInt(JOIN_RATIO) == 0;
+ BuiltinType probeType;
+ if (join) {
+ probeType = indexedType != null ? indexedType : fieldType;
+ } else {
+ probeType = randomElement(FIELD_TYPES, random);
+ }
+ if (probeType.getTypeTag() == ATypeTag.ANY) {
+ continue;
+ }
+
+ Set<TestOption> options = EnumSet.noneOf(TestOption.class);
+ if (random.nextBoolean()) {
+ options.add(TestOption.EXCLUDE_UNKNOWN_KEY);
+ }
+ if (indexedType != null && random.nextBoolean()) {
+ options.add(TestOption.CAST_DEFAULT_NULL);
+ }
+ if (random.nextInt(INDEX_ONLY_ENABLED_RATIO) == 0) {
+ options.add(TestOption.INDEX_ONLY_ENABLED);
+ }
+ if (join) {
+ options.add(random.nextBoolean() ? TestOption.INNER_JOIN_QUERY : TestOption.LEFT_OUTER_JOIN_QUERY);
+ }
+ String searchedDatasetName = indexedType == null ? DATASET_NAME_TYPED : DATASET_NAME_UNTYPED;
+ String probeSourceDatasetName = indexedType == null ? DATASET_NAME_UNTYPED : DATASET_NAME_TYPED;
+ if (i >= testOffset) {
+ TestInstance testCase = new TestInstance(i, DATAVERSE_NAME, searchedDatasetName, indexName, fieldType,
+ indexedType, probeType, probeSourceDatasetName, options, queryOffset);
+ testCases.add(testCase);
+ }
+ i++;
+ }
+
+ return testCases;
+ }
+
+ public SqlppNumericIndexRQGTest(TestInstance testInstance) {
+ this.testInstance = testInstance;
+ }
+
+ @Test
+ public void test() throws Exception {
+ LOGGER.info(testInstance);
+ testInstance.execute();
+ }
+
+ private static boolean hasIndexSearch(ArrayNode planLines, String indexName) {
+ for (int i = 0, n = planLines.size(); i < n; i++) {
+ String line = planLines.get(i).textValue();
+ if (line.contains(BuiltinFunctions.INDEX_SEARCH.getName()) && line.contains('"' + indexName + '"')) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private static void executeUpdateOrDdl(String statement) throws Exception {
+ LOGGER.debug("Executing: " + statement);
+ testExecutor.executeSqlppUpdateOrDdl(statement, TestCaseContext.OutputFormat.CLEAN_JSON);
+ }
+
+ private static Pair<ArrayNode, String> executeQuery(String query, boolean fetchPlan) throws Exception {
+ LOGGER.debug("Executing: " + query);
+
+ List<TestCase.CompilationUnit.Parameter> params;
+ if (fetchPlan) {
+ TestCase.CompilationUnit.Parameter planParameter = new TestCase.CompilationUnit.Parameter();
+ planParameter.setName(QueryServiceRequestParameters.Parameter.OPTIMIZED_LOGICAL_PLAN.str());
+ planParameter.setValue(Boolean.TRUE.toString());
+ planParameter.setType(ParameterTypeEnum.STRING);
+ params = Collections.singletonList(planParameter);
+ } else {
+ params = Collections.emptyList();
+ }
+
+ try (InputStream resultStream = testExecutor.executeQueryService(query, TestCaseContext.OutputFormat.CLEAN_JSON,
+ testExecutor.getEndpoint(Servlets.QUERY_SERVICE), params, true, StandardCharsets.UTF_8)) {
+ JsonNode r = OBJECT_READER.readTree(resultStream);
+ JsonNode errors = r.get("errors");
+ if (errors != null) {
+ Assert.fail("Query failed: " + errors);
+ }
+ JsonNode results = r.get("results");
+ if (!results.isArray()) {
+ Assert.fail("Expected array result, got: " + results);
+ }
+ ArrayNode resultsArray = (ArrayNode) results;
+ String plan = fetchPlan ? r.get("plans").get("optimizedLogicalPlan").asText() : null;
+ return new Pair<>(resultsArray, plan);
+ }
+ }
+
+ private static void addDropDataverse(StringBuilder sb, String dataverseName) {
+ sb.append(String.format("DROP DATAVERSE %s IF EXISTS;\n", dataverseName));
+ }
+
+ private static void addCreateDataverse(StringBuilder sb, String dataverseName) {
+ sb.append(String.format("CREATE DATAVERSE %s;\n", dataverseName));
+ }
+
+ private static void addCreateDataset(StringBuilder sb, String dataverseName, String datasetName, boolean untyped) {
+ sb.append("CREATE DATASET ").append(dataverseName).append('.').append(datasetName);
+ sb.append(" (").append(ID_COLUMN_NAME).append(" string ");
+ if (!untyped) {
+ for (BuiltinType t : FIELD_TYPES) {
+ if (t.getTypeTag() == ATypeTag.ANY) {
+ continue;
+ }
+ sb.append(", ").append(getColumnName(t)).append(' ').append(t.getTypeName());
+ }
+ }
+ sb.append(") ");
+ sb.append("OPEN TYPE PRIMARY KEY ").append(ID_COLUMN_NAME).append(";\n");
+ }
+
+ private static void addLoadDataset(StringBuilder sb, String dataverseName, String datasetName) {
+ int id = 0, nonNumeric = 0;
+ for (double v = MIN_VALUE; v <= MAX_VALUE; v += DATA_STEP) {
+ addInsert(sb, dataverseName, datasetName, id, v);
+ id++;
+
+ // Insert non-numeric values once in a while
+ if (NON_NUMERIC_DATA_RATIO > 0 && id % NON_NUMERIC_DATA_RATIO == 0) {
+ Double nnv;
+ switch (nonNumeric % 3) {
+ case 0:
+ nnv = null;
+ break;
+ case 1:
+ nnv = Double.POSITIVE_INFINITY;
+ break;
+ case 2:
+ nnv = Double.NEGATIVE_INFINITY;
+ break;
+ default:
+ throw new IllegalStateException(String.valueOf(nonNumeric));
+ }
+ addInsert(sb, dataverseName, datasetName, id, nnv);
+ nonNumeric++;
+ id++;
+ }
+ }
+ }
+
+ private static void addInsert(StringBuilder sb, String dataverseName, String datasetName, int id, Double v) {
+ sb.append("INSERT INTO ").append(dataverseName).append('.').append(datasetName).append(" ( { ");
+ sb.append("'").append(ID_COLUMN_NAME).append("': \"").append(datasetName).append(':').append(id).append('"');
+ for (BuiltinType fieldType : FIELD_TYPES) {
+ String columnName = getColumnName(fieldType);
+ sb.append(", '").append(columnName).append("':");
+ if (v == null) {
+ sb.append("null");
+ } else if (Double.isInfinite(v)) {
+ sb.append(fieldType.getTypeTag() == ATypeTag.DOUBLE || fieldType.getTypeTag() == ATypeTag.FLOAT
+ ? String.format("%s('%sINF')", fieldType.getTypeName(), v < 0 ? "-" : "") : "null");
+ } else {
+ BuiltinType valueType;
+ if (fieldType.getTypeTag() == ATypeTag.ANY) {
+ valueType = FIELD_TYPES.get(id % FIELD_TYPES.size());
+ if (valueType.getTypeTag() == ATypeTag.ANY) {
+ valueType = BuiltinType.ASTRING;
+ }
+ } else {
+ valueType = fieldType;
+ }
+ String castFn = valueType.getTypeName();
+ sb.append(castFn).append('(').append(v).append(')');
+ }
+ }
+ sb.append("} );\n");
+ }
+
+ private static String getColumnName(BuiltinType t) {
+ return "c_" + t.getTypeName();
+ }
+
+ private static <T> T randomElement(List<T> list, RandomGenerator randomGenerator) {
+ return list.get(randomGenerator.nextInt(list.size()));
+ }
+
+ private static ArrayNode readLinesIntoArrayNode(String str) {
+ try {
+ ArrayNode arrayNode = OBJECT_MAPPER.createArrayNode();
+ BufferedReader br = new BufferedReader(new StringReader(str));
+ String line;
+ while ((line = br.readLine()) != null) {
+ arrayNode.add(line);
+ }
+ return arrayNode;
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ private enum TestOption {
+ EXCLUDE_UNKNOWN_KEY,
+ CAST_DEFAULT_NULL,
+ INDEX_ONLY_ENABLED,
+ JOIN_QUERY,
+ INNER_JOIN_QUERY,
+ LEFT_OUTER_JOIN_QUERY
+
+ }
+
+ private static class TestInstance {
+
+ private final int id;
+
+ private final String dataverseName;
+
+ private final String datasetName;
+
+ private final String indexName;
+
+ private final BuiltinType fieldType;
+
+ private final BuiltinType indexedType;
+
+ private final BuiltinType probeType;
+
+ private final String probeSourceDatasetName; // for JOIN query
+
+ private final Set<TestOption> options;
+
+ private final int queryOffset;
+
+ public TestInstance(int id, String dataverseName, String datasetName, String indexName, BuiltinType fieldType,
+ BuiltinType indexedType, BuiltinType probeType, String probeSourceDatasetName,
+ Set<TestOption> testOptions, int queryOffset) {
+ this.id = id;
+ this.dataverseName = dataverseName;
+ this.datasetName = datasetName;
+ this.indexName = indexName;
+ this.fieldType = fieldType;
+ this.indexedType = indexedType;
+ this.probeType = probeType;
+ this.probeSourceDatasetName = probeSourceDatasetName;
+ this.options = testOptions == null ? Collections.emptySet() : testOptions;
+ this.queryOffset = queryOffset;
+ }
+
+ void execute() throws Exception {
+ List<String> queries;
+ if (options.contains(TestOption.INNER_JOIN_QUERY)) {
+ queries = Collections.singletonList(generateJoinQuery(false));
+ } else if (options.contains(TestOption.LEFT_OUTER_JOIN_QUERY)) {
+ queries = Collections.singletonList(generateJoinQuery(true));
+ } else {
+ queries = new ArrayList<>((int) ((MAX_VALUE - MIN_VALUE) / PROBE_STEP) + 1);
+ for (double v = MIN_VALUE; v <= MAX_VALUE; v += PROBE_STEP) {
+ for (AlgebricksBuiltinFunctions.ComparisonKind cmpKind : CMP_KINDS) {
+ queries.add(generateSearchQuery(cmpKind, v));
+ }
+ }
+ }
+
+ int queryCount = queries.size();
+
+ executeUpdateOrDdl(generateDropIndex());
+
+ LOGGER.info(String.format("Running queries [%d to %d] without index ...", queryOffset, queryCount - 1));
+ ArrayNode[] resNoIndex = new ArrayNode[queryCount];
+ for (int i = queryOffset; i < queryCount; i++) {
+ String query = queries.get(i);
+ ArrayNode result = executeQuery(query, false).first;
+ resNoIndex[i] = result;
+ }
+
+ LOGGER.info("Creating index");
+ executeUpdateOrDdl(generateCreateIndex());
+
+ LOGGER.info(String.format("Running queries [%d to %d] with index ...", queryOffset, queryCount - 1));
+ for (int i = queryOffset; i < queryCount; i++) {
+ String query = queries.get(i);
+ ArrayNode rNoIndex = resNoIndex[i];
+ Pair<ArrayNode, String> pWithIndex = executeQuery(query, true);
+ ArrayNode planWithIndex = readLinesIntoArrayNode(pWithIndex.second);
+ String comment = String.format("%s;%s", this, query);
+ if (!hasIndexSearch(planWithIndex, indexName)) {
+ File fPlan = SqlppRQGTestBase.writeResult(OUTPUT_DIR, planWithIndex, id, "plan", comment,
+ (out, node) -> out.println(node.asText()));
+ Assert.fail(
+ String.format("Index was not used. Expected to find search of index [%s] in query plan: %s",
+ indexName, fPlan.getAbsolutePath()));
+ }
+ ArrayNode rWithIndex = pWithIndex.first;
+ if (!rNoIndex.equals(rWithIndex)) {
+ File fNoIndex = SqlppRQGTestBase.writeResult(OUTPUT_DIR, rNoIndex, id, "no_index", comment);
+ File fWithIndex = SqlppRQGTestBase.writeResult(OUTPUT_DIR, rWithIndex, id, "with_index", comment);
+ Assert.fail(
+ String.format("Different results for query #%d [%s].%nWithout index: %s%nWith index: %s", i,
+ query, fNoIndex.getAbsolutePath(), fWithIndex.getAbsolutePath()));
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder(64);
+ sb.append('#').append(id).append(' ');
+ sb.append(generateCreateIndex(true));
+ sb.append(String.format(" PROBE(%s)", probeType.getTypeName()));
+ if (options.contains(TestOption.INDEX_ONLY_ENABLED)) {
+ sb.append(" INDEX_ONLY");
+ }
+ if (options.contains(TestOption.INNER_JOIN_QUERY)) {
+ sb.append(" JOIN");
+ } else if (options.contains(TestOption.LEFT_OUTER_JOIN_QUERY)) {
+ sb.append(" LEFT JOIN");
+ }
+ return sb.toString();
+ }
+
+ private String generateDropIndex() {
+ return String.format("DROP INDEX %s.%s.%s IF EXISTS", dataverseName, datasetName, indexName);
+ }
+
+ private String generateCreateIndex() {
+ return generateCreateIndex(false);
+ }
+
+ private String generateCreateIndex(boolean displayOnly) {
+ StringBuilder sb = new StringBuilder(64);
+ String columnName = getColumnName(fieldType);
+ if (!displayOnly) {
+ sb.append("CREATE INDEX ").append(indexName).append(" ON ").append(dataverseName).append('.');
+ }
+ sb.append(datasetName).append("(").append(columnName);
+ if (indexedType != null) {
+ sb.append(':').append(indexedType.getTypeName());
+ }
+ sb.append(")");
+ sb.append(" ");
+ if (options.contains(TestOption.EXCLUDE_UNKNOWN_KEY)) {
+ sb.append(displayOnly ? "-UNKN_KEY" : "EXCLUDE UNKNOWN KEY");
+ } else {
+ sb.append(displayOnly ? "+UNKN_KEY" : "INCLUDE UNKNOWN KEY");
+ }
+ if (options.contains(TestOption.CAST_DEFAULT_NULL)) {
+ sb.append(" CAST(DEFAULT NULL)");
+ }
+ return sb.toString();
+ }
+
+ private String generateSearchQuery(AlgebricksBuiltinFunctions.ComparisonKind cmpKind, double p) {
+ String columnName = getColumnName(fieldType);
+ boolean castDefaultNull = options.contains(TestOption.CAST_DEFAULT_NULL);
+
+ StringBuilder sb = new StringBuilder(128);
+ addQueryProlog(sb);
+
+ String indexedFieldCastFn = castDefaultNull ? getTypeConstructorFunction(indexedType) : "";
+ String probeValueCastFn = probeType.getTypeName();
+
+ sb.append(String.format("SELECT VALUE %s ", columnName));
+ sb.append(String.format("FROM %s.%s ", dataverseName, datasetName));
+ sb.append(String.format("WHERE %s(%s) %s %s(%s) ", indexedFieldCastFn, columnName, getCmpOp(cmpKind),
+ probeValueCastFn, p));
+ sb.append(String.format("ORDER BY %s", columnName));
+ return sb.toString();
+ }
+
+ private String generateJoinQuery(boolean leftOuterJoin) {
+ if (!canJoin(fieldType, indexedType, probeType)) {
+ throw new IllegalStateException();
+ }
+
+ String columnName = getColumnName(fieldType);
+ String cmpOp = getCmpOp(AlgebricksBuiltinFunctions.ComparisonKind.EQ);
+ boolean castDefaultNull = options.contains(TestOption.CAST_DEFAULT_NULL);
+
+ StringBuilder sb = new StringBuilder(128);
+ addQueryProlog(sb);
+
+ String indexedFieldCastFn = castDefaultNull ? getTypeConstructorFunction(indexedType) : "";
+ String probeValueCastFn = probeType.getTypeName();
+
+ sb.append(String.format("SELECT t1.%s AS t1_%s, t2.%s AS t2_%s ", ID_COLUMN_NAME, ID_COLUMN_NAME,
+ ID_COLUMN_NAME, ID_COLUMN_NAME));
+ sb.append(String.format("FROM %s.%s t1 ", dataverseName, probeSourceDatasetName));
+ sb.append(String.format("%s JOIN %s.%s t2 ", leftOuterJoin ? "LEFT" : "", dataverseName, datasetName));
+ sb.append(String.format("ON %s(t1.%s) /* +%s */ %s %s(t2.%s) ", probeValueCastFn, columnName,
+ SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT.getIdentifier(), cmpOp, indexedFieldCastFn, columnName));
+ sb.append(String.format("ORDER BY t1_%s, t2_%s", ID_COLUMN_NAME, ID_COLUMN_NAME));
+ return sb.toString();
+ }
+
+ private void addQueryProlog(StringBuilder sb) {
+ sb.append(String.format(COMPILER_OPTION_FORMAT, CompilerProperties.COMPILER_SORT_PARALLEL_KEY, false));
+ sb.append(String.format(COMPILER_OPTION_FORMAT, CompilerProperties.COMPILER_INDEXONLY_KEY,
+ options.contains(TestOption.INDEX_ONLY_ENABLED)));
+ if (options.contains(TestOption.CAST_DEFAULT_NULL)) {
+ sb.append(String.format(COMPILER_OPTION_FORMAT, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, true));
+ }
+ }
+
+ private String getTypeConstructorFunction(BuiltinType probeType) {
+ return options.contains(TestOption.CAST_DEFAULT_NULL)
+ ? Objects.requireNonNull(TypeUtil.getTypeConstructorDefaultNull(probeType, false)).getName()
+ .replace('-', '_')
+ : probeType.getTypeName();
+ }
+
+ private String getCmpOp(AlgebricksBuiltinFunctions.ComparisonKind cmpKind) {
+ switch (cmpKind) {
+ case GT:
+ return ">";
+ case GE:
+ return ">=";
+ case LE:
+ return "<";
+ case LT:
+ return "<=";
+ case EQ:
+ return "=";
+ case NEQ:
+ default:
+ throw new IllegalStateException(String.valueOf(cmpKind));
+ }
+ }
+
+ private static boolean canJoin(BuiltinType fieldType, BuiltinType indexedType, BuiltinType probeType) {
+ return probeType.getTypeTag() == (indexedType != null ? indexedType : fieldType).getTypeTag();
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
index 2ccab70..e9db5fa 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
@@ -102,7 +102,7 @@
@Test
public void test() throws Exception {
- runTestCase(testcaseId, groupByClause, sqlQuery, sqlppQuery);
+ runTestCase(testcaseId, groupByClause, sqlQuery, sqlppQuery, null);
}
private static TestQuery generateQuery(int testcaseId, Random random) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
index 6f16400..7dfd0f0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
@@ -21,9 +21,13 @@
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.List;
import java.util.function.IntUnaryOperator;
+import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -39,7 +43,9 @@
private final String sqlppQuery;
- private final String desc;
+ private final boolean sqlCompatMode;
+
+ private final String description;
static final String PROJECT_FIELD = "unique1";
@@ -47,33 +53,36 @@
static final String FILTER_FIELD = JOIN_FIELD;
- static final char[] SHAPES = new char[] { 'c', 's' }; //TODO: 'q'
+ static final char[] SHAPES = new char[] { 'c', 's', 'q' };
- @Parameterized.Parameters(name = "SqlppRQGJoinsIT {index}: {3}")
+ @Parameterized.Parameters(name = "SqlppRQGJoinsIT {index}: {1}")
public static Collection<Object[]> tests() {
List<Object[]> testCases = new ArrayList<>();
IntUnaryOperator filterComputer = i -> 2 * (i + 1);
+ boolean[] allSqlCompatModes = new boolean[] { false, true };
String[] allJoinKinds = new String[] { "INNER", "LEFT", "RIGHT" };
String[] queryJoinKinds = new String[3];
int id = 0;
- for (String jk0 : allJoinKinds) {
- queryJoinKinds[0] = jk0;
- TestQuery q1 = generateQuery(queryJoinKinds, 1, filterComputer, SHAPES[0]);
- testCases.add(new Object[] { id++, q1.sqlQuery, q1.sqlppQuery, q1.summary });
+ for (boolean sqlCompatMode : allSqlCompatModes) {
+ for (String jk0 : allJoinKinds) {
+ queryJoinKinds[0] = jk0;
+ TestQuery q1 = generateQuery(queryJoinKinds, 1, filterComputer, SHAPES[0], sqlCompatMode);
+ addTestCase(testCases, id++, q1);
- for (char s : SHAPES) {
- for (String jk1 : allJoinKinds) {
- queryJoinKinds[1] = jk1;
- TestQuery q2 = generateQuery(queryJoinKinds, 2, filterComputer, s);
- testCases.add(new Object[] { id++, q2.sqlQuery, q2.sqlppQuery, q2.summary });
+ for (char s : SHAPES) {
+ for (String jk1 : allJoinKinds) {
+ queryJoinKinds[1] = jk1;
+ TestQuery q2 = generateQuery(queryJoinKinds, 2, filterComputer, s, sqlCompatMode);
+ addTestCase(testCases, id++, q2);
- for (String jk2 : allJoinKinds) {
- queryJoinKinds[2] = jk2;
- TestQuery q3 = generateQuery(queryJoinKinds, 3, filterComputer, s);
- testCases.add(new Object[] { id++, q3.sqlQuery, q3.sqlppQuery, q3.summary });
+ for (String jk2 : allJoinKinds) {
+ queryJoinKinds[2] = jk2;
+ TestQuery q3 = generateQuery(queryJoinKinds, 3, filterComputer, s, sqlCompatMode);
+ addTestCase(testCases, id++, q3);
+ }
}
}
}
@@ -83,7 +92,7 @@
}
private static TestQuery generateQuery(String[] joinKinds, int joinKindsSize, IntUnaryOperator filterComputer,
- char shape) {
+ char shape, boolean sqlCompatMode) {
int tCount = joinKindsSize + 1;
List<String> tDefs = new ArrayList<>(tCount);
for (int i = 0; i < tCount; i++) {
@@ -175,7 +184,7 @@
summary.append(joinKinds[i - 1]);
}
String projectFieldExprSql = String.format(fieldExprFormat, tThis, PROJECT_FIELD);
- String projectFieldExprSqlpp = missing2Null(projectFieldExprSql);
+ String projectFieldExprSqlpp = sqlCompatMode ? projectFieldExprSql : missing2Null(projectFieldExprSql);
String projectFieldAlias = String.format("%s_%s", tThis, PROJECT_FIELD);
String projectFormat = "%s AS %s";
selectClauseSql.append(String.format(projectFormat, projectFieldExprSql, projectFieldAlias));
@@ -188,22 +197,32 @@
summary.append(';').append(shape);
}
+ if (sqlCompatMode) {
+ summary.append(";sql-compat");
+ }
+
String queryFormat = "SELECT %s %sORDER BY %s";
String sqlQuery = String.format(queryFormat, selectClauseSql, fromClauseSql, orderbyClauseSql);
String sqlppQuery = String.format(queryFormat, selectClauseSqlpp, fromClauseSqlpp, orderbyClauseSqlpp);
- return new TestQuery(sqlQuery, sqlppQuery, summary.toString());
+ return new TestQuery(sqlQuery, sqlppQuery, sqlCompatMode, summary.toString());
}
private static String missing2Null(String expr) {
return String.format("if_missing(%s, null)", expr);
}
- public SqlppRQGJoinsIT(int testcaseId, String sqlQuery, String sqlppQuery, String desc) {
+ private static void addTestCase(List<Object[]> testCases, int id, TestQuery q) {
+ testCases.add(new Object[] { id, q.summary, q.sqlQuery, q.sqlppQuery, q.sqlCompatMode });
+ }
+
+ public SqlppRQGJoinsIT(int testcaseId, String description, String sqlQuery, String sqlppQuery,
+ boolean sqlCompatMode) {
this.testcaseId = testcaseId;
+ this.description = description;
this.sqlQuery = sqlQuery;
this.sqlppQuery = sqlppQuery;
- this.desc = desc;
+ this.sqlCompatMode = sqlCompatMode;
}
@BeforeClass
@@ -218,17 +237,27 @@
@Test
public void test() throws Exception {
- runTestCase(testcaseId, desc, sqlQuery, sqlppQuery);
+ List<TestCase.CompilationUnit.Parameter> params = null;
+ if (sqlCompatMode) {
+ TestCase.CompilationUnit.Parameter sqlCompatModeParam = new TestCase.CompilationUnit.Parameter();
+ sqlCompatModeParam.setName(QueryServiceRequestParameters.Parameter.SQL_COMPAT.str());
+ sqlCompatModeParam.setType(ParameterTypeEnum.JSON);
+ sqlCompatModeParam.setValue(String.valueOf(sqlCompatMode));
+ params = Collections.singletonList(sqlCompatModeParam);
+ }
+ runTestCase(testcaseId, description, sqlQuery, sqlppQuery, params);
}
private static class TestQuery {
final String sqlQuery;
final String sqlppQuery;
final String summary;
+ boolean sqlCompatMode;
- TestQuery(String sqlQuery, String sqlppQuery, String summary) {
+ TestQuery(String sqlQuery, String sqlppQuery, boolean sqlCompatMode, String summary) {
this.sqlQuery = sqlQuery;
this.sqlppQuery = sqlppQuery;
+ this.sqlCompatMode = sqlCompatMode;
this.summary = summary;
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
index 434b368..dbf214b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
@@ -25,6 +25,7 @@
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.PrintWriter;
+import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
@@ -54,6 +55,7 @@
import org.apache.asterix.test.common.TestExecutor;
import org.apache.asterix.test.common.TestHelper;
import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.TestCase;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.logging.log4j.LogManager;
@@ -68,18 +70,13 @@
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
-// Prerequisite:
-// setenv TESTCONTAINERS_RYUK_DISABLED true
-
public abstract class SqlppRQGTestBase {
private static final Logger LOGGER = LogManager.getLogger(SqlppRQGTestBase.class);
- protected static final String TESTCONTAINERS_RYUK_DISABLED = "TESTCONTAINERS_RYUK_DISABLED";
-
protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
- protected static final String POSTGRES_IMAGE = "postgres:12.2";
+ protected static final String POSTGRES_IMAGE = "postgres:13.4";
protected static final String TABLE_NAME = "tenk";
@@ -143,8 +140,8 @@
return result;
}
- protected void runTestCase(int testcaseId, String testcaseDescription, String sqlQuery, String sqlppQuery)
- throws Exception {
+ protected void runTestCase(int testcaseId, String testcaseDescription, String sqlQuery, String sqlppQuery,
+ List<TestCase.CompilationUnit.Parameter> params) throws Exception {
LOGGER.info(String.format("Starting testcase #%d: %s", testcaseId, testcaseDescription));
LOGGER.info("Running SQL");
@@ -158,16 +155,17 @@
LOGGER.info("Running SQL++");
LOGGER.info(sqlppQuery);
ArrayNode sqlppResult;
- try (InputStream resultStream = testExecutor.executeQueryService(sqlppQuery,
- testExecutor.getEndpoint(Servlets.QUERY_SERVICE), TestCaseContext.OutputFormat.ADM)) {
+ URI endpoint = testExecutor.getEndpoint(Servlets.QUERY_SERVICE);
+ try (InputStream resultStream = testExecutor.executeQueryService(sqlppQuery, TestCaseContext.OutputFormat.ADM,
+ endpoint, params == null ? Collections.emptyList() : params, false, StandardCharsets.UTF_8)) {
sqlppResult = asJson(
ResultExtractor.extract(resultStream, StandardCharsets.UTF_8, TestCaseContext.OutputFormat.ADM));
}
boolean eq = TestHelper.equalJson(sqlResult, sqlppResult, false, false, false, null);
- File sqlResultFile = writeResult(sqlResult, testcaseId, "sql", testcaseDescription);
- File sqlppResultFile = writeResult(sqlppResult, testcaseId, "sqlpp", testcaseDescription);
+ File sqlResultFile = writeResult(outputDir, sqlResult, testcaseId, "sql", testcaseDescription);
+ File sqlppResultFile = writeResult(outputDir, sqlppResult, testcaseId, "sqlpp", testcaseDescription);
if (!eq) {
/*
@@ -328,7 +326,13 @@
}
}
- protected File writeResult(ArrayNode result, int testcaseId, String resultKind, String comment) throws IOException {
+ static File writeResult(Path outputDir, ArrayNode result, int testcaseId, String resultKind, String comment)
+ throws IOException {
+ return writeResult(outputDir, result, testcaseId, resultKind, comment, SqlppRQGTestBase::prettyPrint);
+ }
+
+ static File writeResult(Path outputDir, ArrayNode result, int testcaseId, String resultKind, String comment,
+ JsonNodePrinter printer) throws IOException {
File outDir = outputDir.toFile();
String outFileName = String.format("%d.%s.txt", testcaseId, resultKind);
FileUtils.forceMkdir(outDir);
@@ -337,12 +341,20 @@
pw.print("---");
pw.println(comment);
for (int i = 0, ln = result.size(); i < ln; i++) {
- pw.println(ResultExtractor.prettyPrint(result.get(i)));
+ printer.print(pw, result.get(i));
}
}
return outFile;
}
+ public interface JsonNodePrinter {
+ void print(PrintWriter out, JsonNode node) throws IOException;
+ }
+
+ private static void prettyPrint(PrintWriter out, JsonNode node) throws JsonProcessingException {
+ out.println(ResultExtractor.prettyPrint(node));
+ }
+
protected static <T> List<T> randomize(Collection<T> input, Random random) {
List<T> output = new ArrayList<>(input);
Collections.shuffle(output, random);
@@ -369,7 +381,7 @@
protected static void startAsterix() throws Exception {
testExecutor = new TestExecutor();
- LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor, false);
loadAsterixData();
}
@@ -378,10 +390,6 @@
}
protected static void startPostgres() throws SQLException, IOException {
- if (!Boolean.parseBoolean(System.getenv(TESTCONTAINERS_RYUK_DISABLED))) {
- throw new IllegalStateException(
- String.format("Set environment variable %s=%s", TESTCONTAINERS_RYUK_DISABLED, true));
- }
LOGGER.info("Starting Postgres");
postgres = new PostgreSQLContainer<>(POSTGRES_IMAGE);
postgres.start();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 2039d53..8a87de7 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -19,6 +19,9 @@
package org.apache.asterix.test.sqlpp;
import static org.apache.hyracks.util.file.FileUtil.canonicalize;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -159,7 +162,17 @@
return mockDataverse;
}
});
- when(metadataProvider.findDataset(Mockito.<DataverseName> any(), Mockito.<String> any()))
+ when(metadataProvider.findDataset(any(DataverseName.class), anyString())).thenAnswer(new Answer<Dataset>() {
+ @Override
+ public Dataset answer(InvocationOnMock invocation) {
+ Object[] args = invocation.getArguments();
+ final Dataset mockDataset = mock(Dataset.class);
+ when(mockDataset.getDataverseName()).thenReturn((DataverseName) args[0]);
+ when(mockDataset.getDatasetName()).thenReturn((String) args[1]);
+ return mockDataset;
+ }
+ });
+ when(metadataProvider.findDataset(any(DataverseName.class), anyString(), anyBoolean()))
.thenAnswer(new Answer<Dataset>() {
@Override
public Dataset answer(InvocationOnMock invocation) {
@@ -170,7 +183,7 @@
return mockDataset;
}
});
- when(metadataProvider.lookupUserDefinedFunction(Mockito.<FunctionSignature> any()))
+ when(metadataProvider.lookupUserDefinedFunction(any(FunctionSignature.class)))
.thenAnswer(new Answer<Function>() {
@Override
public Function answer(InvocationOnMock invocation) {
@@ -189,7 +202,7 @@
if (st.getKind() == Statement.Kind.QUERY) {
Query query = (Query) st;
IQueryRewriter rewriter = sqlppRewriterFactory.createQueryRewriter();
- LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions,
+ LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions, null,
TestUtils.NOOP_WARNING_COLLECTOR, query.getVarCounter());
rewrite(rewriter, query, rwContext);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java
index 50118fc..59efef4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/PersistentLocalResourceRepositoryTest.java
@@ -189,7 +189,7 @@
DatasetResourceReference drr = DatasetResourceReference.of(localResource);
IIndexCheckpointManagerProvider indexCheckpointManagerProvider = ncAppCtx.getIndexCheckpointManagerProvider();
IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(drr);
- indexCheckpointManager.advanceValidComponentSequence(validComponentSequence);
+ indexCheckpointManager.advanceValidComponent(validComponentSequence, 1);
// create components to be merged
String btree = "_b";
String filter = "_f";
diff --git a/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py b/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py
index 9058a01..0d52f01 100644
--- a/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py
+++ b/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py
@@ -25,6 +25,9 @@
def roundtrip(self, *args):
return args
+ def roundstr(self, arg):
+ return arg
+
def warning(self):
raise ArithmeticError("oof")
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.3.query.sqlpp
index 280fff0..022c5a1 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.3.query.sqlpp
@@ -17,7 +17,7 @@
* under the License.
*/
/*
- * Description : create a dataset using time as the primary key
+ * Description : create a dataset using time as the primary key
* Expected Res : Success
* Date : 7 May 2013
* Issue : 363
@@ -27,4 +27,5 @@
SELECT VALUE ds
FROM Employee ds
-WHERE ds.id > time("07:07:07.777Z");
+WHERE ds.id > time("07:07:07.777Z")
+ORDER BY ds.name;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.sqlpp
index dffccc7..bec8fda 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.sqlpp
@@ -17,7 +17,7 @@
* under the License.
*/
/*
- * Description : create a dataset using time for the secondary index
+ * Description : create a dataset using time for the secondary index
* Expected Res : Success
* Date : 26 May 2013
* Issue : 461
@@ -27,4 +27,5 @@
SELECT VALUE ds
FROM Employee ds
-WHERE ds.tm > time("07:07:07.777Z");
+WHERE ds.tm > time("07:07:07.777Z")
+ORDER BY ds.name;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/exception/exception_create_system_view/exception_create_system_view.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/exception/exception_create_system_view/exception_create_system_view.1.ddl.sqlpp
new file mode 100644
index 0000000..5e69371
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/exception/exception_create_system_view/exception_create_system_view.1.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+create view Metadata.MyMetaView as
+ select count(*) from Metadata.`Dataset`;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_1/issue363_temporal_key_1.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_1/issue363_temporal_key_1.1.adm
index 57b8610..e5033de 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_1/issue363_temporal_key_1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_1/issue363_temporal_key_1.1.adm
@@ -1 +1 @@
-{ "id": datetime("2013-01-01T00:00:00.000Z"), "name": "Bob" }
+{ "id": datetime("2013-01-01T00:00:00.000"), "name": "Bob" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_2/issue363_temporal_key_2.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_2/issue363_temporal_key_2.1.adm
index fd08d4f..71895f8 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_2/issue363_temporal_key_2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_2/issue363_temporal_key_2.1.adm
@@ -1,2 +1,2 @@
-{ "id": time("12:37:19.000Z"), "name": "Alex" }
-{ "id": time("01:39:17.948Z"), "name": "Bob" }
+{ "id": time("20:37:19.000"), "name": "Alex" }
+{ "id": time("21:39:17.948"), "name": "Bob" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm
index e3bf18d..9422a0e 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm
@@ -1 +1 @@
-{ "id": datetime("2013-01-01T00:00:00.000Z"), "dt": datetime("2013-01-01T00:00:00.000Z"), "name": "Bob" }
+{ "id": datetime("2013-01-01T00:00:00.000"), "dt": datetime("2013-01-01T00:00:00.000"), "name": "Bob" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm
index a26d069..f2cf80e 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm
@@ -1,2 +1,2 @@
-{ "id": time("12:37:19.000Z"), "tm": time("12:37:19.000Z"), "name": "Alex" }
-{ "id": time("01:39:17.948Z"), "tm": time("01:39:17.948Z"), "name": "Bob" }
+{ "id": time("20:37:19.000"), "tm": time("20:37:19.000"), "name": "Alex" }
+{ "id": time("21:39:17.948"), "tm": time("21:39:17.948"), "name": "Bob" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
index f770710..5f1d071 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Tue Jun 21 15:54:27 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Tue Jun 21 15:54:27 PDT 2016", "PendingOp": 0, "ExcludeUnknownKey": false }
{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:27 PDT 2016", "PendingOp": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index dab91a3..300715d1 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Tue Jun 21 15:54:28 PDT 2016", "PendingOp": 0, "SearchKeyType": [ "string" ], "IsEnforced": true }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Tue Jun 21 15:54:28 PDT 2016", "PendingOp": 0, "SearchKeyType": [ "string" ], "IsEnforced": true, "ExcludeUnknownKey": false }
{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Tue Jun 21 15:54:28 PDT 2016", "PendingOp": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index a29dc48..dc6ae3d 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -363,6 +363,12 @@
</compilation-unit>
</test-case>
<test-case FilePath="exception">
+ <compilation-unit name="exception_create_system_view">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1079: Compilation error: Invalid operation - Cannot create a view belonging to the dataverse: Metadata (in line 20, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="exception">
<compilation-unit name="exception_declare_system_function">
<output-dir compare="Text">none</output-dir>
<expected-error>ASX1079: Compilation error: Invalid operation - Cannot declare a function belonging to the dataverse: Metadata (in line 22, at column 1)</expected-error>
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-1.sqlpp
new file mode 100644
index 0000000..808226a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Quantifying on items in an array.
+FROM Dataset1 D
+WHERE ANY AND EVERY I IN D.items
+ SATISFIES I = 1
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-2.sqlpp
new file mode 100644
index 0000000..4e2690e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-2.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Quantifying on items in an array, that are within an array.
+FROM Dataset1 D
+WHERE ANY AND EVERY I IN D.items
+ SATISFIES (
+ SOME AND EVERY J IN I.items
+ SATISFIES J = 1
+ )
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-3.sqlpp
new file mode 100644
index 0000000..e0a9673
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-3.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Quantifying on items in two arrays.
+FROM Dataset1 D
+WHERE SOME AND EVERY I IN D.items, J IN D.other_items
+ SATISFIES I = 1 AND J = 2
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query1.sqlpp
new file mode 100644
index 0000000..29b6012
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query1.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Explicit UNNEST query with all fields of a composite index.
+FROM Dataset1 D
+UNNEST D.items DI
+WHERE D.field1 = 1 AND
+ DI.field2 = 2 AND
+ DI.field3 = 3 AND
+ D.field4 = 4
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query2.sqlpp
new file mode 100644
index 0000000..cd56abc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query2.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Explicit UNNEST query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset1 D
+UNNEST D.items DI
+WHERE D.field1 = 1 AND
+ DI.field2 = 2 AND
+ DI.field3 = 3 AND
+ DI.field3_notindexed = 3 AND
+ D.field4 = 4 AND
+ D.field4_notindexed = 4
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query3.sqlpp
new file mode 100644
index 0000000..6f0cca8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query3.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Explicit UNNEST query with all fields of a composite index, with extra clauses on non-indexed fields (2).
+FROM Dataset1 D
+UNNEST D.outer_items DOI
+UNNEST DOI.inner_items DII
+WHERE D.field1 = 1 AND
+ DOI.field2_notindexed = 2 AND
+ DII.field2 = 2 AND
+ DII.field3 = 3 AND
+ DII.field3_notindexed = 3 AND
+ D.field4 = 4
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query4.sqlpp
new file mode 100644
index 0000000..d0ff566
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query4.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Existential quantification query with all fields of a composite index.
+FROM Dataset1 D
+WHERE D.field1 = 1 AND
+ ( SOME DI IN D.items
+ SATISFIES DI.field2 = 2 AND
+ DI.field3 = 3 ) AND
+ D.field4 = 4
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query5.sqlpp
new file mode 100644
index 0000000..0668251
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query5.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Existential quantification query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset1 D
+WHERE D.field1 = 1 AND
+ ( SOME DOI IN D.outer_items
+ SATISFIES DOI.field2_notindexed = 2 AND
+ ( SOME DII IN DOI.inner_items
+ SATISFIES DII.field2 = 2 AND
+ DII.field3 = 3 AND
+ DII.field3_notindexed = 3 ) ) AND
+ D.field4 = 4
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query6.sqlpp
new file mode 100644
index 0000000..d5bb4ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query6.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET Dataset2 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Explicit UNNEST join query with all fields of a composite index.
+FROM Dataset2 D2
+INNER JOIN (
+ FROM Dataset1 D1
+ UNNEST D1.items D1I
+ SELECT D1.field1, D1I.field2, D1I.field3, D1.field4
+) AS D1IV
+ON D1IV.field1 /* +indexnl */ = TO_BIGINT(D2.field1) AND
+ D1IV.field2 /* +indexnl */ = TO_BIGINT(D2.field2) AND
+ D1IV.field3 /* +indexnl */ = TO_BIGINT(D2.field3) AND
+ D1IV.field4 /* +indexnl */ = TO_BIGINT(D2.field4)
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query7.sqlpp
new file mode 100644
index 0000000..7d257ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query7.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET Dataset2 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Explicit UNNEST join query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset2 D2
+INNER JOIN (
+ FROM Dataset1 D1
+ UNNEST D1.items D1I
+ SELECT D1.field1_notindexed, D1.field1, D1I.field2, D1I.field3, D1I.field3_notindexed, D1.field4
+) AS D1IV
+ON D1IV.field1 /* +indexnl */ = TO_BIGINT(D2.field1) AND
+ D1IV.field1_notindexed = TO_BIGINT(D2.field1_notindexed) AND
+ D1IV.field2 /* +indexnl */ = TO_BIGINT(D2.field2) AND
+ D1IV.field3 /* +indexnl */ = TO_BIGINT(D2.field3) AND
+ D1IV.field3_notindexed = TO_BIGINT(D2.field3_notindexed) AND
+ D1IV.field4 /* +indexnl */ = TO_BIGINT(D2.field4)
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query8.sqlpp
new file mode 100644
index 0000000..96e5040
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query8.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET Dataset2 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Existential quantification join query with all fields of a composite index.
+FROM Dataset2 D2,
+ Dataset1 D1
+WHERE D1.field1 /* +indexnl */ = TO_BIGINT(D2.field1) AND
+ ( SOME D1I IN D1.items
+ SATISFIES D1I.field2 /* +indexnl */ = TO_BIGINT(D2.field2) AND
+ D1I.field3 /* +indexnl */ = TO_BIGINT(D2.field3) ) AND
+ D1.field4 /* +indexnl */ = TO_BIGINT(D2.field4)
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query9.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query9.sqlpp
new file mode 100644
index 0000000..f16885e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/atomic-and-array-queries/query9.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET Dataset2 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+
+-- Existential quantification query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset2 D2,
+ Dataset1 D1
+WHERE D1.field1 /* +indexnl */ = TO_BIGINT(D2.field1) AND
+ ( SOME DOI IN D1.outer_items
+ SATISFIES DOI.field2_notindexed = TO_BIGINT(D2.field2_notindexed) AND
+ ( SOME DII IN DOI.inner_items
+ SATISFIES DII.field2 /* +indexnl */ = TO_BIGINT(D2.field2) AND
+ DII.field3 /* +indexnl */ = TO_BIGINT(D2.field3) AND
+ DII.field3_notindexed = TO_BIGINT(D2.field3_notindexed) ) ) AND
+ D1.field4 /* +indexnl */ = TO_BIGINT(D2.field4)
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..e5448d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query1.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+ };
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE M.date /*+ indexnl */ IN C.dates AND
+ C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..99b0c17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query2.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+ };
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg" AND
+ ( SOME D IN C.dates
+ SATISFIES M.date /*+ indexnl */ = D )
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query3.sqlpp
new file mode 100644
index 0000000..bde1c66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query3.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+ };
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg" AND
+ LEN(C.dates) > 0 AND
+ ( EVERY D IN C.dates
+ SATISFIES M.date /*+ indexnl */ = D )
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query4.sqlpp
new file mode 100644
index 0000000..bd14312
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query4.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+ };
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckinA(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinB(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinADates ON YelpCheckinA (UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBDates ON YelpCheckinB (UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A, YelpCheckinB B
+WHERE M.date /*+ indexnl */ IN A.dates AND
+ M.date /*+ indexnl */ IN B.dates
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query5.sqlpp
new file mode 100644
index 0000000..5c78253
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-1/query5.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+ };
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg" AND
+ ( SOME AND EVERY D IN C.dates
+ SATISFIES M.date /*+ indexnl */ = D )
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..2cc0978
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query1.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE M.date /*+ indexnl */ IN C.checkin_times.dates AND
+ C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..a7ac426
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query2.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg" AND
+ ( SOME AND EVERY D IN C.checkin_times.dates
+ SATISFIES M.date /*+ indexnl */ = D )
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query3.sqlpp
new file mode 100644
index 0000000..6c0487f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-2/query3.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+ };
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
+
+-- This should NOT be optimized, there is no non-array-emptiness clause.
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg" AND
+ ( EVERY D IN C.checkin_times.dates
+ SATISFIES M.date /*+ indexnl */ = D )
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..6d8615e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-3/query1.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE ( SOME D IN C.checkin_times
+ SATISFIES D.date /*+ indexnl */ = M.date ) AND
+ C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg"
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..3c712f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-3/query2.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE ( SOME AND EVERY D IN C.checkin_times
+ SATISFIES D.date /*+ indexnl */ = M.date ) AND
+ C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg"
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..9e809de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query1.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE ( SOME CT IN C.checkin_times
+ SATISFIES ( SOME D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date ) ) AND
+ C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..1023a24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query2.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE ( EVERY CT IN C.checkin_times
+ SATISFIES ( SOME D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date ) ) AND
+ LEN (C.checkin_times) > 0 AND
+ C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query3.sqlpp
new file mode 100644
index 0000000..7305666
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query3.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE ( SOME AND EVERY CT IN C.checkin_times
+ SATISFIES ( SOME AND EVERY D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date ) ) AND
+ C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query4.sqlpp
new file mode 100644
index 0000000..a7ab028
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-quantified-queries/use-case-4/query4.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckinA(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinB(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDatesA ON YelpCheckinA (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesB ON YelpCheckinB (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A, YelpCheckinB B
+WHERE ( SOME AND EVERY CT IN A.checkin_times
+ SATISFIES SOME AND EVERY D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date ) AND
+ ( SOME AND EVERY CT IN B.checkin_times
+ SATISFIES SOME AND EVERY D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date )
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp
index 3960ee6..653a84f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp
@@ -36,7 +36,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp
index a231ac7..f190af9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp
@@ -36,7 +36,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp
index 8fe3c15..db22a1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp
@@ -36,7 +36,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckinDateMarkers M
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query4.sqlpp
new file mode 100644
index 0000000..909affd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query4.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.dates D
+) AS CD ON CD /*+ indexnl */ = M.datetime;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp
index c10c2a9..bf99dff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp
@@ -39,7 +39,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp
index 8e5cf1e..3d9f1d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp
@@ -39,7 +39,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp
index b8e9b49..10538b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp
@@ -39,7 +39,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckinDateMarkers M
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query4.sqlpp
new file mode 100644
index 0000000..6f33b3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query4.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.checkin_times.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp
index 91f6d8a..9551441 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp
@@ -39,7 +39,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp
index d4742ce..b8ef5ec 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp
@@ -39,7 +39,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp
index e3afa1e..2a3337c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp
@@ -39,7 +39,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp
index f5d3c52..20cfec9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp
@@ -39,7 +39,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckinDateMarkers M
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query5.sqlpp
new file mode 100644
index 0000000..1888ba4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query5.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ SELECT VALUE D.date
+ FROM YelpCheckin C, C.checkin_times D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp
index 1afd508..ebef766 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp
@@ -40,7 +40,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp
index b911a20..042bad1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp
@@ -40,7 +40,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp
index 5a23cb4..5b7a47e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp
@@ -40,7 +40,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT M.marker
FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp
index c142d1a..7e21e98 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp
@@ -40,7 +40,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckinDateMarkers M
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query5.sqlpp
new file mode 100644
index 0000000..f34dc23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query5.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query1.sqlpp
new file mode 100644
index 0000000..6094af8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query1.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST items : bigint ) EXCLUDE UNKNOWN KEY;
+
+-- Single UNNEST, open index.
+FROM Dataset1 D1
+WHERE SOME I IN D1.items, J IN D1.other_items
+ SATISFIES I = 1 AND J = 2
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query2.sqlpp
new file mode 100644
index 0000000..8199a40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query2.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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid,
+ items: [bigint]
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST items ) EXCLUDE UNKNOWN KEY;
+
+-- Single UNNEST, closed index.
+FROM Dataset1 D1
+WHERE SOME I IN D1.items, J IN D1.other_items
+ SATISFIES I = 1 AND J = 2
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query3.sqlpp
new file mode 100644
index 0000000..9dcba5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query3.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST items
+ SELECT item : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Single UNNEST w/ SELECT, open index.
+FROM Dataset1 D1
+WHERE SOME I IN D1.items, J IN D1.other_items
+ SATISFIES I.item = 1 AND J.item = 2
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query4.sqlpp
new file mode 100644
index 0000000..30f36c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query4.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid,
+ items: [{
+ item: bigint
+ }]
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST items
+ SELECT item )
+EXCLUDE UNKNOWN KEY;
+
+-- Single UNNEST w/ SELECT, closed index.
+FROM Dataset1 D1
+WHERE SOME I IN D1.items, J IN D1.other_items
+ SATISFIES I.item = 1 AND J.item = 2
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query5.sqlpp
new file mode 100644
index 0000000..1fdabbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query5.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT item : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Double UNNEST w/ SELECT, open index.
+FROM Dataset1 D1
+WHERE SOME I1 IN D1.outer_items, J IN D1.other_items
+ SATISFIES J.item = 2 AND ( SOME I2 IN I1.inner_items, J2 IN I1.other_inner_items
+ SATISFIES I2.item = 1 AND J2.item = 2 )
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query6.sqlpp
new file mode 100644
index 0000000..dee73ad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query6.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid,
+ outer_items: [{
+ inner_items: [{
+ item: bigint
+ }]
+ }]
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT item )
+EXCLUDE UNKNOWN KEY;
+
+-- Double UNNEST w/ SELECT, open index.
+FROM Dataset1 D1
+WHERE SOME I1 IN D1.outer_items, J IN D1.other_items
+ SATISFIES J.item = 2 AND ( SOME I2 IN I1.inner_items, J2 IN I1.other_inner_items
+ SATISFIES I2.item = 1 AND J2.item = 2 )
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query7.sqlpp
new file mode 100644
index 0000000..7b5c221
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query7.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid,
+ items: [{
+ item: bigint
+ }]
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST items
+ SELECT item )
+EXCLUDE UNKNOWN KEY;
+
+-- Single UNNEST w/ SELECT, closed index.
+FROM Dataset1 D1
+WHERE SOME AND EVERY I IN D1.items, J IN D1.other_items
+ SATISFIES I.item = 1 AND J.item = 2
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query8.sqlpp
new file mode 100644
index 0000000..e930ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query8.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid,
+ outer_items: [{
+ inner_items: [{
+ item: bigint
+ }]
+ }]
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT item )
+EXCLUDE UNKNOWN KEY;
+
+-- Double UNNEST w/ SELECT, open index.
+FROM Dataset1 D1
+WHERE SOME AND EVERY I1 IN D1.outer_items, J IN D1.other_items
+ SATISFIES J.item = 2 AND ( SOME AND EVERY I2 IN I1.inner_items, J2 IN I1.other_inner_items
+ SATISFIES I2.item = 1 AND J2.item = 2 )
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query9.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query9.sqlpp
new file mode 100644
index 0000000..8a48b3d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query9.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid,
+ outer_items: [{
+ inner_item: bigint,
+ inner_items: [{
+ item: bigint
+ }]
+ }]
+};
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT item )
+EXCLUDE UNKNOWN KEY;
+
+-- Double UNNEST w/ SELECT, open index and multiple quantification.
+FROM Dataset1 D1
+WHERE SOME AND EVERY I1 IN D1.outer_items,
+ J IN D1.other_items
+ SATISFIES J.item = 2 AND
+ I1.inner_item = 3 AND
+ ( SOME AND EVERY I2 IN I1.inner_items,
+ J2 IN I1.other_inner_items
+ SATISFIES I2.item = 1 AND
+ J2.item = 2 )
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp
index f267701..2f61919 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp
index 6cbfdff..710fcea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp
index d2dc752..25174e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp
@@ -29,10 +29,10 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C
-WHERE LEN(C.dates) > 0 AND
+WHERE 0 < LEN(C.dates) AND
(EVERY D IN C.dates
SATISFIES D > "2016" AND D < "2017");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query4.sqlpp
new file mode 100644
index 0000000..2082e73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query4.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.dates
+ SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query5.sqlpp
new file mode 100644
index 0000000..11a5929
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query5.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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST dates)
+EXCLUDE UNKNOWN KEY;
+
+-- The following query should NOT be optimized.
+FROM YelpCheckin C
+WHERE 0 < LEN(C.dates_notindexed) AND
+ 0 < LEN(C.dates_notindexed_2) AND
+ ( EVERY D IN C.dates
+ SATISFIES D > "2016" AND D < "2017" )
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp
index 145f8f21..34396bf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp
index 65e15fc..b8e1a49 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp
index 24ae2e4..a344125 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp
@@ -32,10 +32,9 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C
-WHERE LEN(C.checkin_times.dates) > 0 AND
- (EVERY D IN C.checkin_times.dates
- SATISFIES D > "2016" AND D < "2017");
+WHERE SOME AND EVERY D IN C.checkin_times.dates
+ SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query4.sqlpp
new file mode 100644
index 0000000..760f1f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query4.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin ( UNNEST checkin_times.dates )
+EXCLUDE UNKNOWN KEY;
+
+-- This query should NOT be optimized.
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.checkin_times.dates_notindexed
+ SATISFIES D > "2016" AND D < "2017"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query5.sqlpp
new file mode 100644
index 0000000..a3e6912
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query5.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin ( UNNEST checkin_times.dates )
+EXCLUDE UNKNOWN KEY;
+
+-- This query should NOT be optimized.
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.checkin_times.dates
+ SATISFIES LOWER(D) = "2016"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp
index 3c07b9d..eb78853 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp
index c8d5c23..de5e160 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp
index 52f533d..fbdc728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp
@@ -32,10 +32,9 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C
-WHERE LEN(C.checkin_times) > 0 AND
- (EVERY D IN C.checkin_times
- SATISFIES D.date BETWEEN "2016" AND "2017");
+WHERE SOME AND EVERY D IN C.checkin_times
+ SATISFIES D.date BETWEEN "2016" AND "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query4.sqlpp
new file mode 100644
index 0000000..6660a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query4.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin ( UNNEST checkin_times SELECT date )
+EXCLUDE UNKNOWN KEY;
+
+-- The following query should NOT be optimized.
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.checkin_times
+ SATISFIES D.date_notindexed BETWEEN "2016" AND "2017"
+SELECT C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp
index 1bd9564..1ab22fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp
index 204e589..4f3b19c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp
index 8b3fd61..4bd2d75 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp
@@ -32,14 +32,12 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
-/* TODO (GLENN) Support checking for nested length clauses (i.e. EVERY, then EVERY). */
SELECT C.business_id
FROM YelpCheckin C
-WHERE LEN(C.checkin_times) > 0 AND
- (EVERY CT IN C.checkin_times
- SATISFIES (
+WHERE SOME AND EVERY CT IN C.checkin_times
+ SATISFIES (
SOME D IN CT.dates
SATISFIES "2019-06-07" = D
- ));
\ No newline at end of file
+ );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query4.sqlpp
new file mode 100644
index 0000000..1a9d72b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query4.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckin C
+WHERE SOME CT IN C.checkin_times
+ SATISFIES SOME AND EVERY D IN CT.dates
+ SATISFIES D > "2019-06-07"
+SELECT C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query5.sqlpp
new file mode 100644
index 0000000..809dd09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query5.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckin C
+WHERE SOME AND EVERY CT IN C.checkin_times
+ SATISFIES SOME AND EVERY D IN CT.dates
+ SATISFIES D > "2019-06-07"
+SELECT C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query6.sqlpp
new file mode 100644
index 0000000..5b53678
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query6.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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+// This should NOT be optimized, we are missing the non-emptiness conjunct.
+FROM YelpCheckin C
+WHERE ( EVERY CT IN C.checkin_times
+ SATISFIES ( LEN(CT.dates) > 0 AND
+ ( EVERY D IN CT.dates
+ SATISFIES D > "2019-06-07" ) ) )
+SELECT C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query7.sqlpp
new file mode 100644
index 0000000..f2887a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query7.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+FROM YelpCheckin C
+WHERE 0 < LEN(C.checkin_times) AND
+ ( EVERY CT IN C.checkin_times
+ SATISFIES ( ABS(CT.num) > 0 AND
+ 0 < LEN(CT.dates) AND
+ ( EVERY D IN CT.dates
+ SATISFIES D > "2019-06-07" ) ) )
+SELECT C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp
index d4bb5ff..522e988 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp
index ae997b3..d097dcc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp
index 602e924..7f2967c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp
index 0d94cce..ed7fffb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp
index 52d826a..9d86aab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp
index 0755f55..5e89ea4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp
index 49949a0..d8ec1ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp
index 2e3c0a8..6009d3d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times CT, CT.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp
index 89dc790..0f2cfc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_times CT, CT.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp
index 660f62b..8baedb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp
@@ -36,7 +36,8 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates)
+EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp
index ce7fad8..8868afb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp
@@ -36,7 +36,8 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates)
+EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp
index 80a6bcc..5592328 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp
index 3ea8ac2..c8e894f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp
index b9970c5..6520acf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp
index 022d5ea..71f6787 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query3.sqlpp
new file mode 100644
index 0000000..31e3ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query3.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDatesTimes
+ON YelpCheckin ( UNNEST checkin_times SELECT date, time )
+EXCLUDE UNKNOWN KEY;
+
+-- The following query should NOT be optimized.
+FROM YelpCheckin C,
+ C.checkin_times D
+WHERE D.date_notindexed BETWEEN "2016" AND "2017" AND
+ D.time BETWEEN "00:00:00" AND "12:00:00"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp
index 9b36d62..c34af5c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp
index 2cffd50..32f426c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp
@@ -29,7 +29,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query1.sqlpp
new file mode 100644
index 0000000..8cde896
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query1.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Fully open index 1. Unnest flags: [0, 1, 0]
+CREATE INDEX testIndex1
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ SELECT open_field_3a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.open_field_1.open_field_2 F
+WHERE F.open_field_3a > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query2.sqlpp
new file mode 100644
index 0000000..9bd14cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query2.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Fully open index 2. Unnest flags: [0, 1, 0, 0]
+CREATE INDEX testIndex2
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ SELECT open_field_3b.open_field_4 : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.open_field_1.open_field_2 F
+WHERE F.open_field_3b.open_field_4 > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query3.sqlpp
new file mode 100644
index 0000000..e9b5fdd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query3.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Fully open index 3. Unnest flags: [0, 1, 0, 1]
+CREATE INDEX testIndex3
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ UNNEST open_field_3c.open_field_4a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.open_field_1.open_field_2 F1
+UNNEST F1.open_field_3c.open_field_4a F2
+WHERE F2 > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query4.sqlpp
new file mode 100644
index 0000000..b99644c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query4.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Fully open index 4. Unnest flags: [0, 1, 0, 1, 0]
+CREATE INDEX testIndex4
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ UNNEST open_field_3c.open_field_4b
+ SELECT open_field_5 : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.open_field_1.open_field_2 F1
+UNNEST F1.open_field_3c.open_field_4b F2
+WHERE F2.open_field_5 > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query5.sqlpp
new file mode 100644
index 0000000..bb23de6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query5.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Partially open index 1. [0, 1, 0]
+CREATE INDEX testIndex1c
+ON TestDataset ( UNNEST closed_field_1.open_field_2
+ SELECT open_field_3a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.closed_field_1.open_field_2 F
+WHERE F.open_field_3a > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query6.sqlpp
new file mode 100644
index 0000000..905df22
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query6.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Partially open index 2. Unnest flags: [0, 1, 0, 0]
+CREATE INDEX testIndex2c
+ON TestDataset ( UNNEST closed_field_1.closed_field_2
+ SELECT open_field_3b.open_field_4 : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.closed_field_1.closed_field_2 F
+WHERE F.open_field_3b.open_field_4 > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query7.sqlpp
new file mode 100644
index 0000000..c0d8b30
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query7.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Partially open index 3. Unnest flags: [0, 1, 0, 1]
+CREATE INDEX testIndex3c
+ON TestDataset ( UNNEST closed_field_1.closed_field_2
+ UNNEST closed_field_3.open_field_4a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.closed_field_1.closed_field_2 F1
+UNNEST F1.closed_field_3.open_field_4a F2
+WHERE F2 > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query8.sqlpp
new file mode 100644
index 0000000..6731591
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query8.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }],
+ closed_field_4a: bigint,
+ closed_field_4b: [ bigint ]
+ },
+ closed_field_3a: bigint
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+-- Partially open index 4. Unnest flags: [0, 1, 0, 1, 0]
+CREATE INDEX testIndex4c
+ON TestDataset ( UNNEST closed_field_1.closed_field_2
+ UNNEST closed_field_3.closed_field_4
+ SELECT open_field_5 : bigint )
+EXCLUDE UNKNOWN KEY;
+
+FROM TestDataset D
+UNNEST D.closed_field_1.closed_field_2 F1
+UNNEST F1.closed_field_3.closed_field_4 F2
+WHERE F2.open_field_5 > 0
+SELECT *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp
index 9e24961..4cd0e07 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp
@@ -16,8 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-SET `compiler.arrayindex` "true";
-
DROP DATAVERSE TestYelp IF EXISTS;
CREATE DATAVERSE TestYelp;
USE TestYelp;
@@ -27,7 +25,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp
index bdc0089..1afdbf6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp
index 2a8e0ba..37aa59a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp
index 6eaeb8b..b617001 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp
index ff8aaf2..3a82e48 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp
index cf4dbe7..1014a6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp
index e2bc739..a218f25 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp
index 4037a3b..1b44f26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string) EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times CT, CT.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp
index 0435f33..2cd7334 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp
@@ -27,7 +27,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string) EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_times CT, CT.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp
index 2855490..c57acae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp
@@ -27,7 +27,8 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string)
+EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp
index fb0bc10..eea4fc3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp
@@ -27,7 +27,8 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string)
+EXCLUDE UNKNOWN KEY;
SELECT COUNT(*)
FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp
index 37ef7a7..265017c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp
@@ -27,7 +27,8 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string);
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string)
+EXCLUDE UNKNOWN KEY;
SELECT C.business_id
FROM YelpCheckin C, C.checkin_times D
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
index 16cb966..7eb8623 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
@@ -55,9 +55,9 @@
create index twmSndLocIx on TweetMessages (`sender-location`) type rtree;
-create index msgCountAIx on TweetMessages (countA) type btree;
+create index msgCountAIx on TweetMessages (countA) type btree exclude unknown key;
-create index msgCountBIx on TweetMessages (countB) type btree;
+create index msgCountBIx on TweetMessages (countB) type btree exclude unknown key;
create index msgTextIx on TweetMessages (`message-text`) type keyword;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.sqlpp
index 9da4825..4c937e3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_03-index-only.sqlpp
@@ -55,9 +55,9 @@
create index twmSndLocIx on TweetMessages (`sender-location`) type rtree;
-create index msgCountAIx on TweetMessages (countA) type btree;
+create index msgCountAIx on TweetMessages (countA) type btree exclude unknown key;
-create index msgCountBIx on TweetMessages (countB) type btree;
+create index msgCountBIx on TweetMessages (countB) type btree exclude unknown key;
create index msgTextIx on TweetMessages (`message-text`) type keyword;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/btree-index-composite-key-04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/btree-index-composite-key-04.sqlpp
index e9afad8..917d2fd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/btree-index-composite-key-04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/btree-index-composite-key-04.sqlpp
@@ -38,7 +38,7 @@
create dataset employee(Emp) primary key id;
-create index idx_employee_f_l_name on employee (fname,lname) type btree;
+create index idx_employee_f_l_name on employee (fname,lname) type btree exclude unknown key;
select id, fname, lname
from employee
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-01.sqlpp
new file mode 100644
index 0000000..180cd62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-01.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds1(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE INDEX idx1 ON ds1(x: int);
+CREATE VIEW view1(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds1;
+
+// test that idx1 on x is not used since idx1 does not have the CAST modifier and the field x has cast
+SELECT id, x, y FROM view1 WHERE x <= 1 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-02.sqlpp
new file mode 100644
index 0000000..8de9cfc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-02.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds2(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE INDEX idx2 ON ds2(x: int) CAST (DEFAULT NULL);
+CREATE VIEW view2(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds2;
+
+// test that idx2 on x is used since idx2 has the CAST modifier and the field x has cast
+SELECT id, x, y FROM view2 WHERE x <= 1 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-03.sqlpp
new file mode 100644
index 0000000..6001e60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-03.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds3(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE INDEX idx3 ON ds3(x: int) CAST (DEFAULT NULL);
+CREATE VIEW view3(id int, x string, y int) DEFAULT NULL AS SELECT id, x, y FROM ds3;
+
+// test that idx3 on x is not used since idx3 has the CAST modifier on int and the field x has cast as string
+SELECT id, x, y FROM view3 WHERE x <= "1" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-04.sqlpp
new file mode 100644
index 0000000..db3a728
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-04.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds3(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE INDEX idx3 ON ds3(x: int) CAST (DEFAULT NULL);
+CREATE VIEW view3(id int, x string, y int) DEFAULT NULL AS SELECT id, x, y FROM ds3;
+
+// test that idx3 on x is not used since idx3 has the CAST modifier and the field x does not have cast
+SELECT id, x, y FROM ds3 WHERE x <= 1 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-05.sqlpp
new file mode 100644
index 0000000..505d704
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-05.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds2(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds4(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx2 ON ds2(x: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view2(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds2;
+CREATE VIEW view4(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds4;
+
+
+USE test;
+// test that idx2 on view2(x) is used. both v4.x and v2.x are int and both v2.x and idx2 use int-default-null.
+SELECT v4.x AS v4x, v2.x AS v2x
+FROM view4 AS v4, view2 AS v2
+WHERE v4.x /*+ indexnl */ = v2.x ORDER BY v4x, v2x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-06.sqlpp
new file mode 100644
index 0000000..72fe838
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-06.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds3(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds4(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx3 ON ds3(x: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view3(id int, x string, y int) DEFAULT NULL AS SELECT id, x, y FROM ds3;
+CREATE VIEW view4(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds4;
+
+USE test;
+// test that idx3 on view3(x) is not used because v3.x uses string-default-null and idx3 uses int-default-null
+SELECT v4.x AS v4x, v3.x AS v3x
+FROM view4 AS v4, view3 AS v3
+WHERE v4.x /*+ indexnl */ = v3.x ORDER BY v4x, v3x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-07.sqlpp
new file mode 100644
index 0000000..e5f286a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-07.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds1(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds4(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds1(x: int);
+
+CREATE VIEW view1(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds1;
+CREATE VIEW view4(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds4;
+
+USE test;
+// test that idx1 is not used because idx1 does not have CAST modifier (idx1 is just int and v1.x uses int-default-null).
+SELECT v4.x AS v4x, v1.x AS v1x
+FROM view4 AS v4, view1 AS v1
+WHERE v4.x /*+ indexnl */ = v1.x ORDER BY v4x, v1x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-08.sqlpp
new file mode 100644
index 0000000..e5f286a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-08.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds1(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds4(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds1(x: int);
+
+CREATE VIEW view1(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds1;
+CREATE VIEW view4(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds4;
+
+USE test;
+// test that idx1 is not used because idx1 does not have CAST modifier (idx1 is just int and v1.x uses int-default-null).
+SELECT v4.x AS v4x, v1.x AS v1x
+FROM view4 AS v4, view1 AS v1
+WHERE v4.x /*+ indexnl */ = v1.x ORDER BY v4x, v1x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-09.sqlpp
new file mode 100644
index 0000000..b8ee866
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-09.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds2(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds4(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx2 ON ds2(x: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view4(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds4;
+
+USE test;
+// test that idx2 is not used because idx2 uses int-default-null and ds2.x is just int.
+SELECT v4.x AS v4x, ds2.x AS ds2x
+FROM view4 AS v4, ds2 AS ds2
+WHERE v4.x /*+ indexnl */ = ds2.x ORDER BY v4x, ds2x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-10.sqlpp
new file mode 100644
index 0000000..7bf7df5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-10.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds5(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx5_dt_fmt ON ds5(f_dt_fmt: datetime) CAST (DEFAULT NULL datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY');
+
+CREATE VIEW view5_dt_fmt(id int, f_dt_fmt datetime) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_dt_fmt FROM ds5;
+
+// testing that idx5_dt_fmt is selected
+SELECT VALUE v
+FROM view5_dt_fmt AS v
+WHERE v.f_dt_fmt < datetime('2020-12-20T00:00:00')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-11.sqlpp
new file mode 100644
index 0000000..fb81f5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-11.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds5(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx5_d_fmt ON ds5(f_d_fmt: date) CAST (DEFAULT NULL date 'MM/DD/YYYY');
+
+CREATE VIEW view5_d_fmt(id int, f_d_fmt date) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_d_fmt FROM ds5;
+
+// testing that idx5_d_fmt is selected
+SELECT VALUE v
+FROM view5_d_fmt AS v
+WHERE v.f_d_fmt < date('2020-12-20')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-12.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-12.sqlpp
new file mode 100644
index 0000000..2418082
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-12.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds5(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx5_t_fmt ON ds5(f_t_fmt: time) CAST (DEFAULT NULL time 'hh:mm:ss.nnna');
+
+CREATE VIEW view5_t_fmt(id int, f_t_fmt time) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_t_fmt FROM ds5;
+
+// testing that idx5_t_fmt is selected
+SELECT VALUE v
+FROM view5_t_fmt AS v
+WHERE v.f_t_fmt < time('18:13:03')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-13.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-13.sqlpp
new file mode 100644
index 0000000..5d72810
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-13.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds6(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx6_dt ON ds6(f_dt: datetime) CAST (DEFAULT NULL);
+
+CREATE VIEW view6_dt(id int, f_dt datetime) DEFAULT NULL AS SELECT id, f_dt FROM ds6;
+
+// testing that idx6_dt is selected
+SELECT VALUE v
+FROM view6_dt AS v
+WHERE v.f_dt < datetime('2020-12-20T00:00:00')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-14.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-14.sqlpp
new file mode 100644
index 0000000..fd95610
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-14.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds6(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx6_d ON ds6(f_d: date) CAST (DEFAULT NULL);
+
+CREATE VIEW view6_d(id int, f_d date) DEFAULT NULL AS SELECT id, f_d FROM ds6;
+
+// testing that idx6_d is selected
+SELECT VALUE v
+FROM view6_d AS v
+WHERE v.f_d < date('2020-12-20')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-15.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-15.sqlpp
new file mode 100644
index 0000000..e60ac07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-15.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds6(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx6_t ON ds6(f_t: time) CAST (DEFAULT NULL);
+
+CREATE VIEW view6_t(id int, f_t time) DEFAULT NULL AS SELECT id, f_t FROM ds6;
+
+// testing that idx6_t is selected
+SELECT VALUE v
+FROM view6_t AS v
+WHERE v.f_t < time('18:13:03')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-16.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-16.sqlpp
new file mode 100644
index 0000000..815170a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-16.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds5(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx5_dt_fmt ON ds5(f_dt_fmt: datetime) CAST (DEFAULT NULL datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY');
+
+CREATE VIEW view5_dt_fmt(id int, f_dt_fmt datetime) DEFAULT NULL
+datetime 'MM-DD-YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_dt_fmt FROM ds5;
+
+// testing that the index idx5_dt_fmt is not selected because its datetime format is not the same as view view5_dt_fmt
+SELECT VALUE v
+FROM view5_dt_fmt AS v
+WHERE v.f_dt_fmt < datetime('2020-12-20T00:00:00')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-17.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-17.sqlpp
new file mode 100644
index 0000000..691a558
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-17.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds6(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx6_d_fmt ON ds6(f_d: date) CAST (DEFAULT NULL date 'MM/DD/YYYY');
+
+CREATE VIEW view6_d(id int, f_d date) DEFAULT NULL AS SELECT id, f_d FROM ds6;
+
+// testing that idx6_d_fmt is not selected because the date function in index is with format and the view without format
+SELECT VALUE v
+FROM view6_d AS v
+WHERE v.f_d < date('2020-12-20')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-18.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-18.sqlpp
new file mode 100644
index 0000000..30c4e72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-18.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds5(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx5_t ON ds5(f_t_fmt: time) CAST (DEFAULT NULL);
+
+CREATE VIEW view5_t_fmt(id int, f_t_fmt time) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_t_fmt FROM ds5;
+
+// testing that idx5_t is not selected because the time function in index is without format and the view with format
+SELECT VALUE v
+FROM view5_t_fmt AS v
+WHERE v.f_t_fmt < time('18:13:03')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-19.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-19.sqlpp
new file mode 100644
index 0000000..03c649a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-19.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds5(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx5_dt_fmt ON ds5(f_dt_fmt: datetime) CAST (DEFAULT NULL datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY');
+
+CREATE VIEW view5_dt_fmt(id int, f_dt_fmt datetime) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_dt_fmt FROM ds5;
+
+CREATE DATASET ds6(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+
+CREATE INDEX idx6_dt ON ds6(f_dt: datetime) CAST (DEFAULT NULL);
+
+CREATE VIEW view6_dt(id int, f_dt datetime) DEFAULT NULL AS SELECT id, f_dt FROM ds6;
+
+USE test;
+// testing that idx5_dt_fmt is used
+SELECT v6.f_dt AS v6f_dt, v5.f_dt_fmt AS v5f_dt_fmt
+FROM view6_dt AS v6, view5_dt_fmt AS v5
+WHERE v6.f_dt /*+ indexnl */ = v5.f_dt_fmt ORDER BY v6f_dt, v5f_dt_fmt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-20.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-20.sqlpp
new file mode 100644
index 0000000..4fc08dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-20.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int, s_f1: string, s_f2: string?, i_f: int? };
+CREATE DATASET ds7(t1) PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds7(s_f2);
+CREATE INDEX idx2 ON ds7(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds7(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view7_1(id int, s_f2 string, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+CREATE VIEW view7_2(id int, s_f2 int, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+
+USE test;
+// index idx2 should be used
+SELECT id, s_f2 FROM view7_1 WHERE s_f2 < "4" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-21.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-21.sqlpp
new file mode 100644
index 0000000..eb2994c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-21.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int, s_f1: string, s_f2: string?, i_f: int? };
+CREATE DATASET ds7(t1) PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds7(s_f2);
+CREATE INDEX idx2 ON ds7(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds7(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view7_1(id int, s_f2 string, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+CREATE VIEW view7_2(id int, s_f2 int, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+
+USE test;
+// no index is used (and no result is returned because of incompatible types. s_f2 string vs. 4 int)
+SELECT id, s_f2 FROM view7_1 WHERE s_f2 < 4 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-22.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-22.sqlpp
new file mode 100644
index 0000000..dc1875a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-22.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int, s_f1: string, s_f2: string?, i_f: int? };
+CREATE DATASET ds7(t1) PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds7(s_f2);
+CREATE INDEX idx2 ON ds7(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds7(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view7_1(id int, s_f2 string, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+CREATE VIEW view7_2(id int, s_f2 int, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+
+USE test;
+// no index is used (and no result is returned because of incompatible types. s_f2 int vs. "4" string)
+SELECT id, s_f2 FROM view7_2 WHERE s_f2 < "4" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-23.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-23.sqlpp
new file mode 100644
index 0000000..9c8006f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-23.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int, s_f1: string, s_f2: string?, i_f: int? };
+CREATE DATASET ds7(t1) PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds7(s_f2);
+CREATE INDEX idx2 ON ds7(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds7(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view7_1(id int, s_f2 string, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+CREATE VIEW view7_2(id int, s_f2 int, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+
+USE test;
+// index idx3 should be used
+SELECT id, s_f2 FROM view7_2 WHERE s_f2 < 4 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-24.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-24.sqlpp
new file mode 100644
index 0000000..8df277a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/cast-default-null/cast-default-null-24.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+// repeat_exec: %with% = ["", "WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" ]
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int, s_f1: string, s_f2: string?, i_f: int? };
+CREATE DATASET ds7(t1) PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds7(s_f2);
+CREATE INDEX idx2 ON ds7(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds7(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view7_1(id int, s_f2 string, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+CREATE VIEW view7_2(id int, s_f2 int, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+
+USE test;
+// index idx1 should be used
+set `compiler.indexonly` "false"; // there is an issue with using index only plan where nulls/missings are included
+SELECT id, s_f2 FROM ds7 WHERE s_f2 < "4" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/function-on-pk/function-on-pk-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/function-on-pk/function-on-pk-01.sqlpp
new file mode 100644
index 0000000..0f4c817
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/function-on-pk/function-on-pk-01.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds1(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id;
+
+// test that primary index lookup is not performed and a dataset scan is performed
+SELECT a, b FROM ds1 WHERE id + 1 = 3;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/function-on-pk/function-on-pk-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/function-on-pk/function-on-pk-02.sqlpp
new file mode 100644
index 0000000..ce195ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-selection/function-on-pk/function-on-pk-02.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE DATASET ds1(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id;
+
+// test that primary index lookup is not performed and a dataset scan is performed
+SELECT a, b FROM ds1 WHERE 3 = id + 1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01.sqlpp
index af1a099..d181857 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01.sqlpp
@@ -51,8 +51,8 @@
create dataset MyData(MyRecord) primary key id;
-create index btree_index_docid on MyData(docid) type btree;
-create index btree_index_val1 on MyData(val1) type btree;
+create index btree_index_docid on MyData(docid) type btree exclude unknown key;
+create index btree_index_val1 on MyData(val1) type btree exclude unknown key;
create index rtree_index_point on MyData(point) type rtree;
create index rtree_index_rec on MyData(rec) type rtree;
create index ngram_index_title on MyData(title) type ngram(3);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
index e5c4e0c..a34ce04 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
@@ -51,8 +51,8 @@
create dataset MyData(MyRecord) primary key id;
-create index btree_index_docid on MyData(docid) type btree;
-create index btree_index_val1 on MyData(val1) type btree;
+create index btree_index_docid on MyData(docid) type btree exclude unknown key;
+create index btree_index_val1 on MyData(val1) type btree exclude unknown key;
create index rtree_index_point on MyData(point) type rtree;
create index rtree_index_rec on MyData(rec) type rtree;
create index ngram_index_title on MyData(title) type ngram(3);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-10.sqlpp
index 3259a28..c7ee31c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-10.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-10.sqlpp
@@ -87,7 +87,7 @@
create dataset ds_tweet(typeTweet) if not exists primary key id;
-create index create_at_status_count_idx on ds_tweet(user.create_at, user.status_count);
+create index create_at_status_count_idx on ds_tweet(user.create_at, user.status_count) exclude unknown key;
select value count(first.create_at) from (
select t.user.create_at, t.user.status_count, t.id from ds_tweet t
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query1.sqlpp
new file mode 100644
index 0000000..6953d43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query1.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE GenericType AS { _id: bigint, c : bigint };
+CREATE DATASET IndexDatasetA (GenericType)
+PRIMARY KEY _id;
+CREATE DATASET IndexDatasetB (GenericType)
+PRIMARY KEY _id;
+CREATE DATASET ProbeDataset (GenericType)
+PRIMARY KEY _id;
+
+CREATE INDEX indexA
+ON IndexDatasetA (k : int);
+CREATE INDEX indexB
+ON IndexDatasetB (k : int);
+
+-- Query 1, ternary join w/ primary key on probe.
+FROM ProbeDataset P,
+ IndexDatasetA A,
+ IndexDatasetB B
+WHERE P._id /* +indexnl */ = A.k AND
+ P._id /* +indexnl */ = B.k
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query2.sqlpp
new file mode 100644
index 0000000..344fadd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query2.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE GenericType AS { _id: bigint, c : bigint };
+CREATE DATASET IndexDatasetA (GenericType)
+PRIMARY KEY _id;
+CREATE DATASET IndexDatasetB (GenericType)
+PRIMARY KEY _id;
+CREATE DATASET ProbeDataset (GenericType)
+PRIMARY KEY _id;
+
+CREATE INDEX indexA
+ON IndexDatasetA (k : int);
+CREATE INDEX indexB
+ON IndexDatasetB (k : int);
+
+-- Query 2, ternary join w/ closed field on probe.
+FROM ProbeDataset P,
+ IndexDatasetA A,
+ IndexDatasetB B
+WHERE P.c /* +indexnl */ = A.k AND
+ P.c /* +indexnl */ = B.k
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query3.sqlpp
new file mode 100644
index 0000000..ce0d563
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-ternary-inlj/query3.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE GenericType AS { _id: bigint, c : bigint };
+CREATE DATASET IndexDatasetA (GenericType)
+PRIMARY KEY _id;
+CREATE DATASET IndexDatasetB (GenericType)
+PRIMARY KEY _id;
+CREATE DATASET IndexDatasetC (GenericType)
+PRIMARY KEY _id;
+CREATE DATASET ProbeDataset (GenericType)
+PRIMARY KEY _id;
+
+CREATE INDEX indexA
+ON IndexDatasetA (k : int);
+CREATE INDEX indexB
+ON IndexDatasetB (k : int);
+CREATE INDEX indexC
+ON IndexDatasetC (k : int);
+
+-- Query 3, quad-nary join w/ closed field on probe.
+FROM ProbeDataset P,
+ IndexDatasetA A,
+ IndexDatasetB B,
+ IndexDatasetC C
+WHERE P.c /* +indexnl */ = A.k AND
+ P.c /* +indexnl */ = B.k AND
+ P.c /* +indexnl */ = C.k
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/query-ASTERIXDB-2986.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/query-ASTERIXDB-2986.sqlpp
new file mode 100644
index 0000000..9507c4b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/joins/query-ASTERIXDB-2986.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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+use test1;
+
+create type DocumentType as open {
+};
+
+create type MetaType as open {
+ id:string
+};
+
+create dataset customer(DocumentType) with meta(MetaType) primary key meta().id;
+create dataset orders(DocumentType) with meta(MetaType) primary key meta().id;
+create dataset lineitem(DocumentType) with meta(MetaType) primary key meta().id;
+
+select count(*)
+from orders o, lineitem l, customer c
+where tobigint(l.l_orderkey) = o.o_orderkey
+and o.o_custkey + l.l_orderkey = c.c_custkey + l.l_orderkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/query-ASTERIXDB-2988.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/query-ASTERIXDB-2988.sqlpp
new file mode 100644
index 0000000..2add75a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/query-ASTERIXDB-2988.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * No index join because the probe branch does not have a primary key
+ */
+
+DROP dataverse test if exists;
+CREATE dataverse test;
+USE test;
+
+CREATE DATASET ds1(x int, y int) PRIMARY KEY x;
+
+CREATE INDEX idx_y ON ds1(y) TYPE BTREE;
+
+SELECT r1, ds1.x, ds1.y
+FROM range(0, 4) r1 LEFT OUTER JOIN ds1 ON r1 /* +indexnl */ = ds1.y
+ORDER BY r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp
index f42caf5..792dde1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Res : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp
index 33afe2a..438ea9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Res : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp
index 16ac025..b10915d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : There's index-join hint and the probe type is known (string), non-enforced nested composite index.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Index join because there's a hint and the probe type is known (string), non-enforced nested composite
* Expected Res : Success
*/
drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp
index e8d18cf..9dce6972 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : There's index-join join and the probe type is known (bigint), non-enforced nested composite index.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Index join because there's a hint and the probe type is known (bigint), non-enforced nested composite
* Expected Res : Success
*/
drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
index df0510f..6f97c35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/01.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
index b9585f9..0fc5b4e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/02.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp
index 067d6b9..45f87fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/03.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
index 3ca76d7..534df4c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/04.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
index 1aec248..ae384bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/05.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
index 352e390..d7aa320 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/06.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
index 640f3c9..bc7b0a9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/07.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
index cba8aa5..094c8b3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/08.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
index fd8f8bc..37a6e97 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/09.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
index e5a189a..6cd28df4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/10.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
index 080389a..9024f4e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/11.sqlpp
@@ -17,9 +17,7 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp
index 7b0a51a..546a041 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index/non-enforced-composite-key/12.sqlpp
@@ -17,9 +17,8 @@
* under the License.
*/
/*
- * Description : Testing that creating a nested composite key open index is successful.
- * : Currently this index cannot be used for this query because
- * : the index will not store composite keys where some fields are NULL/MISSING.
+ * Description : Testing that creating a nested composite key open index is successful and being used
+ * when multiple indexes match.
* Expected Result : Success
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.sqlpp
index 4b44ca8..b644b15 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.sqlpp
@@ -22,8 +22,6 @@
* : fields of a composite index.
* : Define the BTree index on a composite key (c_x,c_y,c_z)
* : predicate => WHERE c_x = ... and c_z = ...
-* : Currently this index cannot be used for this query because
-* : the index will not store composite keys where some fields are NULL/MISSING.
* Expected Res : Success
* Date : 10 Oct 2019
*/
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-nulls-first-last.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-nulls-first-last.sqlpp
new file mode 100644
index 0000000..db4b75b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-nulls-first-last.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type TestType as {
+ c: bigint
+};
+
+create dataset data(TestType) primary key c;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS LAST, d.b DESC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_dynamic_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_dynamic_partitioning.sqlpp
new file mode 100644
index 0000000..0cbadc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_dynamic_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE spatial_intersect(ps.geom, ls.geom);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_static_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_static_partitioning.sqlpp
new file mode 100644
index 0000000..4dbf727
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_static_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_distance_static_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_distance_static_partitioning.sqlpp
new file mode 100644
index 0000000..09da36d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_distance_static_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: Geometry
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: Geometry
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ st_distance(ps.geom, ls.geom) < 1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_intersects_static_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_intersects_static_partitioning.sqlpp
new file mode 100644
index 0000000..b93e4ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_intersects_static_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: Geometry
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: Geometry
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ st_intersects(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-3006.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-3006.sqlpp
new file mode 100644
index 0000000..6bf64c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-3006.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.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-3006
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset ds1(id int not unknown) open type primary key id;
+create dataset ds2(id int not unknown) open type primary key id;
+
+select value ds1
+from ds1
+where exists(
+ select 1
+ from ds1.a[*].b b
+ join ds2 on ds2.x[0].y = b
+)
+order by ds1.id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
index da74507..0f996d1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
@@ -172,8 +172,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$230, $$232][$$224, $$225] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$230, $$232] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$233, $$235][$$224, $$225] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$233, $$235] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan
new file mode 100644
index 0000000..faac4b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan
@@ -0,0 +1,19 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan
new file mode 100644
index 0000000..b482afb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan
new file mode 100644
index 0000000..d865914
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query1.plan
new file mode 100644
index 0000000..d53724e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$52(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query2.plan
new file mode 100644
index 0000000..ab9c768
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query2.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$58(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query3.plan
new file mode 100644
index 0000000..2c6241a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query3.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query4.plan
new file mode 100644
index 0000000..d25efc7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query4.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query5.plan
new file mode 100644
index 0000000..df8e08b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query5.plan
@@ -0,0 +1,40 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$76(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query6.plan
new file mode 100644
index 0000000..492b219
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query6.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$93(ASC), $$94(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query7.plan
new file mode 100644
index 0000000..63544ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query7.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$111(ASC), $$112(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
new file mode 100644
index 0000000..ea7ceef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query8.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
new file mode 100644
index 0000000..b47e122
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/atomic-and-array-queries/query9.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$104(ASC), $$105(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.Dataset2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query1.plan
new file mode 100644
index 0000000..7306c4d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query1.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC), $$53(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query2.plan
new file mode 100644
index 0000000..cb70d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query2.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$69(ASC), $$61(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query3.plan
new file mode 100644
index 0000000..b81bf80
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query3.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$73(ASC), $$64(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query4.plan
new file mode 100644
index 0000000..d7acd37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query4.plan
@@ -0,0 +1,65 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinB.YelpCheckinB) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$80(ASC), $$81(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinB.IdxYelpCheckinBDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinA.YelpCheckinA) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$78(ASC), $$68(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinA.IdxYelpCheckinADates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query5.plan
new file mode 100644
index 0000000..e06e50b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-1/query5.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$71(ASC), $$63(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query1.plan
new file mode 100644
index 0000000..641e620
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query1.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$63(ASC), $$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query2.plan
new file mode 100644
index 0000000..109cc05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query2.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$73(ASC), $$64(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query3.plan
new file mode 100644
index 0000000..d895e99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-2/query3.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query1.plan
new file mode 100644
index 0000000..09bcc02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query1.plan
@@ -0,0 +1,40 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$71(ASC), $$62(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query2.plan
new file mode 100644
index 0000000..7217e9f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-3/query2.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$73(ASC), $$64(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query1.plan
new file mode 100644
index 0000000..22f9aa6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query1.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$84(ASC), $$73(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query2.plan
new file mode 100644
index 0000000..df73479
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query2.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$88(ASC), $$76(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query3.plan
new file mode 100644
index 0000000..6c55934
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query3.plan
@@ -0,0 +1,45 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$88(ASC), $$77(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query4.plan
new file mode 100644
index 0000000..552ba9b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-quantified-queries/use-case-4/query4.plan
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinB.YelpCheckinB) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$134(ASC), $$135(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinB.IdxYelpCheckinDatesB) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinA.YelpCheckinA) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$132(ASC), $$116(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckinA.IdxYelpCheckinDatesA) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
index e75ce49..fb3c603 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
@@ -13,17 +13,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$49(ASC), $$42(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC), $$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
index 2c83436..70118a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
@@ -12,18 +12,21 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$49(ASC), $$42(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC), $$50(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
index 56db4c9..bb23941 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
@@ -18,15 +18,17 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$75(ASC), $$68(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$75(ASC), $$76(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query4.plan
new file mode 100644
index 0000000..f602149
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query4.plan
@@ -0,0 +1,37 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$70(ASC), $$71(DESC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$70] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- LEFT_OUTER_UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$79(ASC), $$70(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
index 99cd582..2b052dc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
@@ -13,17 +13,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
index 6864a95..1cad651 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
@@ -12,18 +12,21 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
index b4ee1b0..36d1e39 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
@@ -18,15 +18,17 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$77(ASC), $$69(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC), $$78(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query4.plan
new file mode 100644
index 0000000..44fd353
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query4.plan
@@ -0,0 +1,37 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$71(ASC), $$72(DESC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$71] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- LEFT_OUTER_UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$81(ASC), $$71(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
index 54e1f18..bf67e63 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
@@ -13,17 +13,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
index 0bc4272..443990e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
@@ -13,17 +13,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
index 6864a95..1cad651 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
@@ -12,18 +12,21 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$52(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
index f251b77..36d1e39 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
@@ -18,15 +18,17 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$77(ASC), $$70(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC), $$78(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query5.plan
new file mode 100644
index 0000000..0497e66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query5.plan
@@ -0,0 +1,37 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC), $$73(DESC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$72] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- LEFT_OUTER_UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$81(ASC), $$72(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
index 778f88d..e09340f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
@@ -17,17 +17,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC), $$53(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC), $$62(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
index 8db17af..1569695 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
@@ -18,17 +18,20 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$64(ASC), $$55(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC), $$65(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
index d28ee6d..c4d7c08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
@@ -16,18 +16,21 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$61(ASC), $$53(ASC)] |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC), $$62(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
index 22b9b1c..34a1b94 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
@@ -22,15 +22,17 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- PRE_SORTED_DISTINCT_BY |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$87(ASC), $$79(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$87(ASC), $$88(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query5.plan
new file mode 100644
index 0000000..aac007e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query5.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$81(ASC), $$82(DESC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- LEFT_OUTER_UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- LEFT_OUTER_UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$91(ASC), $$81(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan
new file mode 100644
index 0000000..62b49d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan
new file mode 100644
index 0000000..62b49d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan
new file mode 100644
index 0000000..24b6a5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan
new file mode 100644
index 0000000..24b6a5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan
new file mode 100644
index 0000000..7992dcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan
@@ -0,0 +1,44 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$81(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan
new file mode 100644
index 0000000..7992dcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan
@@ -0,0 +1,44 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$81(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
new file mode 100644
index 0000000..7999e77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query7.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$52(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query8.plan
new file mode 100644
index 0000000..cc99bbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query8.plan
@@ -0,0 +1,40 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$85(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query9.plan
new file mode 100644
index 0000000..c9574d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query9.plan
@@ -0,0 +1,40 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$88(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query4.plan
new file mode 100644
index 0000000..1ed6cd6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query4.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$57(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query5.plan
new file mode 100644
index 0000000..9120584
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query5.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
index b9e4c18..10ad036 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
@@ -10,24 +10,22 @@
-- SUBPLAN |PARTITIONED|
{
-- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- UNNEST |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$63(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query4.plan
new file mode 100644
index 0000000..2d561f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query4.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query5.plan
new file mode 100644
index 0000000..2d561f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query5.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
index f8aad44..2fb6ed5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
@@ -8,25 +8,23 @@
-- SUBPLAN |PARTITIONED|
{
-- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- ASSIGN |LOCAL|
- -- UNNEST |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query4.plan
new file mode 100644
index 0000000..7e503bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query4.plan
@@ -0,0 +1,21 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
index bfdc30a..8de78d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
@@ -8,32 +8,30 @@
-- SUBPLAN |PARTITIONED|
{
-- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- SUBPLAN |LOCAL|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- UNNEST |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ASSIGN |LOCAL|
- -- UNNEST |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query4.plan
new file mode 100644
index 0000000..10b415f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query4.plan
@@ -0,0 +1,37 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$50(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query5.plan
new file mode 100644
index 0000000..f1c1609
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query5.plan
@@ -0,0 +1,36 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$52(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query6.plan
new file mode 100644
index 0000000..3f101b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query6.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query7.plan
new file mode 100644
index 0000000..0f3a84c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query7.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$58(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query3.plan
new file mode 100644
index 0000000..4af61e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query3.plan
@@ -0,0 +1,19 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan
new file mode 100644
index 0000000..0d24b2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$34(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan
new file mode 100644
index 0000000..3585861
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan
new file mode 100644
index 0000000..679b2a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan
new file mode 100644
index 0000000..3f26cee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan
new file mode 100644
index 0000000..f1e0798
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$34(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1c) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan
new file mode 100644
index 0000000..a31846b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2c) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan
new file mode 100644
index 0000000..4ee8831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3c) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan
new file mode 100644
index 0000000..c512a31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$48(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4c) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan
index ce1677d..f560061 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/btree-index-composite-key-04.plan
@@ -3,10 +3,29 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.employee) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.employee) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.idx_employee_f_l_name) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.employee.idx_employee_f_l_name) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
new file mode 100644
index 0000000..0dcfc44
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-01.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$66(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
new file mode 100644
index 0000000..7997aad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-02.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$66(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.idx2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
new file mode 100644
index 0000000..d1264cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-03.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$66(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-04.plan
new file mode 100644
index 0000000..3582cf3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-04.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
new file mode 100644
index 0000000..6285f162
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-05.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$122(ASC), $$123(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$139(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.idx2) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds4) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-06.plan
new file mode 100644
index 0000000..510ccca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-06.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$122(ASC), $$123(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$123] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds4) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-07.plan
new file mode 100644
index 0000000..1577585
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-07.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$122(ASC), $$123(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$123] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds4) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-08.plan
new file mode 100644
index 0000000..1577585
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-08.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$122(ASC), $$123(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$122(ASC), $$123(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$122][$$123] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$122] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds4) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-09.plan
new file mode 100644
index 0000000..d60ee1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-09.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$84(ASC), $$85(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$84(ASC), $$85(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$84][$$85] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$84] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds4) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$85] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
new file mode 100644
index 0000000..19fe8ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-10.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_dt_fmt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
new file mode 100644
index 0000000..b636106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-11.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_d_fmt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
new file mode 100644
index 0000000..382c39a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-12.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_t_fmt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
new file mode 100644
index 0000000..babda5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-13.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_dt) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
new file mode 100644
index 0000000..d55a9a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-14.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_d) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
new file mode 100644
index 0000000..2799b48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-15.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$59(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds6.idx6_t) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
new file mode 100644
index 0000000..6291577
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-16.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
new file mode 100644
index 0000000..8757f3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-17.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
new file mode 100644
index 0000000..6291577
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-18.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$56(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
new file mode 100644
index 0000000..cb86b13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-19.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$110(ASC), $$111(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$110(ASC), $$111(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.ds5) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$123(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds5.idx5_dt_fmt) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds6) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
new file mode 100644
index 0000000..885b6ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-20.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
new file mode 100644
index 0000000..a560c7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-21.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
new file mode 100644
index 0000000..a560c7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-22.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
new file mode 100644
index 0000000..88537b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-23.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$65(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$70(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-24.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-24.plan
new file mode 100644
index 0000000..85d3144
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/cast-default-null/cast-default-null-24.plan
@@ -0,0 +1,19 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.ds7) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds7.idx1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-01.plan
new file mode 100644
index 0000000..f621777
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-01.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-02.plan
new file mode 100644
index 0000000..f621777
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-selection/function-on-pk/function-on-pk-02.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan
new file mode 100644
index 0000000..671d511
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query1.plan
@@ -0,0 +1,35 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$73(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$71(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.ProbeDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan
new file mode 100644
index 0000000..c0f794e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query2.plan
@@ -0,0 +1,37 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$73(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$71(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.ProbeDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan
new file mode 100644
index 0000000..b94c26d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-ternary-inlj/query3.plan
@@ -0,0 +1,49 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetC.IndexDatasetC) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$91(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetC.indexC) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetB.IndexDatasetB) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$89(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetB.indexB) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetA.IndexDatasetA) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$87(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestDataverse.IndexDatasetA.indexA) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestDataverse.ProbeDataset) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
index 35fdc41..ece21bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
@@ -1,25 +1,53 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- UNION_ALL |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.KVStore) |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.OfficerLocations.OfficerLocations) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.OfficerLocations) |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- RTREE_SEARCH (test.OfficerLocations.o_location) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.KVStore) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPLIT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- RTREE_SEARCH (test.OfficerLocations.o_location) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.KVStore) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
index 35fdc41..aa7008a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
@@ -6,9 +6,9 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- STREAM_SELECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN (test.KVStore) |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/query-ASTERIXDB-2986.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/query-ASTERIXDB-2986.plan
new file mode 100644
index 0000000..4bbdb3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/query-ASTERIXDB-2986.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$77] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$78] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test1.orders) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$77] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test1.lineitem) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test1.customer) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2988.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2988.plan
new file mode 100644
index 0000000..96fbf3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2988.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$r1(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$r1(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$r1][$$32] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$r1] |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
index de7d951..a33f232 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/01.plan
@@ -3,19 +3,20 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$33][$$34] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds2) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
index de7d951..347599b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/02.plan
@@ -3,19 +3,20 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$33][$$34] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$33] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.ds2) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.ds2.idx) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
index 97401bb..4eb7b4f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/06.plan
@@ -6,25 +6,26 @@
-- STABLE_SORT [$$46(ASC), $$47(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$44] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$44] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen2) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen2.TestOpen2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen2.idx_t2_s) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
index 97401bb..6ecffbf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index-join/non-enforced-composite-key-equi-join/07.plan
@@ -6,25 +6,26 @@
-- STABLE_SORT [$$46(ASC), $$47(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$44] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$44] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen2) |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen2.TestOpen2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen2.idx_t2_i64) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.TestOpen1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
index 5454dab..2ac8171 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/01.plan
@@ -3,6 +3,13 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- BTREE_SEARCH (test.testdst.testdst) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.testdst.sec_Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
index 5454dab..0746bd8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/02.plan
@@ -3,6 +3,13 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.testdst) |PARTITIONED|
+ -- BTREE_SEARCH (test.testdst.testdst) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.testdst.sec_Idx) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
index d874441..13f9dcf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/03.plan
@@ -10,6 +10,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_i64) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
index 58cd521..ceb8921 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/04.plan
@@ -12,6 +12,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_s) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
index 58cd521..2cf4812 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/05.plan
@@ -12,6 +12,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_i64) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
index 58cd521..a63d87f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/06.plan
@@ -12,6 +12,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_i64) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
index 58cd521..a63d87f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/07.plan
@@ -12,6 +12,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_i64) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
index 58cd521..ef289a4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/08.plan
@@ -12,6 +12,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_i8) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
index 58cd521..ef289a4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/09.plan
@@ -12,6 +12,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_i8) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
index 58cd521..c3e596b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/10.plan
@@ -12,6 +12,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_d) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
index bbd2dba..7fb98d3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/11.plan
@@ -13,6 +13,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_i8) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
index fde8db4..fcc0784 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/btree-index/non-enforced-composite-key/12.plan
@@ -12,6 +12,24 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- INTERSECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_3) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_4) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
index f1a1616..b266aa8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-03.plan
@@ -5,6 +5,13 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN (test.TestOpen) |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.TestOpen) |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (test.TestOpen.idx_xyz) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-nulls-first-last.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-nulls-first-last.plan
new file mode 100644
index 0000000..fe5e86f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-nulls-first-last.plan
@@ -0,0 +1,13 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$14(ASC), $$17(ASC), $$15(DESC), $$18(DESC) ] |PARTITIONED|
+ -- STABLE_SORT [$$14(ASC), $$17(ASC), $$15(DESC), $$18(DESC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.data) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan
new file mode 100644
index 0000000..ce4f736
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan
@@ -0,0 +1,156 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPATIAL_JOIN [$$60, $$49] [$$61, $$50] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC), $$49(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$60] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ParkSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- NESTED_LOOP |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ParkSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.LakeSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC), $$50(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- RANDOM_PARTITION_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.LakeSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- NESTED_LOOP |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ParkSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.LakeSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- NESTED_LOOP |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ParkSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.LakeSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan
new file mode 100644
index 0000000..ce2f6e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPATIAL_JOIN [$$53, $$49] [$$54, $$50] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$53(ASC), $$49(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ParkSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC), $$50(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$54] |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.LakeSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
new file mode 100644
index 0000000..25546ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPATIAL_JOIN [$$61, $$56] [$$62, $$57] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC), $$56(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ParkSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$57(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62] |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.LakeSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan
new file mode 100644
index 0000000..43d8da6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SPATIAL_JOIN [$$55, $$52] [$$56, $$53] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC), $$52(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$55] |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ParkSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC), $$53(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$56] |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.LakeSet) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-3006.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-3006.plan
new file mode 100644
index 0000000..a5264a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-3006.plan
@@ -0,0 +1,49 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$64(ASC) ] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$78] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$48] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$48][$$61] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$61] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$b][$$51] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$b] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds1) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$51] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (test.ds2) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
new file mode 100644
index 0000000..6095b26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.1.plans.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+-- param compile-only:string=true
+-- param logical-plan:string=true
+-- param plan-format:string=json
+
+select value v from range(1,2) v where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
new file mode 100644
index 0000000..2a4952f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.2.plans.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test additional information returned when client-type=jdbc
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
new file mode 100644
index 0000000..d7217a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.3.plans.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test additional information returned when client-type=jdbc (with explain)
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+explain select v from range(1,2) v where v between ? and ? ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
new file mode 100644
index 0000000..8d2bd74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/compileonly/compileonly.4.plans.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test additional information returned when client-type=jdbc (update statement)
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+drop dataverse test1 if exists;
+
+create dataverse test1;
+
+create dataset test1.t1(c1 int not unknown, c2 int) primary key c1;
+
+insert into test1.t1 [{"c1": 1, "c2": ? }, {"c1": 3, "c2": ? }];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.1.ddl.sqlpp
new file mode 100644
index 0000000..cb28d2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.1.ddl.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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2.test3 if exists;
+create dataverse test2.test3;
+
+use test1;
+
+create function f1() {
+ select value r
+ from range(0, 9) r
+};
+
+use test2.test3;
+
+create function f23() {
+ select value r
+ from range(10, 19) r
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.2.query.sqlpp
new file mode 100644
index 0000000..765f51b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/x-www-form-urlencoded
+// param dataverse:string=test1
+
+array_sum(f1());
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.3.query.sqlpp
new file mode 100644
index 0000000..c6e311c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param dataverse:json="test2/test3"
+
+array_sum(f23());
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.4.query.sqlpp
new file mode 100644
index 0000000..252ccb0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/request-dataverse/request-dataverse.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Warn if specified dataverse name is unknown
+ */
+
+// requesttype=application/json
+// param dataverse:json="testUnknown"
+// param max-warnings:json=10
+
+array_max(test1.f1());
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.1.signature.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.1.signature.sqlpp
new file mode 100644
index 0000000..3620bde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.1.signature.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- param compile-only:string=true
+
+select v v1, v v2 from range(1,2) v where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.2.signature.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.2.signature.sqlpp
new file mode 100644
index 0000000..37eee90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.2.signature.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test returned signature when client-type=jdbc (primitive types)
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+select
+ int8(v) c_int8,
+ int16(v) c_int16,
+ int32(v) c_int32,
+ int64(v) c_int64,
+ float(v) c_float,
+ double(v) c_double,
+ string(v) c_string,
+ boolean(v) c_boolean,
+ datetime_from_unix_time_in_ms(v) c_datetime,
+ date_from_unix_time_in_days(v) c_date,
+ time_from_unix_time_in_ms(v) c_time,
+ duration_from_months(v) c_duration,
+ year_month_duration(duration_from_months(v)) c_year_month_duration,
+ day_time_duration(duration_from_ms(v)) c_day_time_month_duration,
+ null c_null
+from range(1,2) v
+where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.3.signature.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.3.signature.sqlpp
new file mode 100644
index 0000000..07cc798
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.3.signature.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test returned signature when client-type=jdbc (complex types)
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+select
+ v c_number,
+ { "f": v } c_record,
+ [ v, v ] c_array,
+ {{ v, v }} c_multiset
+from range(1,2) v
+where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.4.signature.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.4.signature.sqlpp
new file mode 100644
index 0000000..3e3d170
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.4.signature.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test returned signature when client-type=jdbc (unknown types)
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+select
+ v c_number,
+ case when v = 1 then { "f": v } else [ v, v ] end c_case,
+ missing c_missing
+from range(1,2) v
+where v > ?;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.5.signature.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.5.signature.sqlpp
new file mode 100644
index 0000000..c776f31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/signature/signature.5.signature.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test returned signature when client-type=jdbc (types from a dataset)
+ */
+
+-- param client-type:string=jdbc
+-- param compile-only:string=true
+
+select DataverseName, DatasetName, ViewDetails
+from Metadata.`Dataset`;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp
index 3e2eeda..2b75805 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp
@@ -32,7 +32,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
-// CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id);
-// CREATE INDEX IdxYelpCheckinBusinessIDDates ON YelpCheckin (business_id, UNNEST dates);
-CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
\ No newline at end of file
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessIDDates ON YelpCheckin (business_id, UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp
index ecef50a..5e214f1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp
@@ -35,6 +35,6 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
-// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, (business_id));
-// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates);
\ No newline at end of file
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, (business_id)) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp
index abd1c6c..94d914c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp
@@ -35,8 +35,15 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
-CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
-CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-// CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin ((UNNEST checkin_times SELECT date, time), (business_id));
-// CREATE INDEX IdxYelpCheckinBusinessDatesTimes ON YelpCheckin (business_id, UNNEST checkin_times SELECT date, time);
\ No newline at end of file
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id) EXCLUDE UNKNOWN KEY;
+
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness
+ON YelpCheckin ((UNNEST checkin_times SELECT date, time),
+ (business_id))
+EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessDatesTimes
+ON YelpCheckin (business_id,
+ UNNEST checkin_times SELECT date, time)
+EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp
index 626e797..4b8297f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp
@@ -35,6 +35,13 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
-// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin ((UNNEST checkin_times UNNEST dates), business_id);
-// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times UNNEST dates);
\ No newline at end of file
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+CREATE INDEX IdxYelpCheckinDatesBusiness
+ON YelpCheckin ((UNNEST checkin_times UNNEST dates),
+ business_id)
+EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessDates
+ON YelpCheckin (business_id,
+ UNNEST checkin_times UNNEST dates)
+EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
index 47b4d5b..58f62e5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -40,4 +40,6 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
\ No newline at end of file
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates)
+EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp
index 415054c..1c444c6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -32,4 +32,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
\ No newline at end of file
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp
index 1de5989..89a1142 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -32,4 +32,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id WITH FILTER ON business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
\ No newline at end of file
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp
index aa3a456..b678952 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp
@@ -30,4 +30,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp
index dbf055c..1e44b75 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp
@@ -19,6 +19,6 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
-// CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id);
-// CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id, UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id, UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp
index 0452c4a..5f9cbe3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp
@@ -19,9 +19,9 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
/* It would be more user-friendly to use (UNNEST checkin_times.dates), (business_id), but we are able to infer this
from our grammar alone. We leave this in to also exercise our parser. */
-// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, business_id);
-// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, business_id) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp
index ee911b4..4154d37 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp
index ad50985..0cfbc1f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp
@@ -20,6 +20,13 @@
USE TestYelp;
DROP INDEX YelpCheckin.IdxYelpCheckinDates;
-CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
-// CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin ((UNNEST checkin_times SELECT date, time), (business_id));
-// CREATE INDEX IdxYelpCheckinBusinessDatesTimes ON YelpCheckin (business_id, UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time) EXCLUDE UNKNOWN KEY;
+
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness
+ON YelpCheckin ((UNNEST checkin_times SELECT date, time),
+ (business_id))
+EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessDatesTimes
+ON YelpCheckin (business_id,
+ UNNEST checkin_times SELECT date, time)
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp
index f571dbc..cd79b9b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp
@@ -19,6 +19,13 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
-// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin ((UNNEST checkin_times UNNEST dates), business_id);
-// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+
+CREATE INDEX IdxYelpCheckinDatesBusiness
+ON YelpCheckin ((UNNEST checkin_times UNNEST dates),
+ business_id)
+EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessDates
+ON YelpCheckin (business_id,
+ UNNEST checkin_times UNNEST dates)
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
index 5ef37bc..01be97a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -19,4 +19,6 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates)
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.1.ddl.sqlpp
new file mode 100644
index 0000000..6312915
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.1.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE GenericType AS { _id: bigint };
+CREATE DATASET TestDataset (GenericType)
+PRIMARY KEY _id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.2.update.sqlpp
new file mode 100644
index 0000000..e901b37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 TestDataverse;
+
+INSERT INTO TestDataset [
+ { "_id": 100, "f": 1, "a": {"b": [{"c": {"d": 1}}]} },
+ { "_id": 101, "f": "a", "a": null },
+ { "_id": 102, "f": "a", "a": {"b": null} },
+ { "_id": 103, "f": "a", "a": {"b": [{"c": null}]} },
+ { "_id": 104, "f": "a", "a": {"b": [{"c": {"d": null}}]} }
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.3.ddl.sqlpp
new file mode 100644
index 0000000..8e4091e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.3.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 TestDataverse;
+
+CREATE INDEX TestIdx
+ON TestDataset (
+ f : bigint,
+ UNNEST a.b
+ SELECT c.d : bigint
+) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.4.query.sqlpp
new file mode 100644
index 0000000..a5de0da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+FROM TestDataset D
+UNNEST D.a.b AB
+WHERE D.f = 1 AND AB.c.d = 1
+SELECT D._id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp
index 0d37224..5a6513a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp
index abacfe8..be8f99d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp
index 9594cf9..4c7e924 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp
index 982affc..f5eddd5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp
@@ -20,4 +20,6 @@
USE TestYelp;
DROP INDEX YelpCheckin.IdxYelpCheckinDates;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string ) ;
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string )
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp
index c590ecb..176da7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.1.ddl.sqlpp
new file mode 100644
index 0000000..e3a2e46
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET Dataset2 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE INDEX d1Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX d2Idx
+ON Dataset1 (
+ field1 : bigint,
+ ( UNNEST outer_items
+ UNNEST inner_items
+ SELECT field2 : bigint,
+ field3 : bigint ),
+ field4 : bigint
+) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.10.query.sqlpp
new file mode 100644
index 0000000..1f113ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Existential quantification query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset2 D2,
+ Dataset1 D1
+WHERE D1.field1 /* +indexnl */ = TO_BIGINT(D2.field1) AND
+ ( SOME DOI IN D1.outer_items
+ SATISFIES DOI.field2_notindexed = TO_BIGINT(D2.field2_notindexed) AND
+ ( SOME DII IN DOI.inner_items
+ SATISFIES DII.field2 /* +indexnl */ = TO_BIGINT(D2.field2) AND
+ DII.field3 /* +indexnl */ = TO_BIGINT(D2.field3) AND
+ DII.field3_notindexed = TO_BIGINT(D2.field3_notindexed) ) ) AND
+ D1.field4 /* +indexnl */ = TO_BIGINT(D2.field4)
+SELECT D1.field1, D1.outer_items, D1.field4
+ORDER BY D1.field1, D1.outer_items, D1.field4;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.2.update.sqlpp
new file mode 100644
index 0000000..f66b749
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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 TestDataverse;
+
+INSERT INTO Dataset1 [
+ { "field1": 1, "items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ], "outer_items": [ { "field2_notindexed": 2, "inner_items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ] } ], "field4": 4, "field4_notindexed": 4 },
+ { "field1": 1, "items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ], "outer_items": [ { "field2_notindexed": 2, "inner_items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ] } ], "field4": 4, "field4_notindexed": 4 },
+ { "field1": 1, "items": [ { "field2": 2, "field3": 3, "field3_notindexed": 300 } ], "outer_items": [ { "field2_notindexed": 2, "inner_items": [ { "field2": 2, "field3": 3, "field3_notindexed": 300 } ] } ], "field4": 4, "field4_notindexed": 400 },
+ { "field1": 100, "items": [ { "field2": 200, "field3": 300, "field3_notindexed": 3 } ], "outer_items": [ { "field2_notindexed": 200, "inner_items": [ { "field2": 200, "field3": 300, "field3_notindexed": 3 } ] } ], "field4": 400, "field4_notindexed": 4 },
+ { "field1": -1, "items": [ { "field2": -2, "field3": -3, "field3_notindexed": -3 } ], "outer_items": [ { "field2_notindexed": -2, "inner_items": [ { "field2": -2, "field3": -3, "field3_notindexed": -3 } ] } ], "field4": -4, "field4_notindexed": -4 },
+ { "field1": -1, "items": [ { "field2": -2, "field3": -3, "field3_notindexed": -3 } ], "outer_items": [ { "field2_notindexed": -2, "inner_items": [ { "field2": -2, "field3": -3, "field3_notindexed": -3 } ] } ], "field4": -4, "field4_notindexed": -4 }
+];
+
+INSERT INTO Dataset2 [
+ { "field1": 1, "field2_notindexed": 2, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 },
+ { "field1": 1, "field2_notindexed": 200, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 },
+ { "field1": 100, "field2_notindexed": 2, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 },
+ { "field1": 100, "field2_notindexed": 200, "field2": 200, "field3": 300, "field3_notindexed": 300, "field4": 400, "field4_notindexed": 400 },
+ { "field1": -100, "field2_notindexed": -200, "field2": -200, "field3": -300, "field3_notindexed": -300, "field4": -400, "field4_notindexed": -400 }
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.3.query.sqlpp
new file mode 100644
index 0000000..e860165
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Explicit UNNEST query with all fields of a composite index.
+FROM Dataset1 D
+UNNEST D.items DI
+WHERE D.field1 = 1 AND
+ DI.field2 = 2 AND
+ DI.field3 = 3 AND
+ D.field4 = 4
+SELECT D.field1, DI.field2, DI.field3, DI.field3_notindexed, D.field4, D.field4_notindexed
+ORDER BY D.field1, DI.field2, DI.field3, DI.field3_notindexed, D.field4, D.field4_notindexed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.4.query.sqlpp
new file mode 100644
index 0000000..b19f711
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Explicit UNNEST query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset1 D
+UNNEST D.items DI
+WHERE D.field1 = 1 AND
+ DI.field2 = 2 AND
+ DI.field3 = 3 AND
+ DI.field3_notindexed = 3 AND
+ D.field4 = 4 AND
+ D.field4_notindexed = 4
+SELECT D.field1, DI.field2, DI.field3, DI.field3_notindexed, D.field4, D.field4_notindexed
+ORDER BY D.field1, DI.field2, DI.field3, DI.field3_notindexed, D.field4, D.field4_notindexed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.5.query.sqlpp
new file mode 100644
index 0000000..e6002c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.5.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Existential quantification query with all fields of a composite index.
+FROM Dataset1 D
+WHERE D.field1 = 1 AND
+ ( SOME DI IN D.items
+ SATISFIES DI.field2 = 2 AND
+ DI.field3 = 3 ) AND
+ D.field4 = 4
+SELECT D.field1, D.items, D.field4, D.field4_notindexed
+ORDER BY D.field1, D.items, D.field4, D.field4_notindexed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.6.query.sqlpp
new file mode 100644
index 0000000..12b6f4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.6.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Existential quantification query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset1 D
+WHERE D.field1 = 1 AND
+ ( SOME DOI IN D.outer_items
+ SATISFIES DOI.field2_notindexed = 2 AND
+ ( SOME DII IN DOI.inner_items
+ SATISFIES DII.field2 = 2 AND
+ DII.field3 = 3 AND
+ DII.field3_notindexed = 3 ) ) AND
+ D.field4 = 4
+SELECT D.field1, D.outer_items, D.field4
+ORDER BY D.field1, D.outer_items, D.field4;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.7.query.sqlpp
new file mode 100644
index 0000000..9ab9213
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Explicit UNNEST join query with all fields of a composite index.
+FROM Dataset2 D2
+INNER JOIN (
+ FROM Dataset1 D1
+ UNNEST D1.items D1I
+ SELECT D1.field1, D1I.field2, D1I.field3, D1.field4
+) AS D1IV
+ON D1IV.field1 /* +indexnl */ = D2.field1 AND
+ D1IV.field2 /* +indexnl */ = D2.field2 AND
+ D1IV.field3 /* +indexnl */ = D2.field3 AND
+ D1IV.field4 /* +indexnl */ = D2.field4
+SELECT D2.field1, D2.field2, D2.field3, D2.field4
+ORDER BY D2.field1, D2.field2, D2.field3, D2.field4;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.8.query.sqlpp
new file mode 100644
index 0000000..907817b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.8.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Explicit UNNEST join query with all fields of a composite index, with extra clauses on non-indexed fields.
+FROM Dataset2 D2
+INNER JOIN (
+ FROM Dataset1 D1
+ UNNEST D1.items D1I
+ SELECT D1.field1, D1I.field2, D1I.field3, D1I.field3_notindexed, D1.field4
+) AS D1IV
+ON D1IV.field1 /* +indexnl */ = TO_BIGINT(D2.field1) AND
+ D1IV.field2 /* +indexnl */ = TO_BIGINT(D2.field2) AND
+ D1IV.field3 /* +indexnl */ = TO_BIGINT(D2.field3) AND
+ D1IV.field3_notindexed = D2.field3_notindexed AND
+ D1IV.field4 /* +indexnl */ = TO_BIGINT(D2.field4)
+SELECT D2.field1, D2.field2, D2.field3, D2.field3_notindexed, D2.field4, D2.field4_notindexed
+ORDER BY D2.field1, D2.field2, D2.field3, D2.field3_notindexed, D2.field4, D2.field4_notindexed;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.9.query.sqlpp
new file mode 100644
index 0000000..f84f82d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/composite-index-queries/composite-index-queries.9.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+-- Existential quantification join query with all fields of a composite index.
+FROM Dataset2 D2,
+ Dataset1 D1
+WHERE D1.field1 /* +indexnl */ = TO_BIGINT(D2.field1) AND
+ ( SOME D1I IN D1.items
+ SATISFIES D1I.field2 /* +indexnl */ = TO_BIGINT(D2.field2) AND
+ D1I.field3 /* +indexnl */ = TO_BIGINT(D2.field3) ) AND
+ D1.field4 /* +indexnl */ = TO_BIGINT(D2.field4)
+SELECT D2.field1, D2.field2, D2.field3, D2.field4
+ORDER BY D2.field1, D2.field2, D2.field3, D2.field4;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-include-unknown-key/index-include-unknown-key.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-include-unknown-key/index-include-unknown-key.1.ddl.sqlpp
new file mode 100644
index 0000000..5929898
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-include-unknown-key/index-include-unknown-key.1.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS { _id: bigint };
+CREATE DATASET IndexedDataset (TestType) PRIMARY KEY _id;
+
+// We MUST specify EXCLUDE UNKNOWN KEY for array indexes.
+CREATE INDEX testIndex ON IndexedDataset ( UNNEST items SELECT item: bigint );
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-include-unknown-key/index-include-unknown-key.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-include-unknown-key/index-include-unknown-key.2.ddl.sqlpp
new file mode 100644
index 0000000..192c64b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-include-unknown-key/index-include-unknown-key.2.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS { _id: bigint };
+CREATE DATASET IndexedDataset (TestType) PRIMARY KEY _id;
+
+CREATE INDEX testIndex ON IndexedDataset ( UNNEST items SELECT item: bigint ) INCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp
deleted file mode 100644
index c26bf92..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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: Verify that an error is thrown when trying to create a mixed composite atomic and array index (not supported for now).
- */
-
-DROP DATAVERSE TestYelp IF EXISTS;
-CREATE DATAVERSE TestYelp;
-USE TestYelp;
-
-CREATE TYPE CheckinTimeType AS {
- time: string
-};
-CREATE TYPE CheckinType AS {
- checkin_id: uuid,
- business_id: string,
- checkin_times: [CheckinTimeType]
-};
-
-CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinTimeBusiness ON YelpCheckin ((UNNEST checkin_times SELECT time), (business_id));
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp
index 685485b..0e9f581 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp
@@ -35,5 +35,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp
index 83d5407..ca5e69b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp
@@ -33,5 +33,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates, UNNEST times);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates, UNNEST times) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp
index d610c14..5be89bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp
@@ -31,4 +31,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin ((UNNEST checkin_times SELECT date), UNNEST checkin_times UNNEST times);
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin ((UNNEST checkin_times SELECT date),
+ UNNEST checkin_times UNNEST times)
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp
index baf5bc0..4838400 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp
@@ -35,5 +35,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ENFORCED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ENFORCED EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp
index 2402382..e2d7054 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp
@@ -35,5 +35,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time SELECT date) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp
index d658dbe..0c73ad9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp
@@ -35,5 +35,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time.date);
-
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time.date) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp
index 7498c23..d75e0ce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -34,4 +34,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp
index 81b322c..aa54f8c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -37,5 +37,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
-CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp
index d65a2eb..0470f10 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -37,5 +37,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
-CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp
index 590c9d3..ff693cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -37,5 +37,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
-CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
index 1697fb3..49406b1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
@@ -35,5 +35,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
index 6ff0c88..d468d7a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -38,4 +38,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date, time) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
index c6739ca..193c4fb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -34,4 +34,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id WITH FILTER ON business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp
index ab374bf..4dd2cc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -33,4 +33,4 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp
index ef8d1c2..37ac4ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -33,5 +33,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
-CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp
index 768692b..904dcfe 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -33,5 +33,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
-CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp
index 068482f..0844653 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -33,5 +33,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
-CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
index cc59e2f..d7d01c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
@@ -33,5 +33,5 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
-CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
index da9eb7ea..9d81f0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -33,4 +33,6 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string );
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string )
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..5e91e18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckinA(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinB(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDatesA ON YelpCheckinA (UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesB ON YelpCheckinB (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..6c84dec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,115 @@
+/*
+ * 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 TestYelp;
+
+INSERT INTO YelpCheckinA (
+ [
+ {
+ "business_id": "1",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "2",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "3",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ }
+ ]
+);
+INSERT INTO YelpCheckinB (
+ [
+ {
+ "business_id": "A",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "B",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "C",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "D",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "E",
+ "dates": [ ]
+ }
+ ]
+);
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2019-06-07 17:54:58", "date": "2016-04-26", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.3.query.sqlpp
new file mode 100644
index 0000000..1728e9f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE M.datetime /*+ indexnl */ IN A.dates
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..d420403
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE ( SOME D IN A.dates
+ SATISFIES M.datetime /*+ indexnl */ = D )
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..2f0ba7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE LEN(A.dates) > 0 AND
+ ( EVERY D IN A.dates
+ SATISFIES M.datetime /*+ indexnl */ = D )
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
new file mode 100644
index 0000000..495eb72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A, YelpCheckinB B
+WHERE M.datetime /*+ indexnl */ IN A.dates AND
+ M.datetime /*+ indexnl */ IN B.dates
+SELECT M.marker, A.business_id AS business_id_a, B.business_id AS business_id_b
+ORDER BY M.marker, A.business_id, B.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..89e3a82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE SOME AND EVERY D IN A.dates
+ SATISFIES M.datetime /*+ indexnl */ = D
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..4baf2d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckinA(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDatesA ON YelpCheckinA (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..044737d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,93 @@
+/*
+ * 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 TestYelp;
+
+INSERT INTO YelpCheckinA (
+ [
+ {
+ "business_id": "1",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "2",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-06-07"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "3",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ }
+ ]
+);
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.3.query.sqlpp
new file mode 100644
index 0000000..d233441
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE M.date /*+ indexnl */ IN A.checkin_times.dates
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..4f641ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE SOME AND EVERY D IN A.checkin_times.dates
+ SATISFIES M.date /*+ indexnl */ = D
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..a20eeab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..24efca4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,69 @@
+/*
+ * 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 TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "1",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "2",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-06-07", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "3",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ }
+ ]
+);
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..1bd6236
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE SOME D IN C.checkin_times
+ SATISFIES D.date /*+ indexnl */ = M.date
+SELECT M.marker, C.business_id
+ORDER BY M.marker, C.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..54a0bc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckin C
+WHERE SOME AND EVERY D IN C.checkin_times
+ SATISFIES D.date /*+ indexnl */ = M.date
+SELECT M.marker, C.business_id
+ORDER BY M.marker, C.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..9d0a698
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckinA(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinB(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDatesA ON YelpCheckinA (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesB ON YelpCheckinB (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..375b518
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,137 @@
+/*
+ * 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 TestYelp;
+
+INSERT INTO YelpCheckinA (
+ [
+ {
+ "business_id": "1",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "2",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-06-07", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2019-06-07"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "3",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ }
+ ]
+);
+INSERT INTO YelpCheckinB (
+ [
+ {
+ "business_id": "A",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "B",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-06-07", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2019-06-07"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "C",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ }
+ ]
+);
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.3.query.sqlpp
new file mode 100644
index 0000000..6bcb7a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE SOME CT IN A.checkin_times
+ SATISFIES ( SOME D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date )
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..ffaf276
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE SOME AND EVERY CT IN A.checkin_times
+ SATISFIES ( SOME D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date )
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..a37d923
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A
+WHERE SOME AND EVERY CT IN A.checkin_times
+ SATISFIES ( SOME AND EVERY D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date )
+SELECT M.marker, A.business_id
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
new file mode 100644
index 0000000..bf5e128
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
@@ -0,0 +1,31 @@
+
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M, YelpCheckinA A, YelpCheckinB B
+WHERE ( SOME AND EVERY CT IN A.checkin_times
+ SATISFIES ( SOME AND EVERY D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date ) ) AND
+ ( SOME AND EVERY CT IN B.checkin_times
+ SATISFIES ( SOME AND EVERY D IN CT.dates
+ SATISFIES D /*+ indexnl */ = M.date ) )
+SELECT M.marker, A.business_id AS business_id_a, B.business_id AS business_id_b
+ORDER BY M.marker, A.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.1.ddl.sqlpp
new file mode 100644
index 0000000..db03322
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE GenericType AS { _id: uuid };
+CREATE DATASET IndexedDataset (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET ProbeDataset (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+
+CREATE INDEX TestIdx
+ON IndexedDataset (
+ UNNEST items
+ SELECT val : bigint
+) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.2.update.sqlpp
new file mode 100644
index 0000000..7f8aca6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.2.update.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 TestDataverse;
+
+INSERT INTO IndexedDataset [
+ {"items": [{"val": 1}, {"val": 2}]}
+];
+INSERT INTO ProbeDataset [
+ {"val": 1},
+ {"val": 1},
+ {"val": 2},
+ {"val": 3},
+ {"val": null}
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.3.query.sqlpp
new file mode 100644
index 0000000..b4bf399
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/loj-subquery/loj-subquery.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+USE TestDataverse;
+
+FROM ProbeDataset P
+LEFT OUTER UNNEST P.unrelated_array PUA
+LEFT OUTER JOIN (
+ FROM IndexedDataset I
+ UNNEST I.items II
+ SELECT VALUE II.val
+) AS IIV
+ON IIV /* +indexnl */ = TO_BIGINT(P.val)
+SELECT P.val
+ORDER BY P.val NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp
index 39f18a0..3e7c0c9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp
@@ -22,7 +22,7 @@
* 1) Implicit INNER JOIN query, with additional predicate on inner collection before the unnest.
* 2) Implicit INNER JOIN query, with additional predicate on the outer collection.
* 3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
- * TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ * 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
*/
DROP DATAVERSE TestYelp IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.8.query.sqlpp
new file mode 100644
index 0000000..7bfe568
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.8.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ FROM YelpCheckin C, C.dates CD
+ SELECT VALUE CD
+) AS D ON D /*+ indexnl */ = M.datetime
+SELECT COUNT(*);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp
index a00ffca..4e89d6a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp
@@ -22,7 +22,7 @@
* 1) INNER JOIN query, with additional predicate on inner collection before the unnest.
* 2) INNER JOIN query, with additional predicate on the outer collection.
* 3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
- * TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ * 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
*/
DROP DATAVERSE TestYelp IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp
index 0ab72fe..caaaad0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.8.query.sqlpp
new file mode 100644
index 0000000..0414384
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.8.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ FROM YelpCheckin C, C.checkin_times.dates CD
+ SELECT VALUE CD
+) AS D ON D /*+ indexnl */ = M.datetime
+SELECT COUNT(*);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp
index fed0346..d7c49d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp
@@ -23,7 +23,7 @@
* 2) INNER JOIN query, with additional predicate on inner collection after the unnest.
* 3) INNER JOIN query, with additional predicate on the outer collection.
* 4) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
- * TODO (GLENN) 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ * 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
*/
DROP DATAVERSE TestYelp IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp
index ee911b4..4154d37 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.9.query.sqlpp
new file mode 100644
index 0000000..27eab43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.9.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ FROM YelpCheckin C, C.checkin_times D
+ SELECT VALUE D.date
+) AS CD ON CD /* +indexnl */ = M.date
+SELECT COUNT(*);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp
index 41d36e9..100cbdd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp
@@ -23,7 +23,7 @@
* 2) INNER JOIN query, with additional predicate on inner collection before the first unnest and before the final unnest.
* 3) INNER JOIN query, with additional predicate on the outer collection.
* 4) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
- * TODO (GLENN) 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ * 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
*/
DROP DATAVERSE TestYelp IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp
index 05a7c76..e59799e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.9.query.sqlpp
new file mode 100644
index 0000000..58ba46a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.9.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+ SELECT VALUE D
+) AS CD ON CD /* +indexnl */ = M.date
+SELECT COUNT(*);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp
index 039df8c..6a74203 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp
@@ -22,7 +22,7 @@
* 1) Implicit INNER JOIN query, with additional predicate on inner collection before the unnest.
* 2) Implicit INNER JOIN query, with additional predicate on the outer collection.
* 3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
- * TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ * 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
*/
DROP DATAVERSE TestYelp IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp
index 0d37224..5a6513a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.8.query.sqlpp
new file mode 100644
index 0000000..7bfe568
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.8.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+SET `compiler.arrayindex` "true";
+USE TestYelp;
+
+FROM YelpCheckinDateMarkers M
+LEFT OUTER JOIN (
+ FROM YelpCheckin C, C.dates CD
+ SELECT VALUE CD
+) AS D ON D /*+ indexnl */ = M.datetime
+SELECT COUNT(*);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp
index 5e18a83..758f0e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -33,5 +33,5 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp
index fcdcde7..c1eb1c0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -36,4 +36,4 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp
index 4b98d54..c144bd7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -36,5 +36,5 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date);
-CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, time) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp
index 205d0bb..4458dc2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -36,5 +36,5 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
index 22c7899..cf95c07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -40,5 +40,7 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates)
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
index 27bc78a..f70b1e1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
@@ -36,5 +36,5 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, t.time);
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, t.time) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
index da4ad89..14eb194 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -33,5 +33,5 @@
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
-CREATE INDEX IdxYelpBusinessCheckinDates ON YelpCheckin (business_id, UNNEST dates);
-CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST dates, business_id);
+CREATE INDEX IdxYelpBusinessCheckinDates ON YelpCheckin (business_id, UNNEST dates) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST dates, business_id) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/complex-structures/complex-structures-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/complex-structures/complex-structures-01.1.ddl.sqlpp
new file mode 100644
index 0000000..e8d78ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/complex-structures/complex-structures-01.1.ddl.sqlpp
@@ -0,0 +1,87 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+CREATE TYPE TestType AS {
+ _id: uuid,
+ closed_field_1: {
+ closed_field_2: [{
+ closed_field_3: {
+ closed_field_4: [{
+ closed_field_5: bigint
+ }]
+ }
+ }]
+ }
+};
+CREATE DATASET TestDataset (TestType)
+PRIMARY KEY _id AUTOGENERATED;
+
+
+-- Fully open index 1. Unnest flags: [0, 1, 0]
+CREATE INDEX testIndex1
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ SELECT open_field_3a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Fully open index 2. Unnest flags: [0, 1, 0, 0]
+CREATE INDEX testIndex2
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ SELECT open_field_3b.open_field_4 : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Fully open index 3. Unnest flags: [0, 1, 0, 1]
+CREATE INDEX testIndex3
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ UNNEST open_field_3c.open_field_4a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Fully open index 4. Unnest flags: [0, 1, 0, 1, 0]
+CREATE INDEX testIndex4
+ON TestDataset ( UNNEST open_field_1.open_field_2
+ UNNEST open_field_3c.open_field_4b
+ SELECT open_field_5 : bigint )
+EXCLUDE UNKNOWN KEY;
+
+
+-- Partially open index 1. [0, 1, 0]
+CREATE INDEX testIndex1c
+ON TestDataset ( UNNEST closed_field_1.open_field_2
+ SELECT open_field_3a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Partially open index 2. Unnest flags: [0, 1, 0, 0]
+CREATE INDEX testIndex2c
+ON TestDataset ( UNNEST closed_field_1.closed_field_2
+ SELECT open_field_3b.open_field_4 : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Partially open index 3. Unnest flags: [0, 1, 0, 1]
+CREATE INDEX testIndex3c
+ON TestDataset ( UNNEST closed_field_1.closed_field_2
+ UNNEST closed_field_3.open_field_4a : bigint )
+EXCLUDE UNKNOWN KEY;
+
+-- Partially open index 4. Unnest flags: [0, 1, 0, 1, 0]
+CREATE INDEX testIndex4c
+ON TestDataset ( UNNEST closed_field_1.closed_field_2
+ UNNEST closed_field_3.closed_field_4
+ SELECT open_field_5 : bigint )
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/complex-structures/complex-structures-01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/complex-structures/complex-structures-01.2.query.sqlpp
new file mode 100644
index 0000000..1b854c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/complex-structures/complex-structures-01.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+FROM Metadata.`Index` D
+WHERE D.IndexName LIKE "testIndex%" AND
+ D.DataverseName = "TestDataverse"
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+ORDER BY D.IndexName ASC;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp
index 62d70c3..d5c0a72 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -30,6 +30,6 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates: string) ;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates: string) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp
index 0edb2cb..2fc09bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -30,6 +30,6 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp
index 733d2d9..89e7cc1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -30,6 +30,6 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?) ;
-CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
-CREATE INDEX IdxYelpCheckinTime ON YelpCheckin (UNNEST checkin_times SELECT time : string ) ;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?);
+CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date : string ) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelpCheckinTime ON YelpCheckin (UNNEST checkin_times SELECT time : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp
index 8ea3bfe..82f80fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -30,6 +30,6 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
index c9b993c..00a06ce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -30,5 +30,7 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string )
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
index 5747a3c..2769349 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
@@ -30,5 +30,8 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date : string , t.time : string ) ;
+CREATE INDEX IdxYelpCheckinDateTime
+ON YelpCheckin (UNNEST checkin_times SELECT date : string ,
+ t.time : string )
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
index 2a9497a..6194b55 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -30,4 +30,26 @@
};
CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
-CREATE INDEX IdxYelpBusinessCheckinDates ON YelpCheckin (business_id : string ?, UNNEST dates : string ?) ;
+CREATE INDEX IdxYelpBusinessCheckinDates
+ON YelpCheckin (business_id : string, UNNEST dates : string)
+EXCLUDE UNKNOWN KEY;
+
+-- More complex indexes created below.
+CREATE INDEX IdxYelp1 ON YelpCheckin (
+ ( UNNEST outer_dates_1
+ SELECT date : string ),
+ backup_business_id : string
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelp2 ON YelpCheckin (
+ business_id : string,
+ ( UNNEST outer_dates_2
+ SELECT time : string )
+) EXCLUDE UNKNOWN KEY;
+CREATE INDEX IdxYelp3 ON YelpCheckin (
+ business_id : string,
+ ( UNNEST outer_dates
+ UNNEST inner_dates
+ SELECT date : string,
+ time : string ),
+ backup_business_id : string
+) EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp
index bee279e..738a69b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp
@@ -17,6 +17,7 @@
* under the License.
*/
-SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
-FROM Metadata.`Index` D
-WHERE D.IndexName = "IdxYelpBusinessCheckinDates" AND D.DataverseName = "TestYelp";
+FROM Metadata.`Index` D
+WHERE D.IndexName LIKE "IdxYelp%" AND D.DataverseName = "TestYelp"
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+ORDER BY D.IndexName;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
index b75bf1b..39fd911 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "dates": [
- "2016-04-26 19:49:16",
- "2016-08-30 18:36:57",
- "2016-10-15 02:45:18",
- "2016-11-18 01:54:50",
- "2017-04-20 18:39:06",
- "2017-05-03 17:58:02",
- "2019-03-19 22:04:48"
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "dates": [
- "2018-05-25 19:52:07",
- "2018-09-18 16:09:44",
- "2019-10-18 21:29:09"
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "dates": [
- "2019-06-07 17:54:58"
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "dates": [
- "2011-05-03 20:54:05",
- "2011-08-23 20:49:45",
- "2014-12-04 06:13:01",
- "2016-11-16 19:25:55"
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "dates": [
- "2016-06-18 21:35:45",
- "2016-10-15 18:17:51"
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "dates": [
- "2015-04-02 21:45:17"
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "dates": [
- "2014-08-27 17:49:18",
- "2015-12-19 21:30:31",
- "2018-11-27 15:53:50"
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "dates": [
- "2014-01-28 20:56:04",
- "2014-11-16 16:11:58",
- "2015-11-15 19:21:53",
- "2015-11-15 19:33:39"
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "dates": [
- "2019-04-11 18:30:12"
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "dates": [
- "2015-06-06 20:01:06",
- "2019-03-14 22:01:52"
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "dates": [
- "2018-09-29 18:55:17",
- "2018-10-20 16:48:05",
- "2018-10-20 22:20:24"
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "dates": [
- "2011-04-23 21:11:22",
- "2014-05-04 19:42:48",
- "2014-05-11 19:16:08",
- "2014-06-04 19:14:18",
- "2015-12-05 19:22:42",
- "2017-05-15 23:19:00"
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "dates": [
- "2012-07-13 21:43:57",
- "2016-12-24 02:27:31",
- "2017-08-31 00:35:26"
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "dates": [
- "2013-04-13 12:35:33",
- "2013-08-19 23:35:49",
- "2013-10-04 19:14:56"
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "dates": [
- "2016-09-10 19:26:19",
- "2018-09-08 14:15:37",
- "2019-09-13 22:47:25"
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "dates": [
- "2017-05-19 14:30:16",
- "2017-05-19 14:30:25",
- "2017-08-28 15:49:37",
- "2017-09-20 20:19:51",
- "2017-10-01 16:31:05",
- "2017-10-01 16:56:27",
- "2017-12-27 23:33:20"
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "dates": [
- "2019-06-05 18:22:49"
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "dates": [
- "2011-09-24 21:37:32",
- "2014-03-10 20:20:07",
- "2015-05-27 00:40:24",
- "2015-08-29 17:58:15",
- "2018-03-16 15:03:26"
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "dates": [
- "2015-03-16 23:51:16",
- "2015-12-21 04:48:01",
- "2016-10-28 20:22:42",
- "2016-10-28 20:23:00"
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "dates": [
- "2013-10-22 16:49:21",
- "2014-11-21 17:39:24"
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "dates": [
- "2014-08-07 18:30:48",
- "2014-09-16 20:41:45",
- "2014-10-12 23:22:27",
- "2015-07-21 20:43:56",
- "2015-07-21 20:45:07"
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "dates": [
- "2015-05-02 19:49:05",
- "2015-05-06 03:52:18",
- "2015-09-26 01:13:19"
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "dates": [
- "2015-04-11 13:14:14",
- "2015-11-21 16:05:56",
- "2016-05-06 14:10:04",
- "2017-08-09 15:15:10",
- "2017-10-21 15:12:56"
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "dates": [
- "2015-12-03 18:44:00",
- "2016-03-17 18:19:21",
- "2016-11-02 15:58:38"
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "dates": [
- "2019-04-04 22:02:37"
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "dates": [
- "2019-02-27 14:03:08"
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "dates": [
- "2011-12-21 19:02:51",
- "2012-04-15 04:21:39",
- "2012-04-15 14:23:56",
- "2013-06-30 22:39:51",
- "2013-10-04 20:34:13",
- "2014-07-16 02:28:40"
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "dates": [
- "2018-06-13 20:16:07"
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "dates": [
- "2015-05-29 16:46:17",
- "2015-06-01 15:03:53"
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "dates": [
- "2011-10-08 12:02:23",
- "2014-08-18 02:11:11",
- "2016-01-07 05:27:51",
- "2016-10-21 20:15:55",
- "2016-12-01 03:57:10",
- "2016-12-29 01:54:42",
- "2018-07-22 19:55:31",
- "2018-09-07 01:42:54",
- "2019-03-08 03:41:06"
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
index e64a93c..5ccbf5d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
@@ -20,7 +20,8 @@
USE TestYelp;
-SELECT COUNT(*)
-FROM YelpCheckin C
-WHERE EVERY D IN C.dates
-SATISFIES D > "2016" AND D < "2017";
+FROM YelpCheckin C
+WHERE LEN(C.dates) > 0 AND
+ ( EVERY D IN C.dates
+ SATISFIES D > "2016" AND D < "2017" )
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..a8eb5bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.dates
+ SATISFIES D > "2016" AND D < "2017"
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
index 2549e8f..d9f0f7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
@@ -18,480 +18,6 @@
*/
USE TestYelp;
-
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": {
- "dates": [
- "2016-04-26",
- "2016-08-30",
- "2016-10-15",
- "2016-11-18",
- "2017-04-20",
- "2017-05-03",
- "2019-03-19"
- ],
- "times": [
- "19:49:16",
- "18:36:57",
- "02:45:18",
- "01:54:50",
- "18:39:06",
- "17:58:02",
- "22:04:48"
- ]
- }
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": {
- "dates": [
- "2018-05-25",
- "2018-09-18",
- "2019-10-18"
- ],
- "times": [
- "19:52:07",
- "16:09:44",
- "21:29:09"
- ]
- }
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": {
- "dates": [
- "2019-06-07"
- ],
- "times": [
- "17:54:58"
- ]
- }
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": {
- "dates": [
- "2011-05-03",
- "2011-08-23",
- "2014-12-04",
- "2016-11-16"
- ],
- "times": [
- "20:54:05",
- "20:49:45",
- "06:13:01",
- "19:25:55"
- ]
- }
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": {
- "dates": [
- "2016-06-18",
- "2016-10-15"
- ],
- "times": [
- "21:35:45",
- "18:17:51"
- ]
- }
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": {
- "dates": [
- "2015-04-02"
- ],
- "times": [
- "21:45:17"
- ]
- }
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": {
- "dates": [
- "2014-08-27",
- "2015-12-19",
- "2018-11-27"
- ],
- "times": [
- "17:49:18",
- "21:30:31",
- "15:53:50"
- ]
- }
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": {
- "dates": [
- "2014-01-28",
- "2014-11-16",
- "2015-11-15",
- "2015-11-15"
- ],
- "times": [
- "20:56:04",
- "16:11:58",
- "19:21:53",
- "19:33:39"
- ]
- }
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": {
- "dates": [
- "2019-04-11"
- ],
- "times": [
- "18:30:12"
- ]
- }
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": {
- "dates": [
- "2015-06-06",
- "2019-03-14"
- ],
- "times": [
- "20:01:06",
- "22:01:52"
- ]
- }
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": {
- "dates": [
- "2018-09-29",
- "2018-10-20",
- "2018-10-20"
- ],
- "times": [
- "18:55:17",
- "16:48:05",
- "22:20:24"
- ]
- }
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": {
- "dates": [
- "2011-04-23",
- "2014-05-04",
- "2014-05-11",
- "2014-06-04",
- "2015-12-05",
- "2017-05-15"
- ],
- "times": [
- "21:11:22",
- "19:42:48",
- "19:16:08",
- "19:14:18",
- "19:22:42",
- "23:19:00"
- ]
- }
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": {
- "dates": [
- "2012-07-13",
- "2016-12-24",
- "2017-08-31"
- ],
- "times": [
- "21:43:57",
- "02:27:31",
- "00:35:26"
- ]
- }
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": {
- "dates": [
- "2013-04-13",
- "2013-08-19",
- "2013-10-04"
- ],
- "times": [
- "12:35:33",
- "23:35:49",
- "19:14:56"
- ]
- }
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": {
- "dates": [
- "2016-09-10",
- "2018-09-08",
- "2019-09-13"
- ],
- "times": [
- "19:26:19",
- "14:15:37",
- "22:47:25"
- ]
- }
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": {
- "dates": [
- "2017-05-19",
- "2017-05-19",
- "2017-08-28",
- "2017-09-20",
- "2017-10-01",
- "2017-10-01",
- "2017-12-27"
- ],
- "times": [
- "14:30:16",
- "14:30:25",
- "15:49:37",
- "20:19:51",
- "16:31:05",
- "16:56:27",
- "23:33:20"
- ]
- }
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": {
- "dates": [
- "2019-06-05"
- ],
- "times": [
- "18:22:49"
- ]
- }
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": {
- "dates": [
- "2011-09-24",
- "2014-03-10",
- "2015-05-27",
- "2015-08-29",
- "2018-03-16"
- ],
- "times": [
- "21:37:32",
- "20:20:07",
- "00:40:24",
- "17:58:15",
- "15:03:26"
- ]
- }
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": {
- "dates": [
- "2015-03-16",
- "2015-12-21",
- "2016-10-28",
- "2016-10-28"
- ],
- "times": [
- "23:51:16",
- "04:48:01",
- "20:22:42",
- "20:23:00"
- ]
- }
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": {
- "dates": [
- "2013-10-22",
- "2014-11-21"
- ],
- "times": [
- "16:49:21",
- "17:39:24"
- ]
- }
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": {
- "dates": [
- "2014-08-07",
- "2014-09-16",
- "2014-10-12",
- "2015-07-21",
- "2015-07-21"
- ],
- "times": [
- "18:30:48",
- "20:41:45",
- "23:22:27",
- "20:43:56",
- "20:45:07"
- ]
- }
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": {
- "dates": [
- "2015-05-02",
- "2015-05-06",
- "2015-09-26"
- ],
- "times": [
- "19:49:05",
- "03:52:18",
- "01:13:19"
- ]
- }
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": {
- "dates": [
- "2015-04-11",
- "2015-11-21",
- "2016-05-06",
- "2017-08-09",
- "2017-10-21"
- ],
- "times": [
- "13:14:14",
- "16:05:56",
- "14:10:04",
- "15:15:10",
- "15:12:56"
- ]
- }
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": {
- "dates": [
- "2015-12-03",
- "2016-03-17",
- "2016-11-02"
- ],
- "times": [
- "18:44:00",
- "18:19:21",
- "15:58:38"
- ]
- }
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": {
- "dates": [
- "2019-04-04"
- ],
- "times": [
- "22:02:37"
- ]
- }
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": {
- "dates": [
- "2019-02-27"
- ],
- "times": [
- "14:03:08"
- ]
- }
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": {
- "dates": [
- "2011-12-21",
- "2012-04-15",
- "2012-04-15",
- "2013-06-30",
- "2013-10-04",
- "2014-07-16"
- ],
- "times": [
- "19:02:51",
- "04:21:39",
- "14:23:56",
- "22:39:51",
- "20:34:13",
- "02:28:40"
- ]
- }
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": {
- "dates": [
- "2018-06-13"
- ],
- "times": [
- "20:16:07"
- ]
- }
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": {
- "dates": [
- "2015-05-29",
- "2015-06-01"
- ],
- "times": [
- "16:46:17",
- "15:03:53"
- ]
- }
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": {
- "dates": [
- "2011-10-08",
- "2014-08-18",
- "2016-01-07",
- "2016-10-21",
- "2016-12-01",
- "2016-12-29",
- "2018-07-22",
- "2018-09-07",
- "2019-03-08"
- ],
- "times": [
- "12:02:23",
- "02:11:11",
- "05:27:51",
- "20:15:55",
- "03:57:10",
- "01:54:42",
- "19:55:31",
- "01:42:54",
- "03:41:06"
- ]
- }
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-2.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp
index 0ab72fe..caaaad0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp
index c9652f1..173f02c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp
@@ -20,7 +20,7 @@
USE TestYelp;
-SELECT COUNT(*)
-FROM YelpCheckin C
-WHERE EVERY D IN C.checkin_times.dates
-SATISFIES D > "2016" AND D < "2017";
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.checkin_times.dates
+ SATISFIES D > "2016" AND D < "2017"
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
index 02ddf21..f05a19b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {"date": "2016-04-26", "time": "19:49:16"},
- {"date": "2016-08-30", "time": "18:36:57"},
- {"date": "2016-10-15", "time": "02:45:18"},
- {"date": "2016-11-18", "time": "01:54:50"},
- {"date": "2017-04-20", "time": "18:39:06"},
- {"date": "2017-05-03", "time": "17:58:02"},
- {"date": "2019-03-19", "time": "22:04:48"}
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {"date": "2018-05-25", "time": "19:52:07"},
- {"date": "2018-09-18", "time": "16:09:44"},
- {"date": "2019-10-18", "time": "21:29:09"}
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {"date": "2019-06-07", "time": "17:54:58"}
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {"date": "2011-05-03", "time": "20:54:05"},
- {"date": "2011-08-23", "time": "20:49:45"},
- {"date": "2014-12-04", "time": "06:13:01"},
- {"date": "2016-11-16", "time": "19:25:55"}
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {"date": "2016-06-18", "time": "21:35:45"},
- {"date": "2016-10-15", "time": "18:17:51"}
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {"date": "2015-04-02", "time": "21:45:17"}
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {"date": "2014-08-27", "time": "17:49:18"},
- {"date": "2015-12-19", "time": "21:30:31"},
- {"date": "2018-11-27", "time": "15:53:50"}
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {"date": "2014-01-28", "time": "20:56:04"},
- {"date": "2014-11-16", "time": "16:11:58"},
- {"date": "2015-11-15", "time": "19:21:53"},
- {"date": "2015-11-15", "time": "19:33:39"}
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {"date": "2019-04-11", "time": "18:30:12"}
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {"date": "2015-06-06", "time": "20:01:06"},
- {"date": "2019-03-14", "time": "22:01:52"}
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {"date": "2018-09-29", "time": "18:55:17"},
- {"date": "2018-10-20", "time": "16:48:05"},
- {"date": "2018-10-20", "time": "22:20:24"}
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {"date": "2011-04-23", "time": "21:11:22"},
- {"date": "2014-05-04", "time": "19:42:48"},
- {"date": "2014-05-11", "time": "19:16:08"},
- {"date": "2014-06-04", "time": "19:14:18"},
- {"date": "2015-12-05", "time": "19:22:42"},
- {"date": "2017-05-15", "time": "23:19:00"}
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {"date": "2012-07-13", "time": "21:43:57"},
- {"date": "2016-12-24", "time": "02:27:31"},
- {"date": "2017-08-31", "time": "00:35:26"}
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {"date": "2013-04-13", "time": "12:35:33"},
- {"date": "2013-08-19", "time": "23:35:49"},
- {"date": "2013-10-04", "time": "19:14:56"}
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {"date": "2016-09-10", "time": "19:26:19"},
- {"date": "2018-09-08", "time": "14:15:37"},
- {"date": "2019-09-13", "time": "22:47:25"}
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {"date": "2017-05-19", "time": "14:30:16"},
- {"date": "2017-05-19", "time": "14:30:25"},
- {"date": "2017-08-28", "time": "15:49:37"},
- {"date": "2017-09-20", "time": "20:19:51"},
- {"date": "2017-10-01", "time": "16:31:05"},
- {"date": "2017-10-01", "time": "16:56:27"},
- {"date": "2017-12-27", "time": "23:33:20"}
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {"date": "2019-06-05", "time": "18:22:49"}
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {"date": "2011-09-24", "time": "21:37:32"},
- {"date": "2014-03-10", "time": "20:20:07"},
- {"date": "2015-05-27", "time": "00:40:24"},
- {"date": "2015-08-29", "time": "17:58:15"},
- {"date": "2018-03-16", "time": "15:03:26"}
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {"date": "2015-03-16", "time": "23:51:16"},
- {"date": "2015-12-21", "time": "04:48:01"},
- {"date": "2016-10-28", "time": "20:22:42"},
- {"date": "2016-10-28", "time": "20:23:00"}
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {"date": "2013-10-22", "time": "16:49:21"},
- {"date": "2014-11-21", "time": "17:39:24"}
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {"date": "2014-08-07", "time": "18:30:48"},
- {"date": "2014-09-16", "time": "20:41:45"},
- {"date": "2014-10-12", "time": "23:22:27"},
- {"date": "2015-07-21", "time": "20:43:56"},
- {"date": "2015-07-21", "time": "20:45:07"}
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {"date": "2015-05-02", "time": "19:49:05"},
- {"date": "2015-05-06", "time": "03:52:18"},
- {"date": "2015-09-26", "time": "01:13:19"}
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {"date": "2015-04-11", "time": "13:14:14"},
- {"date": "2015-11-21", "time": "16:05:56"},
- {"date": "2016-05-06", "time": "14:10:04"},
- {"date": "2017-08-09", "time": "15:15:10"},
- {"date": "2017-10-21", "time": "15:12:56"}
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {"date": "2015-12-03", "time": "18:44:00"},
- {"date": "2016-03-17", "time": "18:19:21"},
- {"date": "2016-11-02", "time": "15:58:38"}
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {"date": "2019-04-04", "time": "22:02:37"}
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {"date": "2019-02-27", "time": "14:03:08"}
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {"date": "2011-12-21", "time": "19:02:51"},
- {"date": "2012-04-15", "time": "04:21:39"},
- {"date": "2012-04-15", "time": "14:23:56"},
- {"date": "2013-06-30", "time": "22:39:51"},
- {"date": "2013-10-04", "time": "20:34:13"},
- {"date": "2014-07-16", "time": "02:28:40"}
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {"date": "2018-06-13", "time": "20:16:07"}
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {"date": "2015-05-29", "time": "16:46:17"},
- {"date": "2015-06-01", "time": "15:03:53"}
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {"date": "2011-10-08", "time": "12:02:23"},
- {"date": "2014-08-18", "time": "02:11:11"},
- {"date": "2016-01-07", "time": "05:27:51"},
- {"date": "2016-10-21", "time": "20:15:55"},
- {"date": "2016-12-01", "time": "03:57:10"},
- {"date": "2016-12-29", "time": "01:54:42"},
- {"date": "2018-07-22", "time": "19:55:31"},
- {"date": "2018-09-07", "time": "01:42:54"},
- {"date": "2019-03-08", "time": "03:41:06"}
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-3.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp
index ee911b4..4154d37 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp
index 6a73591..40b2ab4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp
@@ -20,7 +20,7 @@
USE TestYelp;
-SELECT C.business_id
-FROM YelpCheckin C
-WHERE EVERY D IN C.checkin_times
-SATISFIES D.date BETWEEN "2016" AND "2017";
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.checkin_times
+ SATISFIES D.date > "2016" AND D.date < "2017"
+SELECT C.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
index 7fcf9e4..197d417 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
@@ -19,407 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {
- "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
- "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
- },
- {
- "dates": ["2017-04-20", "2017-05-03"],
- "times": ["18:39:06", "17:58:02"]
- },
- {
- "dates": ["2019-03-19"],
- "times": ["22:04:48"]
- }
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {
- "dates": ["2018-05-25", "2018-09-18"],
- "times": ["19:52:07", "16:09:44"]
- },
- {
- "dates": ["2019-10-18"],
- "times": ["21:29:09"]
- }
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {
- "dates": ["2019-06-07"],
- "times": ["17:54:58"]
- }
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {
- "dates": ["2011-05-03", "2011-08-23"],
- "times": ["20:54:05", "20:49:45"]
- },
- {
- "dates": ["2014-12-04"],
- "times": ["06:13:01"]
- },
- {
- "dates": ["2016-11-16"],
- "times": ["19:25:55"]
- }
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {
- "dates": ["2016-06-18", "2016-10-15"],
- "times": ["21:35:45", "18:17:51"]
- }
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {
- "dates": ["2015-04-02"],
- "times": ["21:45:17"]
- }
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {
- "dates": ["2014-08-27"],
- "times": ["17:49:18"]
- },
- {
- "dates": ["2015-12-19"],
- "times": ["21:30:31"]
- },
- {
- "dates": ["2018-11-27"],
- "times": ["15:53:50"]
- }
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {
- "dates": ["2014-01-28", "2014-11-16"],
- "times": ["20:56:04", "16:11:58"]
- },
- {
- "dates": ["2015-11-15", "2015-11-15"],
- "times": ["19:21:53", "19:33:39"]
- }
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {
- "dates": ["2019-04-11"],
- "times": ["18:30:12"]
- }
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {
- "dates": ["2015-06-06"],
- "times": ["20:01:06"]
- },
- {
- "dates": ["2019-03-14"],
- "times": ["22:01:52"]
- }
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {
- "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
- "times": ["18:55:17", "16:48:05", "22:20:24"]
- }
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {
- "dates": ["2011-04-23"],
- "times": ["21:11:22"]
- },
- {
- "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
- "times": ["19:42:48", "19:16:08", "19:14:18"]
- },
- {
- "dates": ["2015-12-05"],
- "times": ["19:22:42"]
- },
- {
- "dates": ["2017-05-15"],
- "times": ["23:19:00"]
- }
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {
- "dates": ["2012-07-13"],
- "times": ["21:43:57"]
- },
- {
- "dates": ["2016-12-24"],
- "times": ["02:27:31"]
- },
- {
- "dates": ["2017-08-31"],
- "times": ["00:35:26"]
- }
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {
- "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
- "times": ["12:35:33", "23:35:49", "19:14:56"]
- }
-
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {
- "dates": ["2016-09-10"],
- "times": ["19:26:19"]
- },
- {
- "dates": ["2018-09-08"],
- "times": ["14:15:37"]
- },
- {
- "dates": ["2019-09-13"],
- "times": ["22:47:25"]
- }
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {
- "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
- "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
- }
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {
- "dates": ["2019-06-05"],
- "times": ["18:22:49"]
- }
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {
- "dates": ["2011-09-24"],
- "times": ["21:37:32"]
- },
- {
- "dates": ["2014-03-10"],
- "times": ["20:20:07"]
- },
- {
- "dates": ["2015-05-27", "2015-08-29"],
- "times": ["00:40:24", "17:58:15"]
- },
- {
- "dates": ["2018-03-16"],
- "times": ["15:03:26"]
- }
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {
- "dates": ["2015-03-16", "2015-12-21"],
- "times": ["23:51:16", "04:48:01"]
- },
- {
- "dates": ["2016-10-28", "2016-10-28"],
- "times": ["20:22:42", "20:23:00"]
- }
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {
- "dates": ["2013-10-22"],
- "times": ["16:49:21"]
- },
- {
- "dates": ["2014-11-21"],
- "times": ["17:39:24"]
- }
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {
- "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
- "times": ["18:30:48", "20:41:45", "23:22:27"]
- },
- {
- "dates": ["2015-07-21", "2015-07-21"],
- "times": ["20:43:56", "20:45:07"]
- }
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {
- "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
- "times": ["19:49:05", "03:52:18", "01:13:19"]
- }
-
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {
- "dates": ["2015-04-11", "2015-11-21"],
- "times": ["13:14:14", "16:05:56"]
- },
- {
- "dates": ["2016-05-06"],
- "times": ["14:10:04"]
- },
- {
- "dates": ["2017-08-09", "2017-10-21"],
- "times": ["15:15:10", "15:12:56"]
- }
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {
- "dates": ["2015-12-03"],
- "times": ["18:44:00"]
- },
- {
- "dates": ["2016-03-17", "2016-11-02"],
- "times": ["18:19:21", "15:58:38"]
- }
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {
- "dates": ["2019-04-04"],
- "times": ["22:02:37"]
- }
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {
- "dates": ["2019-02-27"],
- "times": ["14:03:08"]
- }
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {
- "dates": ["2011-12-21"],
- "times": ["19:02:51"]
- },
- {
- "dates": ["2012-04-15", "2012-04-15"],
- "times": ["04:21:39", "14:23:56"]
- },
- {
- "dates": ["2013-06-30", "2013-10-04"],
- "times": ["22:39:51", "20:34:13"]
- },
- {
- "dates": ["2014-07-16"],
- "times": ["02:28:40"]
- }
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {
- "dates": ["2018-06-13"],
- "times": ["20:16:07"]
- }
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {
- "dates": ["2015-05-29", "2015-06-01"],
- "times": ["16:46:17", "15:03:53"]
- }
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {
- "dates": ["2011-10-08"],
- "times": ["12:02:23"]
- },
- {
- "dates": ["2014-08-18"],
- "times": ["02:11:11"]
- },
- {
- "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
- "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
- },
- {
- "dates": ["2018-07-22", "2018-09-07"],
- "times": ["19:55:31", "01:42:54"]
- },
- {
- "dates": ["2019-03-08"],
- "times": ["03:41:06"]
- }
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-4.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp
index 05a7c76..e59799e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
index 490dbdd..1035a86 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
@@ -20,10 +20,8 @@
USE TestYelp;
-SELECT C.business_id
-FROM YelpCheckin C
-WHERE EVERY CT IN C.checkin_times
-SATISFIES (
- SOME D IN CT.dates
- SATISFIES "2019-06-07" = D
-);
+FROM YelpCheckin C
+WHERE SOME AND EVERY CT IN C.checkin_times
+ SATISFIES ( SOME D IN CT.dates
+ SATISFIES "2019-06-07" = D )
+SELECT C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp
index 4032e84..a60a21e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp
@@ -20,10 +20,8 @@
USE TestYelp;
-SELECT C.business_id
-FROM YelpCheckin C
-WHERE EVERY CT IN C.checkin_times
-SATISFIES (
- EVERY D IN CT.dates
- SATISFIES D > "2019-06-05"
-);
+FROM YelpCheckin C
+WHERE SOME AND EVERY CT IN C.checkin_times
+ SATISFIES ( SOME AND EVERY D IN CT.dates
+ SATISFIES D > "2019-06-05" )
+SELECT C.business_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp
index 6498c14..5adc3e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -19,288 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "checkin_id": 1,
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "dates": [
- "2016-04-26 19:49:16",
- "2016-08-30 18:36:57",
- "2016-10-15 02:45:18",
- "2016-11-18 01:54:50",
- "2017-04-20 18:39:06",
- "2017-05-03 17:58:02",
- "2019-03-19 22:04:48"
- ]
- },
- {
- "checkin_id": 2,
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "dates": [
- "2018-05-25 19:52:07",
- "2018-09-18 16:09:44",
- "2019-10-18 21:29:09"
- ]
- },
- {
- "checkin_id": 3,
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "dates": [
- "2019-06-07 17:54:58"
- ]
- },
- {
- "checkin_id": 4,
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "dates": [
- "2011-05-03 20:54:05",
- "2011-08-23 20:49:45",
- "2014-12-04 06:13:01",
- "2016-11-16 19:25:55"
- ]
- },
- {
- "checkin_id": 5,
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "dates": [
- "2016-06-18 21:35:45",
- "2016-10-15 18:17:51"
- ]
- },
- {
- "checkin_id": 6,
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "dates": [
- "2015-04-02 21:45:17"
- ]
- },
- {
- "checkin_id": 7,
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "dates": [
- "2014-08-27 17:49:18",
- "2015-12-19 21:30:31",
- "2018-11-27 15:53:50"
- ]
- },
- {
- "checkin_id": 8,
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "dates": [
- "2014-01-28 20:56:04",
- "2014-11-16 16:11:58",
- "2015-11-15 19:21:53",
- "2015-11-15 19:33:39"
- ]
- },
- {
- "checkin_id": 9,
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "dates": [
- "2019-04-11 18:30:12"
- ]
- },
- {
- "checkin_id": 10,
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "dates": [
- "2015-06-06 20:01:06",
- "2019-03-14 22:01:52"
- ]
- },
- {
- "checkin_id": 11,
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "dates": [
- "2018-09-29 18:55:17",
- "2018-10-20 16:48:05",
- "2018-10-20 22:20:24"
- ]
- },
- {
- "checkin_id": 12,
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "dates": [
- "2011-04-23 21:11:22",
- "2014-05-04 19:42:48",
- "2014-05-11 19:16:08",
- "2014-06-04 19:14:18",
- "2015-12-05 19:22:42",
- "2017-05-15 23:19:00"
- ]
- },
- {
- "checkin_id": 13,
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "dates": [
- "2012-07-13 21:43:57",
- "2016-12-24 02:27:31",
- "2017-08-31 00:35:26"
- ]
- },
- {
- "checkin_id": 14,
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "dates": [
- "2013-04-13 12:35:33",
- "2013-08-19 23:35:49",
- "2013-10-04 19:14:56"
- ]
- },
- {
- "checkin_id": 15,
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "dates": [
- "2016-09-10 19:26:19",
- "2018-09-08 14:15:37",
- "2019-09-13 22:47:25"
- ]
- },
- {
- "checkin_id": 16,
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "dates": [
- "2017-05-19 14:30:16",
- "2017-05-19 14:30:25",
- "2017-08-28 15:49:37",
- "2017-09-20 20:19:51",
- "2017-10-01 16:31:05",
- "2017-10-01 16:56:27",
- "2017-12-27 23:33:20"
- ]
- },
- {
- "checkin_id": 17,
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "dates": [
- "2019-06-05 18:22:49"
- ]
- },
- {
- "checkin_id": 18,
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "dates": [
- "2011-09-24 21:37:32",
- "2014-03-10 20:20:07",
- "2015-05-27 00:40:24",
- "2015-08-29 17:58:15",
- "2018-03-16 15:03:26"
- ]
- },
- {
- "checkin_id": 19,
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "dates": [
- "2015-03-16 23:51:16",
- "2015-12-21 04:48:01",
- "2016-10-28 20:22:42",
- "2016-10-28 20:23:00"
- ]
- },
- {
- "checkin_id": 20,
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "dates": [
- "2013-10-22 16:49:21",
- "2014-11-21 17:39:24"
- ]
- },
- {
- "checkin_id": 21,
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "dates": [
- "2014-08-07 18:30:48",
- "2014-09-16 20:41:45",
- "2014-10-12 23:22:27",
- "2015-07-21 20:43:56",
- "2015-07-21 20:45:07"
- ]
- },
- {
- "checkin_id": 22,
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "dates": [
- "2015-05-02 19:49:05",
- "2015-05-06 03:52:18",
- "2015-09-26 01:13:19"
- ]
- },
- {
- "checkin_id": 23,
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "dates": [
- "2015-04-11 13:14:14",
- "2015-11-21 16:05:56",
- "2016-05-06 14:10:04",
- "2017-08-09 15:15:10",
- "2017-10-21 15:12:56"
- ]
- },
- {
- "checkin_id": 24,
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "dates": [
- "2015-12-03 18:44:00",
- "2016-03-17 18:19:21",
- "2016-11-02 15:58:38"
- ]
- },
- {
- "checkin_id": 25,
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "dates": [
- "2019-04-04 22:02:37"
- ]
- },
- {
- "checkin_id": 26,
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "dates": [
- "2019-02-27 14:03:08"
- ]
- },
- {
- "checkin_id": 27,
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "dates": [
- "2011-12-21 19:02:51",
- "2012-04-15 04:21:39",
- "2012-04-15 14:23:56",
- "2013-06-30 22:39:51",
- "2013-10-04 20:34:13",
- "2014-07-16 02:28:40"
- ]
- },
- {
- "checkin_id": 28,
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "dates": [
- "2018-06-13 20:16:07"
- ]
- },
- {
- "checkin_id": 29,
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "dates": [
- "2015-05-29 16:46:17",
- "2015-06-01 15:03:53"
- ]
- },
- {
- "checkin_id": 30,
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "dates": [
- "2011-10-08 12:02:23",
- "2014-08-18 02:11:11",
- "2016-01-07 05:27:51",
- "2016-10-21 20:15:55",
- "2016-12-01 03:57:10",
- "2016-12-29 01:54:42",
- "2018-07-22 19:55:31",
- "2018-09-07 01:42:54",
- "2019-03-08 03:41:06"
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-composite-pk.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp
index e64a93c..90bec06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp
@@ -20,7 +20,7 @@
USE TestYelp;
-SELECT COUNT(*)
-FROM YelpCheckin C
-WHERE EVERY D IN C.dates
-SATISFIES D > "2016" AND D < "2017";
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.dates
+ SATISFIES D > "2016" AND D < "2017"
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp
index b75bf1b..51296bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp
@@ -18,259 +18,6 @@
*/
USE TestYelp;
-
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "dates": [
- "2016-04-26 19:49:16",
- "2016-08-30 18:36:57",
- "2016-10-15 02:45:18",
- "2016-11-18 01:54:50",
- "2017-04-20 18:39:06",
- "2017-05-03 17:58:02",
- "2019-03-19 22:04:48"
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "dates": [
- "2018-05-25 19:52:07",
- "2018-09-18 16:09:44",
- "2019-10-18 21:29:09"
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "dates": [
- "2019-06-07 17:54:58"
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "dates": [
- "2011-05-03 20:54:05",
- "2011-08-23 20:49:45",
- "2014-12-04 06:13:01",
- "2016-11-16 19:25:55"
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "dates": [
- "2016-06-18 21:35:45",
- "2016-10-15 18:17:51"
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "dates": [
- "2015-04-02 21:45:17"
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "dates": [
- "2014-08-27 17:49:18",
- "2015-12-19 21:30:31",
- "2018-11-27 15:53:50"
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "dates": [
- "2014-01-28 20:56:04",
- "2014-11-16 16:11:58",
- "2015-11-15 19:21:53",
- "2015-11-15 19:33:39"
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "dates": [
- "2019-04-11 18:30:12"
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "dates": [
- "2015-06-06 20:01:06",
- "2019-03-14 22:01:52"
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "dates": [
- "2018-09-29 18:55:17",
- "2018-10-20 16:48:05",
- "2018-10-20 22:20:24"
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "dates": [
- "2011-04-23 21:11:22",
- "2014-05-04 19:42:48",
- "2014-05-11 19:16:08",
- "2014-06-04 19:14:18",
- "2015-12-05 19:22:42",
- "2017-05-15 23:19:00"
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "dates": [
- "2012-07-13 21:43:57",
- "2016-12-24 02:27:31",
- "2017-08-31 00:35:26"
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "dates": [
- "2013-04-13 12:35:33",
- "2013-08-19 23:35:49",
- "2013-10-04 19:14:56"
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "dates": [
- "2016-09-10 19:26:19",
- "2018-09-08 14:15:37",
- "2019-09-13 22:47:25"
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "dates": [
- "2017-05-19 14:30:16",
- "2017-05-19 14:30:25",
- "2017-08-28 15:49:37",
- "2017-09-20 20:19:51",
- "2017-10-01 16:31:05",
- "2017-10-01 16:56:27",
- "2017-12-27 23:33:20"
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "dates": [
- "2019-06-05 18:22:49"
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "dates": [
- "2011-09-24 21:37:32",
- "2014-03-10 20:20:07",
- "2015-05-27 00:40:24",
- "2015-08-29 17:58:15",
- "2018-03-16 15:03:26"
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "dates": [
- "2015-03-16 23:51:16",
- "2015-12-21 04:48:01",
- "2016-10-28 20:22:42",
- "2016-10-28 20:23:00"
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "dates": [
- "2013-10-22 16:49:21",
- "2014-11-21 17:39:24"
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "dates": [
- "2014-08-07 18:30:48",
- "2014-09-16 20:41:45",
- "2014-10-12 23:22:27",
- "2015-07-21 20:43:56",
- "2015-07-21 20:45:07"
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "dates": [
- "2015-05-02 19:49:05",
- "2015-05-06 03:52:18",
- "2015-09-26 01:13:19"
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "dates": [
- "2015-04-11 13:14:14",
- "2015-11-21 16:05:56",
- "2016-05-06 14:10:04",
- "2017-08-09 15:15:10",
- "2017-10-21 15:12:56"
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "dates": [
- "2015-12-03 18:44:00",
- "2016-03-17 18:19:21",
- "2016-11-02 15:58:38"
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "dates": [
- "2019-04-04 22:02:37"
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "dates": [
- "2019-02-27 14:03:08"
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "dates": [
- "2011-12-21 19:02:51",
- "2012-04-15 04:21:39",
- "2012-04-15 14:23:56",
- "2013-06-30 22:39:51",
- "2013-10-04 20:34:13",
- "2014-07-16 02:28:40"
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "dates": [
- "2018-06-13 20:16:07"
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "dates": [
- "2015-05-29 16:46:17",
- "2015-06-01 15:03:53"
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "dates": [
- "2011-10-08 12:02:23",
- "2014-08-18 02:11:11",
- "2016-01-07 05:27:51",
- "2016-10-21 20:15:55",
- "2016-12-01 03:57:10",
- "2016-12-29 01:54:42",
- "2018-07-22 19:55:31",
- "2018-09-07 01:42:54",
- "2019-03-08 03:41:06"
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp
index 0d37224..5a6513a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp
index e64a93c..a8eb5bc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp
@@ -20,7 +20,7 @@
USE TestYelp;
-SELECT COUNT(*)
-FROM YelpCheckin C
-WHERE EVERY D IN C.dates
-SATISFIES D > "2016" AND D < "2017";
+FROM YelpCheckin C
+WHERE SOME AND EVERY D IN C.dates
+ SATISFIES D > "2016" AND D < "2017"
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp
index b75bf1b..39fd911 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "dates": [
- "2016-04-26 19:49:16",
- "2016-08-30 18:36:57",
- "2016-10-15 02:45:18",
- "2016-11-18 01:54:50",
- "2017-04-20 18:39:06",
- "2017-05-03 17:58:02",
- "2019-03-19 22:04:48"
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "dates": [
- "2018-05-25 19:52:07",
- "2018-09-18 16:09:44",
- "2019-10-18 21:29:09"
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "dates": [
- "2019-06-07 17:54:58"
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "dates": [
- "2011-05-03 20:54:05",
- "2011-08-23 20:49:45",
- "2014-12-04 06:13:01",
- "2016-11-16 19:25:55"
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "dates": [
- "2016-06-18 21:35:45",
- "2016-10-15 18:17:51"
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "dates": [
- "2015-04-02 21:45:17"
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "dates": [
- "2014-08-27 17:49:18",
- "2015-12-19 21:30:31",
- "2018-11-27 15:53:50"
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "dates": [
- "2014-01-28 20:56:04",
- "2014-11-16 16:11:58",
- "2015-11-15 19:21:53",
- "2015-11-15 19:33:39"
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "dates": [
- "2019-04-11 18:30:12"
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "dates": [
- "2015-06-06 20:01:06",
- "2019-03-14 22:01:52"
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "dates": [
- "2018-09-29 18:55:17",
- "2018-10-20 16:48:05",
- "2018-10-20 22:20:24"
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "dates": [
- "2011-04-23 21:11:22",
- "2014-05-04 19:42:48",
- "2014-05-11 19:16:08",
- "2014-06-04 19:14:18",
- "2015-12-05 19:22:42",
- "2017-05-15 23:19:00"
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "dates": [
- "2012-07-13 21:43:57",
- "2016-12-24 02:27:31",
- "2017-08-31 00:35:26"
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "dates": [
- "2013-04-13 12:35:33",
- "2013-08-19 23:35:49",
- "2013-10-04 19:14:56"
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "dates": [
- "2016-09-10 19:26:19",
- "2018-09-08 14:15:37",
- "2019-09-13 22:47:25"
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "dates": [
- "2017-05-19 14:30:16",
- "2017-05-19 14:30:25",
- "2017-08-28 15:49:37",
- "2017-09-20 20:19:51",
- "2017-10-01 16:31:05",
- "2017-10-01 16:56:27",
- "2017-12-27 23:33:20"
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "dates": [
- "2019-06-05 18:22:49"
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "dates": [
- "2011-09-24 21:37:32",
- "2014-03-10 20:20:07",
- "2015-05-27 00:40:24",
- "2015-08-29 17:58:15",
- "2018-03-16 15:03:26"
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "dates": [
- "2015-03-16 23:51:16",
- "2015-12-21 04:48:01",
- "2016-10-28 20:22:42",
- "2016-10-28 20:23:00"
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "dates": [
- "2013-10-22 16:49:21",
- "2014-11-21 17:39:24"
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "dates": [
- "2014-08-07 18:30:48",
- "2014-09-16 20:41:45",
- "2014-10-12 23:22:27",
- "2015-07-21 20:43:56",
- "2015-07-21 20:45:07"
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "dates": [
- "2015-05-02 19:49:05",
- "2015-05-06 03:52:18",
- "2015-09-26 01:13:19"
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "dates": [
- "2015-04-11 13:14:14",
- "2015-11-21 16:05:56",
- "2016-05-06 14:10:04",
- "2017-08-09 15:15:10",
- "2017-10-21 15:12:56"
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "dates": [
- "2015-12-03 18:44:00",
- "2016-03-17 18:19:21",
- "2016-11-02 15:58:38"
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "dates": [
- "2019-04-04 22:02:37"
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "dates": [
- "2019-02-27 14:03:08"
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "dates": [
- "2011-12-21 19:02:51",
- "2012-04-15 04:21:39",
- "2012-04-15 14:23:56",
- "2013-06-30 22:39:51",
- "2013-10-04 20:34:13",
- "2014-07-16 02:28:40"
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "dates": [
- "2018-06-13 20:16:07"
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "dates": [
- "2015-05-29 16:46:17",
- "2015-06-01 15:03:53"
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "dates": [
- "2011-10-08 12:02:23",
- "2014-08-18 02:11:11",
- "2016-01-07 05:27:51",
- "2016-10-21 20:15:55",
- "2016-12-01 03:57:10",
- "2016-12-29 01:54:42",
- "2018-07-22 19:55:31",
- "2018-09-07 01:42:54",
- "2019-03-08 03:41:06"
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp
index 2549e8f..18135a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp
@@ -19,479 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": {
- "dates": [
- "2016-04-26",
- "2016-08-30",
- "2016-10-15",
- "2016-11-18",
- "2017-04-20",
- "2017-05-03",
- "2019-03-19"
- ],
- "times": [
- "19:49:16",
- "18:36:57",
- "02:45:18",
- "01:54:50",
- "18:39:06",
- "17:58:02",
- "22:04:48"
- ]
- }
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": {
- "dates": [
- "2018-05-25",
- "2018-09-18",
- "2019-10-18"
- ],
- "times": [
- "19:52:07",
- "16:09:44",
- "21:29:09"
- ]
- }
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": {
- "dates": [
- "2019-06-07"
- ],
- "times": [
- "17:54:58"
- ]
- }
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": {
- "dates": [
- "2011-05-03",
- "2011-08-23",
- "2014-12-04",
- "2016-11-16"
- ],
- "times": [
- "20:54:05",
- "20:49:45",
- "06:13:01",
- "19:25:55"
- ]
- }
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": {
- "dates": [
- "2016-06-18",
- "2016-10-15"
- ],
- "times": [
- "21:35:45",
- "18:17:51"
- ]
- }
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": {
- "dates": [
- "2015-04-02"
- ],
- "times": [
- "21:45:17"
- ]
- }
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": {
- "dates": [
- "2014-08-27",
- "2015-12-19",
- "2018-11-27"
- ],
- "times": [
- "17:49:18",
- "21:30:31",
- "15:53:50"
- ]
- }
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": {
- "dates": [
- "2014-01-28",
- "2014-11-16",
- "2015-11-15",
- "2015-11-15"
- ],
- "times": [
- "20:56:04",
- "16:11:58",
- "19:21:53",
- "19:33:39"
- ]
- }
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": {
- "dates": [
- "2019-04-11"
- ],
- "times": [
- "18:30:12"
- ]
- }
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": {
- "dates": [
- "2015-06-06",
- "2019-03-14"
- ],
- "times": [
- "20:01:06",
- "22:01:52"
- ]
- }
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": {
- "dates": [
- "2018-09-29",
- "2018-10-20",
- "2018-10-20"
- ],
- "times": [
- "18:55:17",
- "16:48:05",
- "22:20:24"
- ]
- }
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": {
- "dates": [
- "2011-04-23",
- "2014-05-04",
- "2014-05-11",
- "2014-06-04",
- "2015-12-05",
- "2017-05-15"
- ],
- "times": [
- "21:11:22",
- "19:42:48",
- "19:16:08",
- "19:14:18",
- "19:22:42",
- "23:19:00"
- ]
- }
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": {
- "dates": [
- "2012-07-13",
- "2016-12-24",
- "2017-08-31"
- ],
- "times": [
- "21:43:57",
- "02:27:31",
- "00:35:26"
- ]
- }
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": {
- "dates": [
- "2013-04-13",
- "2013-08-19",
- "2013-10-04"
- ],
- "times": [
- "12:35:33",
- "23:35:49",
- "19:14:56"
- ]
- }
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": {
- "dates": [
- "2016-09-10",
- "2018-09-08",
- "2019-09-13"
- ],
- "times": [
- "19:26:19",
- "14:15:37",
- "22:47:25"
- ]
- }
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": {
- "dates": [
- "2017-05-19",
- "2017-05-19",
- "2017-08-28",
- "2017-09-20",
- "2017-10-01",
- "2017-10-01",
- "2017-12-27"
- ],
- "times": [
- "14:30:16",
- "14:30:25",
- "15:49:37",
- "20:19:51",
- "16:31:05",
- "16:56:27",
- "23:33:20"
- ]
- }
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": {
- "dates": [
- "2019-06-05"
- ],
- "times": [
- "18:22:49"
- ]
- }
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": {
- "dates": [
- "2011-09-24",
- "2014-03-10",
- "2015-05-27",
- "2015-08-29",
- "2018-03-16"
- ],
- "times": [
- "21:37:32",
- "20:20:07",
- "00:40:24",
- "17:58:15",
- "15:03:26"
- ]
- }
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": {
- "dates": [
- "2015-03-16",
- "2015-12-21",
- "2016-10-28",
- "2016-10-28"
- ],
- "times": [
- "23:51:16",
- "04:48:01",
- "20:22:42",
- "20:23:00"
- ]
- }
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": {
- "dates": [
- "2013-10-22",
- "2014-11-21"
- ],
- "times": [
- "16:49:21",
- "17:39:24"
- ]
- }
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": {
- "dates": [
- "2014-08-07",
- "2014-09-16",
- "2014-10-12",
- "2015-07-21",
- "2015-07-21"
- ],
- "times": [
- "18:30:48",
- "20:41:45",
- "23:22:27",
- "20:43:56",
- "20:45:07"
- ]
- }
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": {
- "dates": [
- "2015-05-02",
- "2015-05-06",
- "2015-09-26"
- ],
- "times": [
- "19:49:05",
- "03:52:18",
- "01:13:19"
- ]
- }
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": {
- "dates": [
- "2015-04-11",
- "2015-11-21",
- "2016-05-06",
- "2017-08-09",
- "2017-10-21"
- ],
- "times": [
- "13:14:14",
- "16:05:56",
- "14:10:04",
- "15:15:10",
- "15:12:56"
- ]
- }
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": {
- "dates": [
- "2015-12-03",
- "2016-03-17",
- "2016-11-02"
- ],
- "times": [
- "18:44:00",
- "18:19:21",
- "15:58:38"
- ]
- }
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": {
- "dates": [
- "2019-04-04"
- ],
- "times": [
- "22:02:37"
- ]
- }
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": {
- "dates": [
- "2019-02-27"
- ],
- "times": [
- "14:03:08"
- ]
- }
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": {
- "dates": [
- "2011-12-21",
- "2012-04-15",
- "2012-04-15",
- "2013-06-30",
- "2013-10-04",
- "2014-07-16"
- ],
- "times": [
- "19:02:51",
- "04:21:39",
- "14:23:56",
- "22:39:51",
- "20:34:13",
- "02:28:40"
- ]
- }
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": {
- "dates": [
- "2018-06-13"
- ],
- "times": [
- "20:16:07"
- ]
- }
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": {
- "dates": [
- "2015-05-29",
- "2015-06-01"
- ],
- "times": [
- "16:46:17",
- "15:03:53"
- ]
- }
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": {
- "dates": [
- "2011-10-08",
- "2014-08-18",
- "2016-01-07",
- "2016-10-21",
- "2016-12-01",
- "2016-12-29",
- "2018-07-22",
- "2018-09-07",
- "2019-03-08"
- ],
- "times": [
- "12:02:23",
- "02:11:11",
- "05:27:51",
- "20:15:55",
- "03:57:10",
- "01:54:42",
- "19:55:31",
- "01:42:54",
- "03:41:06"
- ]
- }
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-2.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp
index 0ab72fe..caaaad0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp
index 02ddf21..f05a19b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {"date": "2016-04-26", "time": "19:49:16"},
- {"date": "2016-08-30", "time": "18:36:57"},
- {"date": "2016-10-15", "time": "02:45:18"},
- {"date": "2016-11-18", "time": "01:54:50"},
- {"date": "2017-04-20", "time": "18:39:06"},
- {"date": "2017-05-03", "time": "17:58:02"},
- {"date": "2019-03-19", "time": "22:04:48"}
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {"date": "2018-05-25", "time": "19:52:07"},
- {"date": "2018-09-18", "time": "16:09:44"},
- {"date": "2019-10-18", "time": "21:29:09"}
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {"date": "2019-06-07", "time": "17:54:58"}
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {"date": "2011-05-03", "time": "20:54:05"},
- {"date": "2011-08-23", "time": "20:49:45"},
- {"date": "2014-12-04", "time": "06:13:01"},
- {"date": "2016-11-16", "time": "19:25:55"}
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {"date": "2016-06-18", "time": "21:35:45"},
- {"date": "2016-10-15", "time": "18:17:51"}
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {"date": "2015-04-02", "time": "21:45:17"}
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {"date": "2014-08-27", "time": "17:49:18"},
- {"date": "2015-12-19", "time": "21:30:31"},
- {"date": "2018-11-27", "time": "15:53:50"}
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {"date": "2014-01-28", "time": "20:56:04"},
- {"date": "2014-11-16", "time": "16:11:58"},
- {"date": "2015-11-15", "time": "19:21:53"},
- {"date": "2015-11-15", "time": "19:33:39"}
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {"date": "2019-04-11", "time": "18:30:12"}
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {"date": "2015-06-06", "time": "20:01:06"},
- {"date": "2019-03-14", "time": "22:01:52"}
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {"date": "2018-09-29", "time": "18:55:17"},
- {"date": "2018-10-20", "time": "16:48:05"},
- {"date": "2018-10-20", "time": "22:20:24"}
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {"date": "2011-04-23", "time": "21:11:22"},
- {"date": "2014-05-04", "time": "19:42:48"},
- {"date": "2014-05-11", "time": "19:16:08"},
- {"date": "2014-06-04", "time": "19:14:18"},
- {"date": "2015-12-05", "time": "19:22:42"},
- {"date": "2017-05-15", "time": "23:19:00"}
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {"date": "2012-07-13", "time": "21:43:57"},
- {"date": "2016-12-24", "time": "02:27:31"},
- {"date": "2017-08-31", "time": "00:35:26"}
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {"date": "2013-04-13", "time": "12:35:33"},
- {"date": "2013-08-19", "time": "23:35:49"},
- {"date": "2013-10-04", "time": "19:14:56"}
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {"date": "2016-09-10", "time": "19:26:19"},
- {"date": "2018-09-08", "time": "14:15:37"},
- {"date": "2019-09-13", "time": "22:47:25"}
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {"date": "2017-05-19", "time": "14:30:16"},
- {"date": "2017-05-19", "time": "14:30:25"},
- {"date": "2017-08-28", "time": "15:49:37"},
- {"date": "2017-09-20", "time": "20:19:51"},
- {"date": "2017-10-01", "time": "16:31:05"},
- {"date": "2017-10-01", "time": "16:56:27"},
- {"date": "2017-12-27", "time": "23:33:20"}
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {"date": "2019-06-05", "time": "18:22:49"}
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {"date": "2011-09-24", "time": "21:37:32"},
- {"date": "2014-03-10", "time": "20:20:07"},
- {"date": "2015-05-27", "time": "00:40:24"},
- {"date": "2015-08-29", "time": "17:58:15"},
- {"date": "2018-03-16", "time": "15:03:26"}
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {"date": "2015-03-16", "time": "23:51:16"},
- {"date": "2015-12-21", "time": "04:48:01"},
- {"date": "2016-10-28", "time": "20:22:42"},
- {"date": "2016-10-28", "time": "20:23:00"}
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {"date": "2013-10-22", "time": "16:49:21"},
- {"date": "2014-11-21", "time": "17:39:24"}
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {"date": "2014-08-07", "time": "18:30:48"},
- {"date": "2014-09-16", "time": "20:41:45"},
- {"date": "2014-10-12", "time": "23:22:27"},
- {"date": "2015-07-21", "time": "20:43:56"},
- {"date": "2015-07-21", "time": "20:45:07"}
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {"date": "2015-05-02", "time": "19:49:05"},
- {"date": "2015-05-06", "time": "03:52:18"},
- {"date": "2015-09-26", "time": "01:13:19"}
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {"date": "2015-04-11", "time": "13:14:14"},
- {"date": "2015-11-21", "time": "16:05:56"},
- {"date": "2016-05-06", "time": "14:10:04"},
- {"date": "2017-08-09", "time": "15:15:10"},
- {"date": "2017-10-21", "time": "15:12:56"}
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {"date": "2015-12-03", "time": "18:44:00"},
- {"date": "2016-03-17", "time": "18:19:21"},
- {"date": "2016-11-02", "time": "15:58:38"}
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {"date": "2019-04-04", "time": "22:02:37"}
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {"date": "2019-02-27", "time": "14:03:08"}
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {"date": "2011-12-21", "time": "19:02:51"},
- {"date": "2012-04-15", "time": "04:21:39"},
- {"date": "2012-04-15", "time": "14:23:56"},
- {"date": "2013-06-30", "time": "22:39:51"},
- {"date": "2013-10-04", "time": "20:34:13"},
- {"date": "2014-07-16", "time": "02:28:40"}
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {"date": "2018-06-13", "time": "20:16:07"}
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {"date": "2015-05-29", "time": "16:46:17"},
- {"date": "2015-06-01", "time": "15:03:53"}
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {"date": "2011-10-08", "time": "12:02:23"},
- {"date": "2014-08-18", "time": "02:11:11"},
- {"date": "2016-01-07", "time": "05:27:51"},
- {"date": "2016-10-21", "time": "20:15:55"},
- {"date": "2016-12-01", "time": "03:57:10"},
- {"date": "2016-12-29", "time": "01:54:42"},
- {"date": "2018-07-22", "time": "19:55:31"},
- {"date": "2018-09-07", "time": "01:42:54"},
- {"date": "2019-03-08", "time": "03:41:06"}
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-3.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp
index ee911b4..4154d37 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp
index 7fcf9e4..197d417 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp
@@ -19,407 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {
- "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
- "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
- },
- {
- "dates": ["2017-04-20", "2017-05-03"],
- "times": ["18:39:06", "17:58:02"]
- },
- {
- "dates": ["2019-03-19"],
- "times": ["22:04:48"]
- }
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {
- "dates": ["2018-05-25", "2018-09-18"],
- "times": ["19:52:07", "16:09:44"]
- },
- {
- "dates": ["2019-10-18"],
- "times": ["21:29:09"]
- }
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {
- "dates": ["2019-06-07"],
- "times": ["17:54:58"]
- }
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {
- "dates": ["2011-05-03", "2011-08-23"],
- "times": ["20:54:05", "20:49:45"]
- },
- {
- "dates": ["2014-12-04"],
- "times": ["06:13:01"]
- },
- {
- "dates": ["2016-11-16"],
- "times": ["19:25:55"]
- }
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {
- "dates": ["2016-06-18", "2016-10-15"],
- "times": ["21:35:45", "18:17:51"]
- }
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {
- "dates": ["2015-04-02"],
- "times": ["21:45:17"]
- }
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {
- "dates": ["2014-08-27"],
- "times": ["17:49:18"]
- },
- {
- "dates": ["2015-12-19"],
- "times": ["21:30:31"]
- },
- {
- "dates": ["2018-11-27"],
- "times": ["15:53:50"]
- }
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {
- "dates": ["2014-01-28", "2014-11-16"],
- "times": ["20:56:04", "16:11:58"]
- },
- {
- "dates": ["2015-11-15", "2015-11-15"],
- "times": ["19:21:53", "19:33:39"]
- }
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {
- "dates": ["2019-04-11"],
- "times": ["18:30:12"]
- }
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {
- "dates": ["2015-06-06"],
- "times": ["20:01:06"]
- },
- {
- "dates": ["2019-03-14"],
- "times": ["22:01:52"]
- }
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {
- "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
- "times": ["18:55:17", "16:48:05", "22:20:24"]
- }
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {
- "dates": ["2011-04-23"],
- "times": ["21:11:22"]
- },
- {
- "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
- "times": ["19:42:48", "19:16:08", "19:14:18"]
- },
- {
- "dates": ["2015-12-05"],
- "times": ["19:22:42"]
- },
- {
- "dates": ["2017-05-15"],
- "times": ["23:19:00"]
- }
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {
- "dates": ["2012-07-13"],
- "times": ["21:43:57"]
- },
- {
- "dates": ["2016-12-24"],
- "times": ["02:27:31"]
- },
- {
- "dates": ["2017-08-31"],
- "times": ["00:35:26"]
- }
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {
- "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
- "times": ["12:35:33", "23:35:49", "19:14:56"]
- }
-
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {
- "dates": ["2016-09-10"],
- "times": ["19:26:19"]
- },
- {
- "dates": ["2018-09-08"],
- "times": ["14:15:37"]
- },
- {
- "dates": ["2019-09-13"],
- "times": ["22:47:25"]
- }
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {
- "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
- "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
- }
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {
- "dates": ["2019-06-05"],
- "times": ["18:22:49"]
- }
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {
- "dates": ["2011-09-24"],
- "times": ["21:37:32"]
- },
- {
- "dates": ["2014-03-10"],
- "times": ["20:20:07"]
- },
- {
- "dates": ["2015-05-27", "2015-08-29"],
- "times": ["00:40:24", "17:58:15"]
- },
- {
- "dates": ["2018-03-16"],
- "times": ["15:03:26"]
- }
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {
- "dates": ["2015-03-16", "2015-12-21"],
- "times": ["23:51:16", "04:48:01"]
- },
- {
- "dates": ["2016-10-28", "2016-10-28"],
- "times": ["20:22:42", "20:23:00"]
- }
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {
- "dates": ["2013-10-22"],
- "times": ["16:49:21"]
- },
- {
- "dates": ["2014-11-21"],
- "times": ["17:39:24"]
- }
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {
- "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
- "times": ["18:30:48", "20:41:45", "23:22:27"]
- },
- {
- "dates": ["2015-07-21", "2015-07-21"],
- "times": ["20:43:56", "20:45:07"]
- }
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {
- "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
- "times": ["19:49:05", "03:52:18", "01:13:19"]
- }
-
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {
- "dates": ["2015-04-11", "2015-11-21"],
- "times": ["13:14:14", "16:05:56"]
- },
- {
- "dates": ["2016-05-06"],
- "times": ["14:10:04"]
- },
- {
- "dates": ["2017-08-09", "2017-10-21"],
- "times": ["15:15:10", "15:12:56"]
- }
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {
- "dates": ["2015-12-03"],
- "times": ["18:44:00"]
- },
- {
- "dates": ["2016-03-17", "2016-11-02"],
- "times": ["18:19:21", "15:58:38"]
- }
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {
- "dates": ["2019-04-04"],
- "times": ["22:02:37"]
- }
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {
- "dates": ["2019-02-27"],
- "times": ["14:03:08"]
- }
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {
- "dates": ["2011-12-21"],
- "times": ["19:02:51"]
- },
- {
- "dates": ["2012-04-15", "2012-04-15"],
- "times": ["04:21:39", "14:23:56"]
- },
- {
- "dates": ["2013-06-30", "2013-10-04"],
- "times": ["22:39:51", "20:34:13"]
- },
- {
- "dates": ["2014-07-16"],
- "times": ["02:28:40"]
- }
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {
- "dates": ["2018-06-13"],
- "times": ["20:16:07"]
- }
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {
- "dates": ["2015-05-29", "2015-06-01"],
- "times": ["16:46:17", "15:03:53"]
- }
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {
- "dates": ["2011-10-08"],
- "times": ["12:02:23"]
- },
- {
- "dates": ["2014-08-18"],
- "times": ["02:11:11"]
- },
- {
- "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
- "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
- },
- {
- "dates": ["2018-07-22", "2018-09-07"],
- "times": ["19:55:31", "01:42:54"]
- },
- {
- "dates": ["2019-03-08"],
- "times": ["03:41:06"]
- }
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-4.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp
index 05a7c76..e59799e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
index 884c224..8c7303c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -19,599 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2016-04-26",
- "2016-08-30",
- "2016-10-15",
- "2016-11-18",
- "2017-04-20",
- "2017-05-03",
- "2019-03-19"
- ],
- "times": [
- "19:49:16",
- "18:36:57",
- "02:45:18",
- "01:54:50",
- "18:39:06",
- "17:58:02",
- "22:04:48"
- ]
- }
- }
- }
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2018-05-25",
- "2018-09-18",
- "2019-10-18"
- ],
- "times": [
- "19:52:07",
- "16:09:44",
- "21:29:09"
- ]
- }
- }
- }
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-06-07"
- ],
- "times": [
- "17:54:58"
- ]
- }
- }
- }
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-05-03",
- "2011-08-23",
- "2014-12-04",
- "2016-11-16"
- ],
- "times": [
- "20:54:05",
- "20:49:45",
- "06:13:01",
- "19:25:55"
- ]
- }
- }
- }
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2016-06-18",
- "2016-10-15"
- ],
- "times": [
- "21:35:45",
- "18:17:51"
- ]
- }
- }
- }
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-04-02"
- ],
- "times": [
- "21:45:17"
- ]
- }
- }
- }
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2014-08-27",
- "2015-12-19",
- "2018-11-27"
- ],
- "times": [
- "17:49:18",
- "21:30:31",
- "15:53:50"
- ]
- }
- }
- }
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2014-01-28",
- "2014-11-16",
- "2015-11-15",
- "2015-11-15"
- ],
- "times": [
- "20:56:04",
- "16:11:58",
- "19:21:53",
- "19:33:39"
- ]
- }
- }
- }
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-04-11"
- ],
- "times": [
- "18:30:12"
- ]
- }
- }
- }
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-06-06",
- "2019-03-14"
- ],
- "times": [
- "20:01:06",
- "22:01:52"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2018-09-29",
- "2018-10-20",
- "2018-10-20"
- ],
- "times": [
- "18:55:17",
- "16:48:05",
- "22:20:24"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-04-23",
- "2014-05-04",
- "2014-05-11",
- "2014-06-04",
- "2015-12-05",
- "2017-05-15"
- ],
- "times": [
- "21:11:22",
- "19:42:48",
- "19:16:08",
- "19:14:18",
- "19:22:42",
- "23:19:00"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2012-07-13",
- "2016-12-24",
- "2017-08-31"
- ],
- "times": [
- "21:43:57",
- "02:27:31",
- "00:35:26"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2013-04-13",
- "2013-08-19",
- "2013-10-04"
- ],
- "times": [
- "12:35:33",
- "23:35:49",
- "19:14:56"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2016-09-10",
- "2018-09-08",
- "2019-09-13"
- ],
- "times": [
- "19:26:19",
- "14:15:37",
- "22:47:25"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2017-05-19",
- "2017-05-19",
- "2017-08-28",
- "2017-09-20",
- "2017-10-01",
- "2017-10-01",
- "2017-12-27"
- ],
- "times": [
- "14:30:16",
- "14:30:25",
- "15:49:37",
- "20:19:51",
- "16:31:05",
- "16:56:27",
- "23:33:20"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-06-05"
- ],
- "times": [
- "18:22:49"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-09-24",
- "2014-03-10",
- "2015-05-27",
- "2015-08-29",
- "2018-03-16"
- ],
- "times": [
- "21:37:32",
- "20:20:07",
- "00:40:24",
- "17:58:15",
- "15:03:26"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-03-16",
- "2015-12-21",
- "2016-10-28",
- "2016-10-28"
- ],
- "times": [
- "23:51:16",
- "04:48:01",
- "20:22:42",
- "20:23:00"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2013-10-22",
- "2014-11-21"
- ],
- "times": [
- "16:49:21",
- "17:39:24"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2014-08-07",
- "2014-09-16",
- "2014-10-12",
- "2015-07-21",
- "2015-07-21"
- ],
- "times": [
- "18:30:48",
- "20:41:45",
- "23:22:27",
- "20:43:56",
- "20:45:07"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-05-02",
- "2015-05-06",
- "2015-09-26"
- ],
- "times": [
- "19:49:05",
- "03:52:18",
- "01:13:19"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-04-11",
- "2015-11-21",
- "2016-05-06",
- "2017-08-09",
- "2017-10-21"
- ],
- "times": [
- "13:14:14",
- "16:05:56",
- "14:10:04",
- "15:15:10",
- "15:12:56"
- ]
- }
- }
- }
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-12-03",
- "2016-03-17",
- "2016-11-02"
- ],
- "times": [
- "18:44:00",
- "18:19:21",
- "15:58:38"
- ]
- }
- }
- }
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-04-04"
- ],
- "times": [
- "22:02:37"
- ]
- }
- }
- }
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-02-27"
- ],
- "times": [
- "14:03:08"
- ]
- }
- }
- }
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-12-21",
- "2012-04-15",
- "2012-04-15",
- "2013-06-30",
- "2013-10-04",
- "2014-07-16"
- ],
- "times": [
- "19:02:51",
- "04:21:39",
- "14:23:56",
- "22:39:51",
- "20:34:13",
- "02:28:40"
- ]
- }
- }
- }
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2018-06-13"
- ],
- "times": [
- "20:16:07"
- ]
- }
- }
- }
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-05-29",
- "2015-06-01"
- ],
- "times": [
- "16:46:17",
- "15:03:53"
- ]
- }
- }
- }
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-10-08",
- "2014-08-18",
- "2016-01-07",
- "2016-10-21",
- "2016-12-01",
- "2016-12-29",
- "2018-07-22",
- "2018-09-07",
- "2019-03-08"
- ],
- "times": [
- "12:02:23",
- "02:11:11",
- "05:27:51",
- "20:15:55",
- "03:57:10",
- "01:54:42",
- "19:55:31",
- "01:42:54",
- "03:41:06"
- ]
- }
- }
- }
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-3-level-record-path.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
index 5ef37bc..01be97a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -19,4 +19,6 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates)
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
index 6498c14..3594aee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -19,288 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "checkin_id": 1,
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "dates": [
- "2016-04-26 19:49:16",
- "2016-08-30 18:36:57",
- "2016-10-15 02:45:18",
- "2016-11-18 01:54:50",
- "2017-04-20 18:39:06",
- "2017-05-03 17:58:02",
- "2019-03-19 22:04:48"
- ]
- },
- {
- "checkin_id": 2,
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "dates": [
- "2018-05-25 19:52:07",
- "2018-09-18 16:09:44",
- "2019-10-18 21:29:09"
- ]
- },
- {
- "checkin_id": 3,
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "dates": [
- "2019-06-07 17:54:58"
- ]
- },
- {
- "checkin_id": 4,
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "dates": [
- "2011-05-03 20:54:05",
- "2011-08-23 20:49:45",
- "2014-12-04 06:13:01",
- "2016-11-16 19:25:55"
- ]
- },
- {
- "checkin_id": 5,
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "dates": [
- "2016-06-18 21:35:45",
- "2016-10-15 18:17:51"
- ]
- },
- {
- "checkin_id": 6,
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "dates": [
- "2015-04-02 21:45:17"
- ]
- },
- {
- "checkin_id": 7,
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "dates": [
- "2014-08-27 17:49:18",
- "2015-12-19 21:30:31",
- "2018-11-27 15:53:50"
- ]
- },
- {
- "checkin_id": 8,
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "dates": [
- "2014-01-28 20:56:04",
- "2014-11-16 16:11:58",
- "2015-11-15 19:21:53",
- "2015-11-15 19:33:39"
- ]
- },
- {
- "checkin_id": 9,
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "dates": [
- "2019-04-11 18:30:12"
- ]
- },
- {
- "checkin_id": 10,
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "dates": [
- "2015-06-06 20:01:06",
- "2019-03-14 22:01:52"
- ]
- },
- {
- "checkin_id": 11,
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "dates": [
- "2018-09-29 18:55:17",
- "2018-10-20 16:48:05",
- "2018-10-20 22:20:24"
- ]
- },
- {
- "checkin_id": 12,
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "dates": [
- "2011-04-23 21:11:22",
- "2014-05-04 19:42:48",
- "2014-05-11 19:16:08",
- "2014-06-04 19:14:18",
- "2015-12-05 19:22:42",
- "2017-05-15 23:19:00"
- ]
- },
- {
- "checkin_id": 13,
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "dates": [
- "2012-07-13 21:43:57",
- "2016-12-24 02:27:31",
- "2017-08-31 00:35:26"
- ]
- },
- {
- "checkin_id": 14,
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "dates": [
- "2013-04-13 12:35:33",
- "2013-08-19 23:35:49",
- "2013-10-04 19:14:56"
- ]
- },
- {
- "checkin_id": 15,
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "dates": [
- "2016-09-10 19:26:19",
- "2018-09-08 14:15:37",
- "2019-09-13 22:47:25"
- ]
- },
- {
- "checkin_id": 16,
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "dates": [
- "2017-05-19 14:30:16",
- "2017-05-19 14:30:25",
- "2017-08-28 15:49:37",
- "2017-09-20 20:19:51",
- "2017-10-01 16:31:05",
- "2017-10-01 16:56:27",
- "2017-12-27 23:33:20"
- ]
- },
- {
- "checkin_id": 17,
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "dates": [
- "2019-06-05 18:22:49"
- ]
- },
- {
- "checkin_id": 18,
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "dates": [
- "2011-09-24 21:37:32",
- "2014-03-10 20:20:07",
- "2015-05-27 00:40:24",
- "2015-08-29 17:58:15",
- "2018-03-16 15:03:26"
- ]
- },
- {
- "checkin_id": 19,
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "dates": [
- "2015-03-16 23:51:16",
- "2015-12-21 04:48:01",
- "2016-10-28 20:22:42",
- "2016-10-28 20:23:00"
- ]
- },
- {
- "checkin_id": 20,
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "dates": [
- "2013-10-22 16:49:21",
- "2014-11-21 17:39:24"
- ]
- },
- {
- "checkin_id": 21,
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "dates": [
- "2014-08-07 18:30:48",
- "2014-09-16 20:41:45",
- "2014-10-12 23:22:27",
- "2015-07-21 20:43:56",
- "2015-07-21 20:45:07"
- ]
- },
- {
- "checkin_id": 22,
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "dates": [
- "2015-05-02 19:49:05",
- "2015-05-06 03:52:18",
- "2015-09-26 01:13:19"
- ]
- },
- {
- "checkin_id": 23,
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "dates": [
- "2015-04-11 13:14:14",
- "2015-11-21 16:05:56",
- "2016-05-06 14:10:04",
- "2017-08-09 15:15:10",
- "2017-10-21 15:12:56"
- ]
- },
- {
- "checkin_id": 24,
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "dates": [
- "2015-12-03 18:44:00",
- "2016-03-17 18:19:21",
- "2016-11-02 15:58:38"
- ]
- },
- {
- "checkin_id": 25,
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "dates": [
- "2019-04-04 22:02:37"
- ]
- },
- {
- "checkin_id": 26,
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "dates": [
- "2019-02-27 14:03:08"
- ]
- },
- {
- "checkin_id": 27,
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "dates": [
- "2011-12-21 19:02:51",
- "2012-04-15 04:21:39",
- "2012-04-15 14:23:56",
- "2013-06-30 22:39:51",
- "2013-10-04 20:34:13",
- "2014-07-16 02:28:40"
- ]
- },
- {
- "checkin_id": 28,
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "dates": [
- "2018-06-13 20:16:07"
- ]
- },
- {
- "checkin_id": 29,
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "dates": [
- "2015-05-29 16:46:17",
- "2015-06-01 15:03:53"
- ]
- },
- {
- "checkin_id": 30,
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "dates": [
- "2011-10-08 12:02:23",
- "2014-08-18 02:11:11",
- "2016-01-07 05:27:51",
- "2016-10-21 20:15:55",
- "2016-12-01 03:57:10",
- "2016-12-29 01:54:42",
- "2018-07-22 19:55:31",
- "2018-09-07 01:42:54",
- "2019-03-08 03:41:06"
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-composite-pk.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
index 02ddf21..cb5c748 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {"date": "2016-04-26", "time": "19:49:16"},
- {"date": "2016-08-30", "time": "18:36:57"},
- {"date": "2016-10-15", "time": "02:45:18"},
- {"date": "2016-11-18", "time": "01:54:50"},
- {"date": "2017-04-20", "time": "18:39:06"},
- {"date": "2017-05-03", "time": "17:58:02"},
- {"date": "2019-03-19", "time": "22:04:48"}
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {"date": "2018-05-25", "time": "19:52:07"},
- {"date": "2018-09-18", "time": "16:09:44"},
- {"date": "2019-10-18", "time": "21:29:09"}
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {"date": "2019-06-07", "time": "17:54:58"}
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {"date": "2011-05-03", "time": "20:54:05"},
- {"date": "2011-08-23", "time": "20:49:45"},
- {"date": "2014-12-04", "time": "06:13:01"},
- {"date": "2016-11-16", "time": "19:25:55"}
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {"date": "2016-06-18", "time": "21:35:45"},
- {"date": "2016-10-15", "time": "18:17:51"}
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {"date": "2015-04-02", "time": "21:45:17"}
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {"date": "2014-08-27", "time": "17:49:18"},
- {"date": "2015-12-19", "time": "21:30:31"},
- {"date": "2018-11-27", "time": "15:53:50"}
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {"date": "2014-01-28", "time": "20:56:04"},
- {"date": "2014-11-16", "time": "16:11:58"},
- {"date": "2015-11-15", "time": "19:21:53"},
- {"date": "2015-11-15", "time": "19:33:39"}
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {"date": "2019-04-11", "time": "18:30:12"}
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {"date": "2015-06-06", "time": "20:01:06"},
- {"date": "2019-03-14", "time": "22:01:52"}
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {"date": "2018-09-29", "time": "18:55:17"},
- {"date": "2018-10-20", "time": "16:48:05"},
- {"date": "2018-10-20", "time": "22:20:24"}
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {"date": "2011-04-23", "time": "21:11:22"},
- {"date": "2014-05-04", "time": "19:42:48"},
- {"date": "2014-05-11", "time": "19:16:08"},
- {"date": "2014-06-04", "time": "19:14:18"},
- {"date": "2015-12-05", "time": "19:22:42"},
- {"date": "2017-05-15", "time": "23:19:00"}
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {"date": "2012-07-13", "time": "21:43:57"},
- {"date": "2016-12-24", "time": "02:27:31"},
- {"date": "2017-08-31", "time": "00:35:26"}
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {"date": "2013-04-13", "time": "12:35:33"},
- {"date": "2013-08-19", "time": "23:35:49"},
- {"date": "2013-10-04", "time": "19:14:56"}
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {"date": "2016-09-10", "time": "19:26:19"},
- {"date": "2018-09-08", "time": "14:15:37"},
- {"date": "2019-09-13", "time": "22:47:25"}
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {"date": "2017-05-19", "time": "14:30:16"},
- {"date": "2017-05-19", "time": "14:30:25"},
- {"date": "2017-08-28", "time": "15:49:37"},
- {"date": "2017-09-20", "time": "20:19:51"},
- {"date": "2017-10-01", "time": "16:31:05"},
- {"date": "2017-10-01", "time": "16:56:27"},
- {"date": "2017-12-27", "time": "23:33:20"}
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {"date": "2019-06-05", "time": "18:22:49"}
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {"date": "2011-09-24", "time": "21:37:32"},
- {"date": "2014-03-10", "time": "20:20:07"},
- {"date": "2015-05-27", "time": "00:40:24"},
- {"date": "2015-08-29", "time": "17:58:15"},
- {"date": "2018-03-16", "time": "15:03:26"}
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {"date": "2015-03-16", "time": "23:51:16"},
- {"date": "2015-12-21", "time": "04:48:01"},
- {"date": "2016-10-28", "time": "20:22:42"},
- {"date": "2016-10-28", "time": "20:23:00"}
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {"date": "2013-10-22", "time": "16:49:21"},
- {"date": "2014-11-21", "time": "17:39:24"}
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {"date": "2014-08-07", "time": "18:30:48"},
- {"date": "2014-09-16", "time": "20:41:45"},
- {"date": "2014-10-12", "time": "23:22:27"},
- {"date": "2015-07-21", "time": "20:43:56"},
- {"date": "2015-07-21", "time": "20:45:07"}
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {"date": "2015-05-02", "time": "19:49:05"},
- {"date": "2015-05-06", "time": "03:52:18"},
- {"date": "2015-09-26", "time": "01:13:19"}
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {"date": "2015-04-11", "time": "13:14:14"},
- {"date": "2015-11-21", "time": "16:05:56"},
- {"date": "2016-05-06", "time": "14:10:04"},
- {"date": "2017-08-09", "time": "15:15:10"},
- {"date": "2017-10-21", "time": "15:12:56"}
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {"date": "2015-12-03", "time": "18:44:00"},
- {"date": "2016-03-17", "time": "18:19:21"},
- {"date": "2016-11-02", "time": "15:58:38"}
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {"date": "2019-04-04", "time": "22:02:37"}
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {"date": "2019-02-27", "time": "14:03:08"}
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {"date": "2011-12-21", "time": "19:02:51"},
- {"date": "2012-04-15", "time": "04:21:39"},
- {"date": "2012-04-15", "time": "14:23:56"},
- {"date": "2013-06-30", "time": "22:39:51"},
- {"date": "2013-10-04", "time": "20:34:13"},
- {"date": "2014-07-16", "time": "02:28:40"}
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {"date": "2018-06-13", "time": "20:16:07"}
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {"date": "2015-05-29", "time": "16:46:17"},
- {"date": "2015-06-01", "time": "15:03:53"}
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {"date": "2011-10-08", "time": "12:02:23"},
- {"date": "2014-08-18", "time": "02:11:11"},
- {"date": "2016-01-07", "time": "05:27:51"},
- {"date": "2016-10-21", "time": "20:15:55"},
- {"date": "2016-12-01", "time": "03:57:10"},
- {"date": "2016-12-29", "time": "01:54:42"},
- {"date": "2018-07-22", "time": "19:55:31"},
- {"date": "2018-09-07", "time": "01:42:54"},
- {"date": "2019-03-08", "time": "03:41:06"}
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-3.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp
index 20de338..2c9136a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp
@@ -19,4 +19,6 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness
+ON YelpCheckin (UNNEST checkin_times SELECT date, time)
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
index a0fc1bf..6fa8e96 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp
index b75bf1b..39fd911 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "dates": [
- "2016-04-26 19:49:16",
- "2016-08-30 18:36:57",
- "2016-10-15 02:45:18",
- "2016-11-18 01:54:50",
- "2017-04-20 18:39:06",
- "2017-05-03 17:58:02",
- "2019-03-19 22:04:48"
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "dates": [
- "2018-05-25 19:52:07",
- "2018-09-18 16:09:44",
- "2019-10-18 21:29:09"
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "dates": [
- "2019-06-07 17:54:58"
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "dates": [
- "2011-05-03 20:54:05",
- "2011-08-23 20:49:45",
- "2014-12-04 06:13:01",
- "2016-11-16 19:25:55"
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "dates": [
- "2016-06-18 21:35:45",
- "2016-10-15 18:17:51"
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "dates": [
- "2015-04-02 21:45:17"
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "dates": [
- "2014-08-27 17:49:18",
- "2015-12-19 21:30:31",
- "2018-11-27 15:53:50"
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "dates": [
- "2014-01-28 20:56:04",
- "2014-11-16 16:11:58",
- "2015-11-15 19:21:53",
- "2015-11-15 19:33:39"
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "dates": [
- "2019-04-11 18:30:12"
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "dates": [
- "2015-06-06 20:01:06",
- "2019-03-14 22:01:52"
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "dates": [
- "2018-09-29 18:55:17",
- "2018-10-20 16:48:05",
- "2018-10-20 22:20:24"
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "dates": [
- "2011-04-23 21:11:22",
- "2014-05-04 19:42:48",
- "2014-05-11 19:16:08",
- "2014-06-04 19:14:18",
- "2015-12-05 19:22:42",
- "2017-05-15 23:19:00"
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "dates": [
- "2012-07-13 21:43:57",
- "2016-12-24 02:27:31",
- "2017-08-31 00:35:26"
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "dates": [
- "2013-04-13 12:35:33",
- "2013-08-19 23:35:49",
- "2013-10-04 19:14:56"
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "dates": [
- "2016-09-10 19:26:19",
- "2018-09-08 14:15:37",
- "2019-09-13 22:47:25"
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "dates": [
- "2017-05-19 14:30:16",
- "2017-05-19 14:30:25",
- "2017-08-28 15:49:37",
- "2017-09-20 20:19:51",
- "2017-10-01 16:31:05",
- "2017-10-01 16:56:27",
- "2017-12-27 23:33:20"
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "dates": [
- "2019-06-05 18:22:49"
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "dates": [
- "2011-09-24 21:37:32",
- "2014-03-10 20:20:07",
- "2015-05-27 00:40:24",
- "2015-08-29 17:58:15",
- "2018-03-16 15:03:26"
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "dates": [
- "2015-03-16 23:51:16",
- "2015-12-21 04:48:01",
- "2016-10-28 20:22:42",
- "2016-10-28 20:23:00"
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "dates": [
- "2013-10-22 16:49:21",
- "2014-11-21 17:39:24"
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "dates": [
- "2014-08-07 18:30:48",
- "2014-09-16 20:41:45",
- "2014-10-12 23:22:27",
- "2015-07-21 20:43:56",
- "2015-07-21 20:45:07"
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "dates": [
- "2015-05-02 19:49:05",
- "2015-05-06 03:52:18",
- "2015-09-26 01:13:19"
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "dates": [
- "2015-04-11 13:14:14",
- "2015-11-21 16:05:56",
- "2016-05-06 14:10:04",
- "2017-08-09 15:15:10",
- "2017-10-21 15:12:56"
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "dates": [
- "2015-12-03 18:44:00",
- "2016-03-17 18:19:21",
- "2016-11-02 15:58:38"
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "dates": [
- "2019-04-04 22:02:37"
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "dates": [
- "2019-02-27 14:03:08"
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "dates": [
- "2011-12-21 19:02:51",
- "2012-04-15 04:21:39",
- "2012-04-15 14:23:56",
- "2013-06-30 22:39:51",
- "2013-10-04 20:34:13",
- "2014-07-16 02:28:40"
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "dates": [
- "2018-06-13 20:16:07"
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "dates": [
- "2015-05-29 16:46:17",
- "2015-06-01 15:03:53"
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "dates": [
- "2011-10-08 12:02:23",
- "2014-08-18 02:11:11",
- "2016-01-07 05:27:51",
- "2016-10-21 20:15:55",
- "2016-12-01 03:57:10",
- "2016-12-29 01:54:42",
- "2018-07-22 19:55:31",
- "2018-09-07 01:42:54",
- "2019-03-08 03:41:06"
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp
index 0d37224..5a6513a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp
index 2549e8f..18135a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp
@@ -19,479 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": {
- "dates": [
- "2016-04-26",
- "2016-08-30",
- "2016-10-15",
- "2016-11-18",
- "2017-04-20",
- "2017-05-03",
- "2019-03-19"
- ],
- "times": [
- "19:49:16",
- "18:36:57",
- "02:45:18",
- "01:54:50",
- "18:39:06",
- "17:58:02",
- "22:04:48"
- ]
- }
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": {
- "dates": [
- "2018-05-25",
- "2018-09-18",
- "2019-10-18"
- ],
- "times": [
- "19:52:07",
- "16:09:44",
- "21:29:09"
- ]
- }
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": {
- "dates": [
- "2019-06-07"
- ],
- "times": [
- "17:54:58"
- ]
- }
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": {
- "dates": [
- "2011-05-03",
- "2011-08-23",
- "2014-12-04",
- "2016-11-16"
- ],
- "times": [
- "20:54:05",
- "20:49:45",
- "06:13:01",
- "19:25:55"
- ]
- }
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": {
- "dates": [
- "2016-06-18",
- "2016-10-15"
- ],
- "times": [
- "21:35:45",
- "18:17:51"
- ]
- }
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": {
- "dates": [
- "2015-04-02"
- ],
- "times": [
- "21:45:17"
- ]
- }
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": {
- "dates": [
- "2014-08-27",
- "2015-12-19",
- "2018-11-27"
- ],
- "times": [
- "17:49:18",
- "21:30:31",
- "15:53:50"
- ]
- }
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": {
- "dates": [
- "2014-01-28",
- "2014-11-16",
- "2015-11-15",
- "2015-11-15"
- ],
- "times": [
- "20:56:04",
- "16:11:58",
- "19:21:53",
- "19:33:39"
- ]
- }
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": {
- "dates": [
- "2019-04-11"
- ],
- "times": [
- "18:30:12"
- ]
- }
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": {
- "dates": [
- "2015-06-06",
- "2019-03-14"
- ],
- "times": [
- "20:01:06",
- "22:01:52"
- ]
- }
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": {
- "dates": [
- "2018-09-29",
- "2018-10-20",
- "2018-10-20"
- ],
- "times": [
- "18:55:17",
- "16:48:05",
- "22:20:24"
- ]
- }
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": {
- "dates": [
- "2011-04-23",
- "2014-05-04",
- "2014-05-11",
- "2014-06-04",
- "2015-12-05",
- "2017-05-15"
- ],
- "times": [
- "21:11:22",
- "19:42:48",
- "19:16:08",
- "19:14:18",
- "19:22:42",
- "23:19:00"
- ]
- }
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": {
- "dates": [
- "2012-07-13",
- "2016-12-24",
- "2017-08-31"
- ],
- "times": [
- "21:43:57",
- "02:27:31",
- "00:35:26"
- ]
- }
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": {
- "dates": [
- "2013-04-13",
- "2013-08-19",
- "2013-10-04"
- ],
- "times": [
- "12:35:33",
- "23:35:49",
- "19:14:56"
- ]
- }
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": {
- "dates": [
- "2016-09-10",
- "2018-09-08",
- "2019-09-13"
- ],
- "times": [
- "19:26:19",
- "14:15:37",
- "22:47:25"
- ]
- }
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": {
- "dates": [
- "2017-05-19",
- "2017-05-19",
- "2017-08-28",
- "2017-09-20",
- "2017-10-01",
- "2017-10-01",
- "2017-12-27"
- ],
- "times": [
- "14:30:16",
- "14:30:25",
- "15:49:37",
- "20:19:51",
- "16:31:05",
- "16:56:27",
- "23:33:20"
- ]
- }
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": {
- "dates": [
- "2019-06-05"
- ],
- "times": [
- "18:22:49"
- ]
- }
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": {
- "dates": [
- "2011-09-24",
- "2014-03-10",
- "2015-05-27",
- "2015-08-29",
- "2018-03-16"
- ],
- "times": [
- "21:37:32",
- "20:20:07",
- "00:40:24",
- "17:58:15",
- "15:03:26"
- ]
- }
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": {
- "dates": [
- "2015-03-16",
- "2015-12-21",
- "2016-10-28",
- "2016-10-28"
- ],
- "times": [
- "23:51:16",
- "04:48:01",
- "20:22:42",
- "20:23:00"
- ]
- }
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": {
- "dates": [
- "2013-10-22",
- "2014-11-21"
- ],
- "times": [
- "16:49:21",
- "17:39:24"
- ]
- }
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": {
- "dates": [
- "2014-08-07",
- "2014-09-16",
- "2014-10-12",
- "2015-07-21",
- "2015-07-21"
- ],
- "times": [
- "18:30:48",
- "20:41:45",
- "23:22:27",
- "20:43:56",
- "20:45:07"
- ]
- }
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": {
- "dates": [
- "2015-05-02",
- "2015-05-06",
- "2015-09-26"
- ],
- "times": [
- "19:49:05",
- "03:52:18",
- "01:13:19"
- ]
- }
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": {
- "dates": [
- "2015-04-11",
- "2015-11-21",
- "2016-05-06",
- "2017-08-09",
- "2017-10-21"
- ],
- "times": [
- "13:14:14",
- "16:05:56",
- "14:10:04",
- "15:15:10",
- "15:12:56"
- ]
- }
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": {
- "dates": [
- "2015-12-03",
- "2016-03-17",
- "2016-11-02"
- ],
- "times": [
- "18:44:00",
- "18:19:21",
- "15:58:38"
- ]
- }
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": {
- "dates": [
- "2019-04-04"
- ],
- "times": [
- "22:02:37"
- ]
- }
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": {
- "dates": [
- "2019-02-27"
- ],
- "times": [
- "14:03:08"
- ]
- }
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": {
- "dates": [
- "2011-12-21",
- "2012-04-15",
- "2012-04-15",
- "2013-06-30",
- "2013-10-04",
- "2014-07-16"
- ],
- "times": [
- "19:02:51",
- "04:21:39",
- "14:23:56",
- "22:39:51",
- "20:34:13",
- "02:28:40"
- ]
- }
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": {
- "dates": [
- "2018-06-13"
- ],
- "times": [
- "20:16:07"
- ]
- }
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": {
- "dates": [
- "2015-05-29",
- "2015-06-01"
- ],
- "times": [
- "16:46:17",
- "15:03:53"
- ]
- }
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": {
- "dates": [
- "2011-10-08",
- "2014-08-18",
- "2016-01-07",
- "2016-10-21",
- "2016-12-01",
- "2016-12-29",
- "2018-07-22",
- "2018-09-07",
- "2019-03-08"
- ],
- "times": [
- "12:02:23",
- "02:11:11",
- "05:27:51",
- "20:15:55",
- "03:57:10",
- "01:54:42",
- "19:55:31",
- "01:42:54",
- "03:41:06"
- ]
- }
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-2.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp
index abacfe8..be8f99d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp
index 02ddf21..f05a19b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {"date": "2016-04-26", "time": "19:49:16"},
- {"date": "2016-08-30", "time": "18:36:57"},
- {"date": "2016-10-15", "time": "02:45:18"},
- {"date": "2016-11-18", "time": "01:54:50"},
- {"date": "2017-04-20", "time": "18:39:06"},
- {"date": "2017-05-03", "time": "17:58:02"},
- {"date": "2019-03-19", "time": "22:04:48"}
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {"date": "2018-05-25", "time": "19:52:07"},
- {"date": "2018-09-18", "time": "16:09:44"},
- {"date": "2019-10-18", "time": "21:29:09"}
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {"date": "2019-06-07", "time": "17:54:58"}
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {"date": "2011-05-03", "time": "20:54:05"},
- {"date": "2011-08-23", "time": "20:49:45"},
- {"date": "2014-12-04", "time": "06:13:01"},
- {"date": "2016-11-16", "time": "19:25:55"}
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {"date": "2016-06-18", "time": "21:35:45"},
- {"date": "2016-10-15", "time": "18:17:51"}
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {"date": "2015-04-02", "time": "21:45:17"}
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {"date": "2014-08-27", "time": "17:49:18"},
- {"date": "2015-12-19", "time": "21:30:31"},
- {"date": "2018-11-27", "time": "15:53:50"}
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {"date": "2014-01-28", "time": "20:56:04"},
- {"date": "2014-11-16", "time": "16:11:58"},
- {"date": "2015-11-15", "time": "19:21:53"},
- {"date": "2015-11-15", "time": "19:33:39"}
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {"date": "2019-04-11", "time": "18:30:12"}
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {"date": "2015-06-06", "time": "20:01:06"},
- {"date": "2019-03-14", "time": "22:01:52"}
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {"date": "2018-09-29", "time": "18:55:17"},
- {"date": "2018-10-20", "time": "16:48:05"},
- {"date": "2018-10-20", "time": "22:20:24"}
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {"date": "2011-04-23", "time": "21:11:22"},
- {"date": "2014-05-04", "time": "19:42:48"},
- {"date": "2014-05-11", "time": "19:16:08"},
- {"date": "2014-06-04", "time": "19:14:18"},
- {"date": "2015-12-05", "time": "19:22:42"},
- {"date": "2017-05-15", "time": "23:19:00"}
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {"date": "2012-07-13", "time": "21:43:57"},
- {"date": "2016-12-24", "time": "02:27:31"},
- {"date": "2017-08-31", "time": "00:35:26"}
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {"date": "2013-04-13", "time": "12:35:33"},
- {"date": "2013-08-19", "time": "23:35:49"},
- {"date": "2013-10-04", "time": "19:14:56"}
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {"date": "2016-09-10", "time": "19:26:19"},
- {"date": "2018-09-08", "time": "14:15:37"},
- {"date": "2019-09-13", "time": "22:47:25"}
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {"date": "2017-05-19", "time": "14:30:16"},
- {"date": "2017-05-19", "time": "14:30:25"},
- {"date": "2017-08-28", "time": "15:49:37"},
- {"date": "2017-09-20", "time": "20:19:51"},
- {"date": "2017-10-01", "time": "16:31:05"},
- {"date": "2017-10-01", "time": "16:56:27"},
- {"date": "2017-12-27", "time": "23:33:20"}
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {"date": "2019-06-05", "time": "18:22:49"}
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {"date": "2011-09-24", "time": "21:37:32"},
- {"date": "2014-03-10", "time": "20:20:07"},
- {"date": "2015-05-27", "time": "00:40:24"},
- {"date": "2015-08-29", "time": "17:58:15"},
- {"date": "2018-03-16", "time": "15:03:26"}
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {"date": "2015-03-16", "time": "23:51:16"},
- {"date": "2015-12-21", "time": "04:48:01"},
- {"date": "2016-10-28", "time": "20:22:42"},
- {"date": "2016-10-28", "time": "20:23:00"}
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {"date": "2013-10-22", "time": "16:49:21"},
- {"date": "2014-11-21", "time": "17:39:24"}
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {"date": "2014-08-07", "time": "18:30:48"},
- {"date": "2014-09-16", "time": "20:41:45"},
- {"date": "2014-10-12", "time": "23:22:27"},
- {"date": "2015-07-21", "time": "20:43:56"},
- {"date": "2015-07-21", "time": "20:45:07"}
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {"date": "2015-05-02", "time": "19:49:05"},
- {"date": "2015-05-06", "time": "03:52:18"},
- {"date": "2015-09-26", "time": "01:13:19"}
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {"date": "2015-04-11", "time": "13:14:14"},
- {"date": "2015-11-21", "time": "16:05:56"},
- {"date": "2016-05-06", "time": "14:10:04"},
- {"date": "2017-08-09", "time": "15:15:10"},
- {"date": "2017-10-21", "time": "15:12:56"}
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {"date": "2015-12-03", "time": "18:44:00"},
- {"date": "2016-03-17", "time": "18:19:21"},
- {"date": "2016-11-02", "time": "15:58:38"}
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {"date": "2019-04-04", "time": "22:02:37"}
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {"date": "2019-02-27", "time": "14:03:08"}
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {"date": "2011-12-21", "time": "19:02:51"},
- {"date": "2012-04-15", "time": "04:21:39"},
- {"date": "2012-04-15", "time": "14:23:56"},
- {"date": "2013-06-30", "time": "22:39:51"},
- {"date": "2013-10-04", "time": "20:34:13"},
- {"date": "2014-07-16", "time": "02:28:40"}
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {"date": "2018-06-13", "time": "20:16:07"}
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {"date": "2015-05-29", "time": "16:46:17"},
- {"date": "2015-06-01", "time": "15:03:53"}
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {"date": "2011-10-08", "time": "12:02:23"},
- {"date": "2014-08-18", "time": "02:11:11"},
- {"date": "2016-01-07", "time": "05:27:51"},
- {"date": "2016-10-21", "time": "20:15:55"},
- {"date": "2016-12-01", "time": "03:57:10"},
- {"date": "2016-12-29", "time": "01:54:42"},
- {"date": "2018-07-22", "time": "19:55:31"},
- {"date": "2018-09-07", "time": "01:42:54"},
- {"date": "2019-03-08", "time": "03:41:06"}
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-3.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp
index 9594cf9..4c7e924 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp
index 7fcf9e4..197d417 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp
@@ -19,407 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {
- "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
- "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
- },
- {
- "dates": ["2017-04-20", "2017-05-03"],
- "times": ["18:39:06", "17:58:02"]
- },
- {
- "dates": ["2019-03-19"],
- "times": ["22:04:48"]
- }
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {
- "dates": ["2018-05-25", "2018-09-18"],
- "times": ["19:52:07", "16:09:44"]
- },
- {
- "dates": ["2019-10-18"],
- "times": ["21:29:09"]
- }
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {
- "dates": ["2019-06-07"],
- "times": ["17:54:58"]
- }
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {
- "dates": ["2011-05-03", "2011-08-23"],
- "times": ["20:54:05", "20:49:45"]
- },
- {
- "dates": ["2014-12-04"],
- "times": ["06:13:01"]
- },
- {
- "dates": ["2016-11-16"],
- "times": ["19:25:55"]
- }
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {
- "dates": ["2016-06-18", "2016-10-15"],
- "times": ["21:35:45", "18:17:51"]
- }
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {
- "dates": ["2015-04-02"],
- "times": ["21:45:17"]
- }
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {
- "dates": ["2014-08-27"],
- "times": ["17:49:18"]
- },
- {
- "dates": ["2015-12-19"],
- "times": ["21:30:31"]
- },
- {
- "dates": ["2018-11-27"],
- "times": ["15:53:50"]
- }
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {
- "dates": ["2014-01-28", "2014-11-16"],
- "times": ["20:56:04", "16:11:58"]
- },
- {
- "dates": ["2015-11-15", "2015-11-15"],
- "times": ["19:21:53", "19:33:39"]
- }
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {
- "dates": ["2019-04-11"],
- "times": ["18:30:12"]
- }
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {
- "dates": ["2015-06-06"],
- "times": ["20:01:06"]
- },
- {
- "dates": ["2019-03-14"],
- "times": ["22:01:52"]
- }
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {
- "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
- "times": ["18:55:17", "16:48:05", "22:20:24"]
- }
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {
- "dates": ["2011-04-23"],
- "times": ["21:11:22"]
- },
- {
- "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
- "times": ["19:42:48", "19:16:08", "19:14:18"]
- },
- {
- "dates": ["2015-12-05"],
- "times": ["19:22:42"]
- },
- {
- "dates": ["2017-05-15"],
- "times": ["23:19:00"]
- }
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {
- "dates": ["2012-07-13"],
- "times": ["21:43:57"]
- },
- {
- "dates": ["2016-12-24"],
- "times": ["02:27:31"]
- },
- {
- "dates": ["2017-08-31"],
- "times": ["00:35:26"]
- }
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {
- "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
- "times": ["12:35:33", "23:35:49", "19:14:56"]
- }
-
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {
- "dates": ["2016-09-10"],
- "times": ["19:26:19"]
- },
- {
- "dates": ["2018-09-08"],
- "times": ["14:15:37"]
- },
- {
- "dates": ["2019-09-13"],
- "times": ["22:47:25"]
- }
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {
- "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
- "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
- }
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {
- "dates": ["2019-06-05"],
- "times": ["18:22:49"]
- }
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {
- "dates": ["2011-09-24"],
- "times": ["21:37:32"]
- },
- {
- "dates": ["2014-03-10"],
- "times": ["20:20:07"]
- },
- {
- "dates": ["2015-05-27", "2015-08-29"],
- "times": ["00:40:24", "17:58:15"]
- },
- {
- "dates": ["2018-03-16"],
- "times": ["15:03:26"]
- }
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {
- "dates": ["2015-03-16", "2015-12-21"],
- "times": ["23:51:16", "04:48:01"]
- },
- {
- "dates": ["2016-10-28", "2016-10-28"],
- "times": ["20:22:42", "20:23:00"]
- }
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {
- "dates": ["2013-10-22"],
- "times": ["16:49:21"]
- },
- {
- "dates": ["2014-11-21"],
- "times": ["17:39:24"]
- }
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {
- "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
- "times": ["18:30:48", "20:41:45", "23:22:27"]
- },
- {
- "dates": ["2015-07-21", "2015-07-21"],
- "times": ["20:43:56", "20:45:07"]
- }
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {
- "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
- "times": ["19:49:05", "03:52:18", "01:13:19"]
- }
-
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {
- "dates": ["2015-04-11", "2015-11-21"],
- "times": ["13:14:14", "16:05:56"]
- },
- {
- "dates": ["2016-05-06"],
- "times": ["14:10:04"]
- },
- {
- "dates": ["2017-08-09", "2017-10-21"],
- "times": ["15:15:10", "15:12:56"]
- }
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {
- "dates": ["2015-12-03"],
- "times": ["18:44:00"]
- },
- {
- "dates": ["2016-03-17", "2016-11-02"],
- "times": ["18:19:21", "15:58:38"]
- }
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {
- "dates": ["2019-04-04"],
- "times": ["22:02:37"]
- }
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {
- "dates": ["2019-02-27"],
- "times": ["14:03:08"]
- }
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {
- "dates": ["2011-12-21"],
- "times": ["19:02:51"]
- },
- {
- "dates": ["2012-04-15", "2012-04-15"],
- "times": ["04:21:39", "14:23:56"]
- },
- {
- "dates": ["2013-06-30", "2013-10-04"],
- "times": ["22:39:51", "20:34:13"]
- },
- {
- "dates": ["2014-07-16"],
- "times": ["02:28:40"]
- }
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {
- "dates": ["2018-06-13"],
- "times": ["20:16:07"]
- }
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {
- "dates": ["2015-05-29", "2015-06-01"],
- "times": ["16:46:17", "15:03:53"]
- }
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {
- "dates": ["2011-10-08"],
- "times": ["12:02:23"]
- },
- {
- "dates": ["2014-08-18"],
- "times": ["02:11:11"]
- },
- {
- "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
- "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
- },
- {
- "dates": ["2018-07-22", "2018-09-07"],
- "times": ["19:55:31", "01:42:54"]
- },
- {
- "dates": ["2019-03-08"],
- "times": ["03:41:06"]
- }
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-4.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp
index c590ecb..176da7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp
@@ -19,4 +19,4 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.1.ddl.sqlpp
new file mode 100644
index 0000000..3180b30
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.1.ddl.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a feed (+ meta field).
+ * 1) UNNEST query, performing a point search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string
+};
+
+CREATE TYPE MetaType AS {
+ _id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+WITH META(MetaType)
+PRIMARY KEY business_id;
+
+CREATE FEED YelpCheckinStream WITH {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "CheckinType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/yelp-checkin/feeds.csv",
+ "format": "csv",
+ "delimiter": ",",
+ "record-format": "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header": "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.2.update.sqlpp
new file mode 100644
index 0000000..13a634b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 TestYelp;
+
+SET `wait-for-completion-feed` "true";
+CONNECT FEED YelpCheckinStream
+TO DATASET YelpCheckin;
+
+START FEED YelpCheckinStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.3.ddl.sqlpp
new file mode 100644
index 0000000..630b71e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 TestYelp;
+
+CREATE INDEX checkinDatesIdx
+ON YelpCheckin ( UNNEST dates : string )
+EXCLUDE UNKNOWN KEY;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.4.query.sqlpp
new file mode 100644
index 0000000..9078e62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-new-index/using-feed-new-index.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 TestYelp;
+SET `compiler.arrayindex` "true";
+
+FROM YelpCheckin C
+UNNEST C.dates D
+WHERE D > ""
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.1.ddl.sqlpp
new file mode 100644
index 0000000..e1e82a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.1.ddl.sqlpp
@@ -0,0 +1,59 @@
+/*
+ * 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: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a feed (+ meta field).
+ * 1) UNNEST query, performing a point search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string
+};
+
+CREATE TYPE MetaType AS {
+ _id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType)
+WITH META(MetaType)
+PRIMARY KEY business_id;
+
+CREATE INDEX checkinDatesIdx
+ON YelpCheckin ( UNNEST dates : string )
+EXCLUDE UNKNOWN KEY;
+
+CREATE FEED YelpCheckinStream WITH {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "CheckinType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/yelp-checkin/feeds.csv",
+ "format": "csv",
+ "delimiter": ",",
+ "record-format": "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header": "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.2.update.sqlpp
new file mode 100644
index 0000000..13a634b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 TestYelp;
+
+SET `wait-for-completion-feed` "true";
+CONNECT FEED YelpCheckinStream
+TO DATASET YelpCheckin;
+
+START FEED YelpCheckinStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.3.query.sqlpp
new file mode 100644
index 0000000..9078e62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/using-feed-old-index/using-feed-old-index.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 TestYelp;
+SET `compiler.arrayindex` "true";
+
+FROM YelpCheckin C
+UNNEST C.dates D
+WHERE D > ""
+SELECT COUNT(*);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
index 884c224..8c7303c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -19,599 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2016-04-26",
- "2016-08-30",
- "2016-10-15",
- "2016-11-18",
- "2017-04-20",
- "2017-05-03",
- "2019-03-19"
- ],
- "times": [
- "19:49:16",
- "18:36:57",
- "02:45:18",
- "01:54:50",
- "18:39:06",
- "17:58:02",
- "22:04:48"
- ]
- }
- }
- }
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2018-05-25",
- "2018-09-18",
- "2019-10-18"
- ],
- "times": [
- "19:52:07",
- "16:09:44",
- "21:29:09"
- ]
- }
- }
- }
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-06-07"
- ],
- "times": [
- "17:54:58"
- ]
- }
- }
- }
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-05-03",
- "2011-08-23",
- "2014-12-04",
- "2016-11-16"
- ],
- "times": [
- "20:54:05",
- "20:49:45",
- "06:13:01",
- "19:25:55"
- ]
- }
- }
- }
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2016-06-18",
- "2016-10-15"
- ],
- "times": [
- "21:35:45",
- "18:17:51"
- ]
- }
- }
- }
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-04-02"
- ],
- "times": [
- "21:45:17"
- ]
- }
- }
- }
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2014-08-27",
- "2015-12-19",
- "2018-11-27"
- ],
- "times": [
- "17:49:18",
- "21:30:31",
- "15:53:50"
- ]
- }
- }
- }
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2014-01-28",
- "2014-11-16",
- "2015-11-15",
- "2015-11-15"
- ],
- "times": [
- "20:56:04",
- "16:11:58",
- "19:21:53",
- "19:33:39"
- ]
- }
- }
- }
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-04-11"
- ],
- "times": [
- "18:30:12"
- ]
- }
- }
- }
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-06-06",
- "2019-03-14"
- ],
- "times": [
- "20:01:06",
- "22:01:52"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2018-09-29",
- "2018-10-20",
- "2018-10-20"
- ],
- "times": [
- "18:55:17",
- "16:48:05",
- "22:20:24"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-04-23",
- "2014-05-04",
- "2014-05-11",
- "2014-06-04",
- "2015-12-05",
- "2017-05-15"
- ],
- "times": [
- "21:11:22",
- "19:42:48",
- "19:16:08",
- "19:14:18",
- "19:22:42",
- "23:19:00"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2012-07-13",
- "2016-12-24",
- "2017-08-31"
- ],
- "times": [
- "21:43:57",
- "02:27:31",
- "00:35:26"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2013-04-13",
- "2013-08-19",
- "2013-10-04"
- ],
- "times": [
- "12:35:33",
- "23:35:49",
- "19:14:56"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2016-09-10",
- "2018-09-08",
- "2019-09-13"
- ],
- "times": [
- "19:26:19",
- "14:15:37",
- "22:47:25"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2017-05-19",
- "2017-05-19",
- "2017-08-28",
- "2017-09-20",
- "2017-10-01",
- "2017-10-01",
- "2017-12-27"
- ],
- "times": [
- "14:30:16",
- "14:30:25",
- "15:49:37",
- "20:19:51",
- "16:31:05",
- "16:56:27",
- "23:33:20"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-06-05"
- ],
- "times": [
- "18:22:49"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-09-24",
- "2014-03-10",
- "2015-05-27",
- "2015-08-29",
- "2018-03-16"
- ],
- "times": [
- "21:37:32",
- "20:20:07",
- "00:40:24",
- "17:58:15",
- "15:03:26"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-03-16",
- "2015-12-21",
- "2016-10-28",
- "2016-10-28"
- ],
- "times": [
- "23:51:16",
- "04:48:01",
- "20:22:42",
- "20:23:00"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2013-10-22",
- "2014-11-21"
- ],
- "times": [
- "16:49:21",
- "17:39:24"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2014-08-07",
- "2014-09-16",
- "2014-10-12",
- "2015-07-21",
- "2015-07-21"
- ],
- "times": [
- "18:30:48",
- "20:41:45",
- "23:22:27",
- "20:43:56",
- "20:45:07"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-05-02",
- "2015-05-06",
- "2015-09-26"
- ],
- "times": [
- "19:49:05",
- "03:52:18",
- "01:13:19"
- ]
- }
- }
- }
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-04-11",
- "2015-11-21",
- "2016-05-06",
- "2017-08-09",
- "2017-10-21"
- ],
- "times": [
- "13:14:14",
- "16:05:56",
- "14:10:04",
- "15:15:10",
- "15:12:56"
- ]
- }
- }
- }
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-12-03",
- "2016-03-17",
- "2016-11-02"
- ],
- "times": [
- "18:44:00",
- "18:19:21",
- "15:58:38"
- ]
- }
- }
- }
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-04-04"
- ],
- "times": [
- "22:02:37"
- ]
- }
- }
- }
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2019-02-27"
- ],
- "times": [
- "14:03:08"
- ]
- }
- }
- }
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-12-21",
- "2012-04-15",
- "2012-04-15",
- "2013-06-30",
- "2013-10-04",
- "2014-07-16"
- ],
- "times": [
- "19:02:51",
- "04:21:39",
- "14:23:56",
- "22:39:51",
- "20:34:13",
- "02:28:40"
- ]
- }
- }
- }
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2018-06-13"
- ],
- "times": [
- "20:16:07"
- ]
- }
- }
- }
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2015-05-29",
- "2015-06-01"
- ],
- "times": [
- "16:46:17",
- "15:03:53"
- ]
- }
- }
- }
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_data": {
- "checkin_temporal": {
- "checkin_times": {
- "dates": [
- "2011-10-08",
- "2014-08-18",
- "2016-01-07",
- "2016-10-21",
- "2016-12-01",
- "2016-12-29",
- "2018-07-22",
- "2018-09-07",
- "2019-03-08"
- ],
- "times": [
- "12:02:23",
- "02:11:11",
- "05:27:51",
- "20:15:55",
- "03:57:10",
- "01:54:42",
- "19:55:31",
- "01:42:54",
- "03:41:06"
- ]
- }
- }
- }
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-3-level-record-path.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
index 88a6782..db37a3d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -19,4 +19,6 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ) ;
+CREATE INDEX IdxYelpCheckinDates
+ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string )
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp
index 02ddf21..f05a19b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -19,258 +19,6 @@
USE TestYelp;
-INSERT INTO YelpCheckin (
- [
- {
- "business_id": "--1UhMGODdWsrMastO9DZw",
- "checkin_times": [
- {"date": "2016-04-26", "time": "19:49:16"},
- {"date": "2016-08-30", "time": "18:36:57"},
- {"date": "2016-10-15", "time": "02:45:18"},
- {"date": "2016-11-18", "time": "01:54:50"},
- {"date": "2017-04-20", "time": "18:39:06"},
- {"date": "2017-05-03", "time": "17:58:02"},
- {"date": "2019-03-19", "time": "22:04:48"}
- ]
- },
- {
- "business_id": "--EF5N7P70J_UYBTPypYlA",
- "checkin_times": [
- {"date": "2018-05-25", "time": "19:52:07"},
- {"date": "2018-09-18", "time": "16:09:44"},
- {"date": "2019-10-18", "time": "21:29:09"}
- ]
- },
- {
- "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
- "checkin_times": [
- {"date": "2019-06-07", "time": "17:54:58"}
- ]
- },
- {
- "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
- "checkin_times": [
- {"date": "2011-05-03", "time": "20:54:05"},
- {"date": "2011-08-23", "time": "20:49:45"},
- {"date": "2014-12-04", "time": "06:13:01"},
- {"date": "2016-11-16", "time": "19:25:55"}
- ]
- },
- {
- "business_id": "--YPwqIlRJrhHkJcjY3eiA",
- "checkin_times": [
- {"date": "2016-06-18", "time": "21:35:45"},
- {"date": "2016-10-15", "time": "18:17:51"}
- ]
- },
- {
- "business_id": "--e8PjCNhEz32pprnPhCwQ",
- "checkin_times": [
- {"date": "2015-04-02", "time": "21:45:17"}
- ]
- },
- {
- "business_id": "--kinfHwmtdjz03g8B8z8Q",
- "checkin_times": [
- {"date": "2014-08-27", "time": "17:49:18"},
- {"date": "2015-12-19", "time": "21:30:31"},
- {"date": "2018-11-27", "time": "15:53:50"}
- ]
- },
- {
- "business_id": "--q6datkI-f0EoVheXNEeQ",
- "checkin_times": [
- {"date": "2014-01-28", "time": "20:56:04"},
- {"date": "2014-11-16", "time": "16:11:58"},
- {"date": "2015-11-15", "time": "19:21:53"},
- {"date": "2015-11-15", "time": "19:33:39"}
- ]
- },
- {
- "business_id": "--qvQS4MigHPykD2GV0-zw",
- "checkin_times": [
- {"date": "2019-04-11", "time": "18:30:12"}
- ]
- },
- {
- "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
- "checkin_times": [
- {"date": "2015-06-06", "time": "20:01:06"},
- {"date": "2019-03-14", "time": "22:01:52"}
- ]
- },
- {
- "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
- "checkin_times": [
- {"date": "2018-09-29", "time": "18:55:17"},
- {"date": "2018-10-20", "time": "16:48:05"},
- {"date": "2018-10-20", "time": "22:20:24"}
- ]
- },
- {
- "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
- "checkin_times": [
- {"date": "2011-04-23", "time": "21:11:22"},
- {"date": "2014-05-04", "time": "19:42:48"},
- {"date": "2014-05-11", "time": "19:16:08"},
- {"date": "2014-06-04", "time": "19:14:18"},
- {"date": "2015-12-05", "time": "19:22:42"},
- {"date": "2017-05-15", "time": "23:19:00"}
- ]
- },
- {
- "business_id": "-0KMvRFwDWdVBeTpT11iHw",
- "checkin_times": [
- {"date": "2012-07-13", "time": "21:43:57"},
- {"date": "2016-12-24", "time": "02:27:31"},
- {"date": "2017-08-31", "time": "00:35:26"}
- ]
- },
- {
- "business_id": "-0LPtgJC31FWMrMv317p0Q",
- "checkin_times": [
- {"date": "2013-04-13", "time": "12:35:33"},
- {"date": "2013-08-19", "time": "23:35:49"},
- {"date": "2013-10-04", "time": "19:14:56"}
- ]
- },
- {
- "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
- "checkin_times": [
- {"date": "2016-09-10", "time": "19:26:19"},
- {"date": "2018-09-08", "time": "14:15:37"},
- {"date": "2019-09-13", "time": "22:47:25"}
- ]
- },
- {
- "business_id": "-0RRiWDtfnS16AKCtfvBZg",
- "checkin_times": [
- {"date": "2017-05-19", "time": "14:30:16"},
- {"date": "2017-05-19", "time": "14:30:25"},
- {"date": "2017-08-28", "time": "15:49:37"},
- {"date": "2017-09-20", "time": "20:19:51"},
- {"date": "2017-10-01", "time": "16:31:05"},
- {"date": "2017-10-01", "time": "16:56:27"},
- {"date": "2017-12-27", "time": "23:33:20"}
- ]
- },
- {
- "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
- "checkin_times": [
- {"date": "2019-06-05", "time": "18:22:49"}
- ]
- },
- {
- "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
- "checkin_times": [
- {"date": "2011-09-24", "time": "21:37:32"},
- {"date": "2014-03-10", "time": "20:20:07"},
- {"date": "2015-05-27", "time": "00:40:24"},
- {"date": "2015-08-29", "time": "17:58:15"},
- {"date": "2018-03-16", "time": "15:03:26"}
- ]
- },
- {
- "business_id": "-0aOudcaAyac0VJbMX-L1g",
- "checkin_times": [
- {"date": "2015-03-16", "time": "23:51:16"},
- {"date": "2015-12-21", "time": "04:48:01"},
- {"date": "2016-10-28", "time": "20:22:42"},
- {"date": "2016-10-28", "time": "20:23:00"}
- ]
- },
- {
- "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
- "checkin_times": [
- {"date": "2013-10-22", "time": "16:49:21"},
- {"date": "2014-11-21", "time": "17:39:24"}
- ]
- },
- {
- "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
- "checkin_times": [
- {"date": "2014-08-07", "time": "18:30:48"},
- {"date": "2014-09-16", "time": "20:41:45"},
- {"date": "2014-10-12", "time": "23:22:27"},
- {"date": "2015-07-21", "time": "20:43:56"},
- {"date": "2015-07-21", "time": "20:45:07"}
- ]
- },
- {
- "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
- "checkin_times": [
- {"date": "2015-05-02", "time": "19:49:05"},
- {"date": "2015-05-06", "time": "03:52:18"},
- {"date": "2015-09-26", "time": "01:13:19"}
- ]
- },
- {
- "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
- "checkin_times": [
- {"date": "2015-04-11", "time": "13:14:14"},
- {"date": "2015-11-21", "time": "16:05:56"},
- {"date": "2016-05-06", "time": "14:10:04"},
- {"date": "2017-08-09", "time": "15:15:10"},
- {"date": "2017-10-21", "time": "15:12:56"}
- ]
- },
- {
- "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
- "checkin_times": [
- {"date": "2015-12-03", "time": "18:44:00"},
- {"date": "2016-03-17", "time": "18:19:21"},
- {"date": "2016-11-02", "time": "15:58:38"}
- ]
- },
- {
- "business_id": "-1E2CQu_38mkghvmZgCCRw",
- "checkin_times": [
- {"date": "2019-04-04", "time": "22:02:37"}
- ]
- },
- {
- "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
- "checkin_times": [
- {"date": "2019-02-27", "time": "14:03:08"}
- ]
- },
- {
- "business_id": "-23R9P2eG7VTc6DVLjFKzA",
- "checkin_times": [
- {"date": "2011-12-21", "time": "19:02:51"},
- {"date": "2012-04-15", "time": "04:21:39"},
- {"date": "2012-04-15", "time": "14:23:56"},
- {"date": "2013-06-30", "time": "22:39:51"},
- {"date": "2013-10-04", "time": "20:34:13"},
- {"date": "2014-07-16", "time": "02:28:40"}
- ]
- },
- {
- "business_id": "-26MGfikhJiTfCI-GqmzhQ",
- "checkin_times": [
- {"date": "2018-06-13", "time": "20:16:07"}
- ]
- },
- {
- "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
- "checkin_times": [
- {"date": "2015-05-29", "time": "16:46:17"},
- {"date": "2015-06-01", "time": "15:03:53"}
- ]
- },
- {
- "business_id": "-2hDBMaza_ldqnZdiU06LQ",
- "checkin_times": [
- {"date": "2011-10-08", "time": "12:02:23"},
- {"date": "2014-08-18", "time": "02:11:11"},
- {"date": "2016-01-07", "time": "05:27:51"},
- {"date": "2016-10-21", "time": "20:15:55"},
- {"date": "2016-12-01", "time": "03:57:10"},
- {"date": "2016-12-29", "time": "01:54:42"},
- {"date": "2018-07-22", "time": "19:55:31"},
- {"date": "2018-09-07", "time": "01:42:54"},
- {"date": "2019-03-08", "time": "03:41:06"}
- ]
- }
- ]
-);
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-3.json"),
+ ("format"="json"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp
index a8cf7d4..4ba9f82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp
@@ -19,4 +19,6 @@
USE TestYelp;
-CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string );
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness
+ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string )
+EXCLUDE UNKNOWN KEY;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/datetime_tzeq/datetime_tzeq.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/datetime_tzeq/datetime_tzeq.4.query.sqlpp
new file mode 100644
index 0000000..77444a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/datetime_tzeq/datetime_tzeq.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+{
+ 'result1':(datetime('2000-03-01T02:00:00+04:00') = datetime('2000-03-01T02:00:00+05:00')),
+ 'result2':(datetime('2000-03-01T02:00:00+04:00') = datetime('2000-03-01T02:00:00Z')),
+ 'result3':(datetime('2000-03-01T02:00:00+04:00') = datetime('2000-03-01T02:00:00')),
+ 'result4':(datetime('2000-03-01T02:00:00Z') = datetime('2000-03-01T02:00:00'))
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/is_distinct_01/is_distinct_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/is_distinct_01/is_distinct_01.1.query.sqlpp
new file mode 100644
index 0000000..c0ed568
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/is_distinct_01/is_distinct_01.1.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+from
+ [ int64(2), float(2.0), double(2.5), "str1", "str2", true, false, null, missing ] arg1,
+ [ int32(2), double(2.0), float(2.5), "str1", "str2", true, false, null, missing ] arg2
+let
+ is_distinct = arg1 is distinct from arg2,
+ is_not_distinct = arg1 is not distinct from arg2
+select
+ case
+ when arg1 is missing then "MISSING"
+ when arg1 is null then "NULL"
+ else arg1
+ end as arg1,
+ case
+ when arg2 is missing then "MISSING"
+ when arg2 is null then "NULL"
+ else arg2
+ end as arg2,
+ is_distinct as `!!==`,
+ is_not_distinct as `==`
+order by is_distinct, arg1, arg2
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/is_distinct_01/is_distinct_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/is_distinct_01/is_distinct_01.2.query.sqlpp
new file mode 100644
index 0000000..fcc1dc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/is_distinct_01/is_distinct_01.2.query.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+let
+ x = 2, y = 3
+select
+ /* test expression */
+ x between 1 and 3 is not distinct from x between 0 and 4 as t1,
+ /* expected operator precedence */
+ (x between 1 and 3) is not distinct from (x between 0 and 4) as t1_expected,
+ /* unexpected operator precedence */
+ (((x between 1 and 3) is not distinct from x) between 0 and 4) as t1_unexpected,
+
+ /* test expression */
+ x is unknown is not distinct from y is unknown as t2,
+ /* expected operator precedence */
+ (x is unknown) is not distinct from (y is unknown) as t2_expected,
+ /* unexpected operator precedence */
+ (((x is unknown) is not distinct from y) is unknown) as t2_unexpected,
+
+ /* test expression */
+ x is not unknown is distinct from y is unknown as t3,
+ /* expected operator precedence */
+ (x is not unknown) is distinct from (y is unknown) as t3_expected,
+ /* unexpected operator precedence */
+ (((x is not unknown) is distinct from y) is unknown) as t3_unexpected;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_01/binary_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_01/binary_01.3.query.sqlpp
index 509d97d..75c4c53 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_01/binary_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_01/binary_01.3.query.sqlpp
@@ -17,20 +17,40 @@
* under the License.
*/
-use test;
+let
+test = [
+ 'ABCDEF0123456789',
+ 'abcdef0123456789',
+ '0A0B0C0D0E0F',
+ '01020304050607080900',
+ '',
+ test.hex('ABCDEF0123456789')
+],
+testNull = [
+ null,
+ '@#!',
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ date('2020-01-01'),
+ datetime('2020-01-01T00:00:00Z'),
+ time("00:00:00"),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-[
- test.hex('ABCDEF0123456789'),
- test.hex('abcdef0123456789'),
- test.hex('0A0B0C0D0E0F'),
- test.hex('01020304050607080900'),
- test.hex(''),
- test.hex(test.hex('ABCDEF0123456789')),
-
- test.base64('0123456789qwertyuiopasdfghjklzxcvbnmQWERTYUIOPLKJHGFDSAZXCVBNM+/'),
- test.base64(''),
- test.base64('QXN0ZXJpeA=='),
- test.base64('QXN0ZXJpeAE='),
- test.base64('QXN0ZXJpeAE8'),
- test.base64(test.base64('QXN0ZXJpeAE8'))
-];
+select 0 g, i, hex(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, hex(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, hex(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_01/binary_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_01/binary_01.4.query.sqlpp
new file mode 100644
index 0000000..3b929e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_01/binary_01.4.query.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+ '0123456789qwertyuiopasdfghjklzxcvbnmQWERTYUIOPLKJHGFDSAZXCVBNM+/',
+ '',
+ 'QXN0ZXJpeA==',
+ 'QXN0ZXJpeAE=',
+ 'QXN0ZXJpeAE8',
+ test.base64('QXN0ZXJpeAE8')
+],
+testNull = [
+ null,
+ '@#!',
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ date('2020-01-01'),
+ datetime('2020-01-01T00:00:00Z'),
+ time("00:00:00"),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
+
+select 0 g, i, base64(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, base64(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, base64(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_02/binary_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_02/binary_02.1.query.sqlpp
new file mode 100644
index 0000000..d150323
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_02/binary_02.1.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": hex("@#!1"),
+ "null_1": hex(date("2020-01-01")),
+ "null_2": hex([]),
+ "null_3": hex({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_02/binary_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_02/binary_02.2.query.sqlpp
new file mode 100644
index 0000000..7e63bb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/binary_02/binary_02.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": base64("@#!2"),
+ "null_1": base64(date("2020-01-01")),
+ "null_2": base64([]),
+ "null_3": base64({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/boolean_01/boolean_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/boolean_01/boolean_01.3.query.sqlpp
index e7eb630..6a2d6e0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/boolean_01/boolean_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/boolean_01/boolean_01.3.query.sqlpp
@@ -17,10 +17,67 @@
* under the License.
*/
-use test;
+let
+testFalse = [
+ false,
+ test.boolean('false'),
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ float("NaN"),
+ double(0),
+ double("NaN"),
+ 'false'
+],
+testTrue = [
+ true,
+ int8(1),
+ int8(2),
+ int16(1),
+ int16(-1),
+ int32(1),
+ int32(2),
+ int64(1),
+ int64(3),
+ float(1),
+ float("INF"),
+ float("-INF"),
+ double(1),
+ double("INF"),
+ double("-INF"),
+ 'true'
+],
+testNull = [
+ null,
+ 'TRUE',
+ 'FALSE',
+ 'abc',
+ date('1970-01-01'),
+ datetime('1970-01-01T00:00:00Z'),
+ time("00:00:00"),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ [null],
+ {},
+ {"a":null}
+],
+testMissing = [
+ missing
+]
-{
- 'boolean1':test.boolean('true'),
- 'boolean2':test.boolean('false'),
- 'boolean3':test.boolean(test.boolean('false'))
-};
+select 0 g, i, false expected, boolean(testFalse[i]) actual
+from range(0, len(testFalse)-1) i
+union all
+select 1 g, i, true expected, boolean(testTrue[i]) actual
+from range(0, len(testTrue)-1) i
+union all
+select 2 g, i, null expected, boolean(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 3 g, i, true expected, boolean(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/boolean_02/boolean_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/boolean_02/boolean_02.1.query.sqlpp
new file mode 100644
index 0000000..18e1859
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/boolean_02/boolean_02.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": boolean("FALSE"),
+ "null_1": boolean("TRUE"),
+ "null_2": boolean(date("2020-01-01")),
+ "null_3": boolean([]),
+ "null_4": boolean({})
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.3.query.sqlpp
index 8292dbd..a96ef28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.3.query.sqlpp
@@ -17,18 +17,49 @@
* under the License.
*/
-use test;
+let
+test = [
+ '2010-10-30',
+ '1987-11-19',
+ '-1987-11-19',
+ '0001-12-27',
+ '-1951-12-27',
+ '-2043-11-19',
+ '-19280329',
+ '19280329',
+ '19000228',
+ '20000229',
+ test.date('20000229'),
+ datetime('2010-10-30T01:02:03Z')
+],
+testNull = [
+ null,
+ false,
+ true,
+ '@#!',
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-{
- 'date1':test.date('2010-10-30'),
- 'date2':test.date('1987-11-19'),
- 'date3':test.date('-1987-11-19'),
- 'date4':test.date('0001-12-27'),
- 'date5':test.date('-1951-12-27'),
- 'date6':test.date('-2043-11-19'),
- 'date7':test.date('-19280329'),
- 'date8':test.date('19280329'),
- 'date9':test.date('19000228'),
- 'date10':test.date('20000229'),
- 'date11':test.date(test.date('20000229'))
-};
+select 0 g, i, date(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, date(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, date(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
new file mode 100644
index 0000000..be6b9c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_01/date_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+ '10/30/2010',
+ '31-12-2020'
+],
+fmt = [
+ 'MM/DD/YYYY',
+ 'DD-MM-YYYY'
+]
+
+select i, date(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
new file mode 100644
index 0000000..6a157f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/date_02/date_02.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": date("@#!"),
+ "null_1": date(false),
+ "null_2": date(0),
+ "null_3": date(time("01:02:03")),
+ "null_4": date([]),
+ "null_5": date({}),
+ "null_6": date("12/31/2020", "INVALID_FORMAT")
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.3.query.sqlpp
index 7f91427..97a9955 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.3.query.sqlpp
@@ -17,22 +17,51 @@
* under the License.
*/
-use test;
+let
+test = [
+ '2010-10-30T10:50:56.999+05:45',
+ '2010-10-30T10:30:56.250-10:00',
+ '1987-11-19T09:20:00.200Z',
+ '1987-11-19T10:50:56Z',
+ '-1987-11-19T10:50:56.099-05:30',
+ '-0001-11-19T10:50:56.719Z',
+ '1951-12-27T12:20:15Z',
+ '2043-11-19T10:50:56.719Z',
+ '-19280329T174937374-0630',
+ '-19280329T174937374+0630',
+ '-19280329T174937374',
+ '-19280329T174937374+0630',
+ '-19280329T17493737+0630',
+ '-19280301T05493737+0630',
+ test.datetime('-19280301T05493737+0630'),
+ date('2020-01-02')
+],
+testNull = [
+ null,
+ false,
+ true,
+ '@#!',
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D')
+],
+testMissing = [
+ missing
+]
-{
- 'datetime1':test.datetime('2010-10-30T10:50:56.999+05:45'),
- 'datetime2':test.datetime('2010-10-30T10:30:56.250-10:00'),
- 'datetime3':test.datetime('1987-11-19T09:20:00.200Z'),
- 'datetime4':test.datetime('1987-11-19T10:50:56Z'),
- 'datetime5':test.datetime('-1987-11-19T10:50:56.099-05:30'),
- 'datetime6':test.datetime('-0001-11-19T10:50:56.719Z'),
- 'datetime7':test.datetime('1951-12-27T12:20:15Z'),
- 'datetime8':test.datetime('2043-11-19T10:50:56.719Z'),
- 'datetime9':test.datetime('-19280329T174937374-0630'),
- 'datetime10':test.datetime('-19280329T174937374+0630'),
- 'datetime11':test.datetime('-19280329T174937374'),
- 'datetime12':test.datetime('-19280329T174937374+0630'),
- 'datetime13':test.datetime('-19280329T17493737+0630'),
- 'datetime14':test.datetime('-19280301T05493737+0630'),
- 'datetime15':test.datetime(test.datetime('-19280301T05493737+0630'))
-};
+select 0 g, i, datetime(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, datetime(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, datetime(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
new file mode 100644
index 0000000..98b3ee6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_01/datetime_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+ '10/30/2010 58-57-10',
+ '31-12-2020 11/58/59'
+],
+fmt = [
+ 'MM/DD/YYYY ss-mm-hh',
+ 'DD-MM-YYYY hh/mm/ss'
+]
+
+select i, datetime(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
new file mode 100644
index 0000000..082d840
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/datetime_02/datetime_02.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": datetime("@#!"),
+ "null_1": datetime(false),
+ "null_2": datetime(0),
+ "null_3": datetime(time("01:02:03")),
+ "null_4": datetime([]),
+ "null_5": datetime({}),
+ "null_6": datetime("1951-12-27T12:20:15Z", "INVALID_FORMAT")
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/double_01/double_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/double_01/double_01.3.query.sqlpp
index a6bdbc5..c7fd21c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/double_01/double_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/double_01/double_01.3.query.sqlpp
@@ -17,14 +17,45 @@
* under the License.
*/
-use test;
+let
+test = [
+ 'NaN',
+ 'INF',
+ '-INF',
+ '-80.20d',
+ '-20.56e-30',
+ '-20.56e-300',
+ test.double('-20.56e-300'),
+ int8(8),
+ int16(16),
+ int32(32),
+ int64(64),
+ float(2.5),
+ false,
+ true
+],
+testNull = [
+ null,
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-{
- 'double1':test.double('NaN'),
- 'double2':test.double('INF'),
- 'double3':test.double('-INF'),
- 'double4':test.double('-80.20d'),
- 'double5':test.double('-20.56e-30'),
- 'double6':test.double('-20.56e-300'),
- 'double7':test.double(test.double('-20.56e-300'))
-};
+select 0 g, i, double(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, double(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, double(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/double_02/double_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/double_02/double_02.1.query.sqlpp
new file mode 100644
index 0000000..5a68610
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/double_02/double_02.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": double("@#!"),
+ "null_1": double(datetime('1987-11-19T10:50:56Z')),
+ "null_2": double(date('2020-01-02')),
+ "null_3": double(time("01:02:03")),
+ "null_4": double([]),
+ "null_5": double({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.3.query.sqlpp
index d0520df..db2913c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.3.query.sqlpp
@@ -22,20 +22,52 @@
* Date : 7 May 2013
*/
-use test;
+let
+test = [
+ 'P30Y10M25DT13H12M50S',
+ 'P25DT13H12M50S',
+ 'PT13H12M50S',
+ 'P30YT12MS',
+ 'PT13H',
+ '-P30Y10M25DT13H12M50S',
+ '-P25DT13H12M50S',
+ '-PT13H50S',
+ 'P120D',
+ '-P28M',
+ 'PT29M90.937S',
+ 'P300Y15M60DT300H98M482.435S',
+ test.duration('P300Y15M60DT300H98M482.435S'),
+ year_month_duration('P30Y10M'),
+ day_time_duration('P25DT13H12M50S')
+],
+testNull = [
+ null,
+ false,
+ true,
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-{
- 'duration1':test.duration('P30Y10M25DT13H12M50S'),
- 'duration2':test.duration('P25DT13H12M50S'),
- 'duration3':test.duration('PT13H12M50S'),
- 'duration4':test.duration('P30YT12MS'),
- 'duration5':test.duration('PT13H'),
- 'duration6':test.duration('-P30Y10M25DT13H12M50S'),
- 'duration7':test.duration('-P25DT13H12M50S'),
- 'duration8':test.duration('-PT13H50S'),
- 'duration9':test.duration('P120D'),
- 'duration10':test.duration('-P28M'),
- 'duration11':test.duration('PT29M90.937S'),
- 'duration12':test.duration('P300Y15M60DT300H98M482.435S'),
- 'duration13':test.duration(test.duration('P300Y15M60DT300H98M482.435S'))
-};
+select 0 g, i, duration(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, duration(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, duration(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.4.query.sqlpp
new file mode 100644
index 0000000..1f34234
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.4.query.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * 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 : test year_month_duration constructors
+ * Expected Res : Success
+ * Date : 7 May 2013
+ * issue : 363
+ */
+
+let
+test = [
+ 'P30Y10M',
+ 'P30Y',
+ '-P30Y10M',
+ '-P28M',
+ 'P300Y15M',
+ test.year_month_duration('P300Y15M'),
+ duration('P300Y16M60DT300H98M482.435S')
+],
+testNull = [
+ null,
+ false,
+ true,
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ day_time_duration('P25DT13H12M50S'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
+
+select 0 g, i, year_month_duration(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, year_month_duration(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, year_month_duration(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.5.query.sqlpp
new file mode 100644
index 0000000..4c081db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_01/duration_01.5.query.sqlpp
@@ -0,0 +1,68 @@
+/*
+ * 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 : test day_time_duration constructors
+ * Expected Res : Success
+ * Date : 7 May 2013
+ * issue : 363
+ */
+
+let
+test = [
+ 'P25DT13H12M50S',
+ 'PT13H12M50S',
+ 'PT13H',
+ '-P25DT13H12M50S',
+ '-PT13H50S',
+ 'P120D',
+ 'PT29M90.937S',
+ test.day_time_duration('PT14H'),
+ duration('P1Y2M3DT4H5M6S')
+],
+testNull = [
+ null,
+ false,
+ true,
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ year_month_duration('P1Y'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
+
+select 0 g, i, day_time_duration(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, day_time_duration(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, day_time_duration(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.1.ddl.sqlpp
deleted file mode 100644
index 2b48d01..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.1.ddl.sqlpp
+++ /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 : test sub type duration (year_month_duration and day_time_duration) constructors
- * Expected Res : Success
- * Date : 7 May 2013
- * issue : 363
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.1.query.sqlpp
new file mode 100644
index 0000000..548d6d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": duration("@#!"),
+ "null_1": duration(0),
+ "null_2": duration(datetime('1987-11-19T10:50:56Z')),
+ "null_3": duration(date('2020-01-02')),
+ "null_4": duration(time("01:02:03")),
+ "null_5": duration([]),
+ "null_6": duration({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.2.query.sqlpp
new file mode 100644
index 0000000..6b268a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.2.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": year_month_duration("@#!"),
+ "null_1": year_month_duration(0),
+ "null_2": year_month_duration(datetime('1987-11-19T10:50:56Z')),
+ "null_3": year_month_duration(date('2020-01-02')),
+ "null_4": year_month_duration(time("01:02:03")),
+ "null_5": year_month_duration([]),
+ "null_6": year_month_duration({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.2.update.sqlpp
deleted file mode 100644
index dbf9050..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.2.update.sqlpp
+++ /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 : test sub type duration (year_month_duration and day_time_duration) constructors
- * Expected Res : Success
- * Date : 7 May 2013
- * issue : 363
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.3.query.sqlpp
index e722070..dd82a21 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/duration_02/duration_02.3.query.sqlpp
@@ -16,27 +16,16 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : test sub type duration (year_month_duration and day_time_duration) constructors
- * Expected Res : Success
- * Date : 7 May 2013
- * issue : 363
- */
-use test;
+// requesttype=application/json
+// param max-warnings:json=100
{
- 'duration1':test.`year_month_duration`('P30Y10M'),
- 'duration2':test.`day_time_duration`('P25DT13H12M50S'),
- 'duration3':test.`day_time_duration`('PT13H12M50S'),
- 'duration4':test.`year_month_duration`('P30Y'),
- 'duration5':test.`day_time_duration`('PT13H'),
- 'duration6':test.`year_month_duration`('-P30Y10M'),
- 'duration7':test.`day_time_duration`('-P25DT13H12M50S'),
- 'duration8':test.`day_time_duration`('-PT13H50S'),
- 'duration9':test.`day_time_duration`('P120D'),
- 'duration10':test.`year_month_duration`('-P28M'),
- 'duration11':test.`day_time_duration`('PT29M90.937S'),
- 'duration12':test.`year_month_duration`('P300Y15M'),
- 'duration13':test.`year_month_duration`(test.`year_month_duration`('P300Y15M'))
-};
+ "null_0": day_time_duration("@#!"),
+ "null_1": day_time_duration(0),
+ "null_2": day_time_duration(datetime('1987-11-19T10:50:56Z')),
+ "null_3": day_time_duration(date('2020-01-02')),
+ "null_4": day_time_duration(time("01:02:03")),
+ "null_5": day_time_duration([]),
+ "null_6": day_time_duration({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/float_01/float_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/float_01/float_01.3.query.sqlpp
index 2186a4c..bea0f0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/float_01/float_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/float_01/float_01.3.query.sqlpp
@@ -17,13 +17,44 @@
* under the License.
*/
-use test;
+let
+test = [
+ 'NaN',
+ 'INF',
+ '-INF',
+ '-80.20',
+ '-20.56e-30',
+ test.float('-20.56e-30'),
+ int8(8),
+ int16(16),
+ int32(32),
+ int64(64),
+ double(2.5),
+ false,
+ true
+],
+testNull = [
+ null,
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-{
- 'float1':test.float('NaN'),
- 'float2':test.float('INF'),
- 'float3':test.float('-INF'),
- 'float4':test.float('-80.20'),
- 'float5':test.float('-20.56e-30'),
- 'float6':test.float(test.float('-20.56e-30'))
-};
+select 0 g, i, float(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, float(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, float(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/float_02/float_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/float_02/float_02.1.query.sqlpp
new file mode 100644
index 0000000..d0a6ebe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/float_02/float_02.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": float("@#!"),
+ "null_1": float(datetime('1987-11-19T10:50:56Z')),
+ "null_2": float(date('2020-01-02')),
+ "null_3": float(time("01:02:03")),
+ "null_4": float([]),
+ "null_5": float({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.3.query.sqlpp
index 32d9c17..4ac431a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.3.query.sqlpp
@@ -17,23 +17,58 @@
* under the License.
*/
-use test;
+let
+test = [
+ '+80i8',
+ '-80',
+ test.tinyint('+80i8'),
+ int16(16),
+ int32(32),
+ int64(64),
+ float(1.25),
+ double(2.25),
+ false,
+ true,
+ '80.25',
+ '80.5',
+ '80.75',
+ '-80.25',
+ '-80.5',
+ '-80.75',
+ '8.125e1',
+ '-812.5e-1',
+ /* out of range */
+ int16(999),
+ int32(999),
+ int64(999),
+ float(999),
+ double(999),
+ 'INF',
+ '-INF',
+ 'NaN'
+],
+testNull = [
+ null,
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-{
- 'int8':test.tinyint('+80i8'),
- 'int16':test.smallint('160'),
- 'int32':test.integer('+320i32'),
- 'int64':test.bigint('640'),
-
- 'int8_2':test.tinyint('-80'),
- 'int16_2':test.smallint('-160i16'),
- 'int32_2':test.integer('-320'),
- 'int64_2':test.bigint('-640i64'),
-
- 'int64_min':test.bigint('-9223372036854775808'),
-
- 'int8_3':test.tinyint(test.tinyint('+80i8')),
- 'int16_3':test.smallint(test.smallint('160')),
- 'int32_3':test.integer(test.integer('+320i32')),
- 'int64_3':test.bigint(test.bigint('640'))
-};
+select 0 g, i, tinyint(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, tinyint(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, tinyint(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.4.query.sqlpp
new file mode 100644
index 0000000..5c4363a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.4.query.sqlpp
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+let
+test = [
+ '160',
+ '-160i16',
+ test.smallint('160'),
+ int8(8),
+ int32(32),
+ int64(64),
+ float(1.25),
+ double(2.25),
+ false,
+ true,
+ '160.25',
+ '160.5',
+ '160.75',
+ '-160.25',
+ '-160.5',
+ '-160.75',
+ '16.125e1',
+ '-1612.5e-1',
+ /* out of range */
+ int32(99999),
+ int64(99999),
+ float(99999),
+ double(99999),
+ 'INF',
+ '-INF',
+ 'NaN'
+],
+testNull = [
+ null,
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
+
+select 0 g, i, smallint(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, smallint(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, smallint(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.5.query.sqlpp
new file mode 100644
index 0000000..d0b9f33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.5.query.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+ '+320i32',
+ '-320',
+ test.integer('+320i32'),
+ int8(8),
+ int16(16),
+ int64(64),
+ float(1.25),
+ double(2.25),
+ false,
+ true,
+ '320.25',
+ '320.5',
+ '320.75',
+ '-320.25',
+ '-320.5',
+ '-320.75',
+ '32.125e1',
+ '-3212.5e-1',
+ /* out of range */
+ int64(9999999999),
+ float(9999999999),
+ double(9999999999),
+ 'INF',
+ '-INF',
+ 'NaN'
+],
+testNull = [
+ null,
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
+
+select 0 g, i, integer(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, integer(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, integer(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.6.query.sqlpp
new file mode 100644
index 0000000..af0919f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_01/int_01.6.query.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+ '640',
+ '-640i64',
+ '-9223372036854775808',
+ test.bigint('640'),
+ int8(8),
+ int16(16),
+ int32(32),
+ float(1.25),
+ double(2.25),
+ false,
+ true,
+ '640.25',
+ '640.5',
+ '640.75',
+ '-640.25',
+ '-640.5',
+ '-640.75',
+ '64.125e1',
+ '-6412.5e-1',
+ /* out of range */
+ float('1e100'),
+ double('1e100'),
+ 'INF',
+ '-INF',
+ 'NaN'
+],
+testNull = [
+ null,
+ '@#!',
+ datetime('1987-11-19T10:50:56Z'),
+ date('2020-01-02'),
+ time('01:02:03'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
+
+select 0 g, i, bigint(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, bigint(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, bigint(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.1.query.sqlpp
new file mode 100644
index 0000000..83919c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": tinyint("@#!"),
+ "null_1": tinyint(datetime('1987-11-19T10:50:56Z')),
+ "null_2": tinyint(date('2020-01-02')),
+ "null_3": tinyint(time("01:02:03")),
+ "null_4": tinyint([]),
+ "null_5": tinyint({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.2.query.sqlpp
new file mode 100644
index 0000000..7d39f17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": smallint("@#!"),
+ "null_1": smallint(datetime('1987-11-19T10:50:56Z')),
+ "null_2": smallint(date('2020-01-02')),
+ "null_3": smallint(time("01:02:03")),
+ "null_4": smallint([]),
+ "null_5": smallint({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.3.query.sqlpp
new file mode 100644
index 0000000..c4e2b6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": integer("@#!"),
+ "null_1": integer(datetime('1987-11-19T10:50:56Z')),
+ "null_2": integer(date('2020-01-02')),
+ "null_3": integer(time("01:02:03")),
+ "null_4": integer([]),
+ "null_5": integer({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.4.query.sqlpp
new file mode 100644
index 0000000..2900811
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/int_02/int_02.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": bigint("@#!"),
+ "null_1": bigint(datetime('1987-11-19T10:50:56Z')),
+ "null_2": bigint(date('2020-01-02')),
+ "null_3": bigint(time("01:02:03")),
+ "null_4": bigint([]),
+ "null_5": bigint({})
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/string_01/string_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/string_01/string_01.3.query.sqlpp
index 81fbd1f..6181751 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/string_01/string_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/string_01/string_01.3.query.sqlpp
@@ -17,15 +17,42 @@
* under the License.
*/
-use test;
+let
+test = [
+ 'true',
+ 'false"',
+ int8('8'),
+ int16('16'),
+ int32('32'),
+ int64('64'),
+ float('1.25'),
+ double('2.5'),
+ true,
+ false,
+ datetime('2020-01-02T03:04:05Z'),
+ date('2021-01-02'),
+ time('01:02:03'),
+ duration('P30Y10M25DT13H12M50S'),
+ year_month_duration('P2Y'),
+ day_time_duration('P4S'),
+ uuid('02a199ca-bf58-412e-bd9f-60a0c975a8ac'),
+ base64('0123456789qwertyuiopasdfghjklzxcvbnmQWERTYUIOPLKJHGFDSAZXCVBNM+/')
+],
+testNull = [
+ null,
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-{
- 'string1':test.string('true'),
- 'string2':test.string('false"'),
- 'string3':test.string(test.int8('8')),
- 'string4':test.string(test.int16('16')),
- 'string5':test.string(test.int32('32')),
- 'string6':test.string(test.int64('64')),
- 'string7':test.string(test.float('1.25')),
- 'string8':test.string(test.double('2.5'))
-};
+select 0 g, i, string(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, string(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, string(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/string_02/string_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/string_02/string_02.1.query.sqlpp
new file mode 100644
index 0000000..006fa13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/string_02/string_02.1.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": string([]),
+ "null_1": string({})
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.3.query.sqlpp
index 1395d7d..24d2aad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.3.query.sqlpp
@@ -17,19 +17,50 @@
* under the License.
*/
-use test;
+let
+test = [
+ '10:50:56.200+05:00',
+ '10:50:56.200-10:15',
+ '10:50:56',
+ '10:50:56.200Z',
+ '23:59:59.999-13:30',
+ '00:00:00.000+14:45',
+ '12:59:00.019-01:00',
+ '12:59:00.01-01:00',
+ '12:59:00.019-01:00',
+ '12590001-0100',
+ '125900019+0100',
+ test.time('125900019+0100'),
+ datetime('2010-10-30T01:02:03Z')
+],
+testNull = [
+ null,
+ false,
+ true,
+ '@#!',
+ int8(0),
+ int16(0),
+ int32(0),
+ int64(0),
+ float(0),
+ double(0),
+ date('2020-01-02'),
+ duration('PT0H'),
+ year_month_duration('P0Y0M'),
+ day_time_duration('P0D'),
+ [],
+ {}
+],
+testMissing = [
+ missing
+]
-{
- 'time1':test.time('10:50:56.200+05:00'),
- 'time2':test.time('10:50:56.200-10:15'),
- 'time3':test.time('10:50:56'),
- 'time4':test.time('10:50:56.200Z'),
- 'time5':test.time('23:59:59.999-13:30'),
- 'time6':test.time('00:00:00.000+14:45'),
- 'time7':test.time('12:59:00.019-01:00'),
- 'time8':test.time('12:59:00.01-01:00'),
- 'time9':test.time('12:59:00.019-01:00'),
- 'time10':test.time('12590001-0100'),
- 'time11':test.time('125900019+0100'),
- 'time12':test.time(test.time('125900019+0100'))
-};
+select 0 g, i, time(test[i]) actual
+from range(0, len(test)-1) i
+union all
+select 1 g, i, null expected, time(testNull[i]) actual
+from range(0, len(testNull)-1) i
+union all
+select 2 g, i, true expected, time(testMissing[i]) is missing actual
+from range(0, len(testMissing)-1) i
+order by g, i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
new file mode 100644
index 0000000..37eba54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_01/time_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+let
+test = [
+ '11/58/59',
+ '58-57-10'
+],
+fmt = [
+ 'hh/mm/ss',
+ 'ss-mm-hh'
+]
+
+select i, time(test[i], fmt[i]) actual
+from range(0, len(test)-1) i
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
new file mode 100644
index 0000000..b97ec3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/time_02/time_02.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": time("@#!"),
+ "null_1": time(false),
+ "null_2": time(0),
+ "null_3": time(date("2020-01-02")),
+ "null_4": time([]),
+ "null_5": time({}),
+ "null_6": time("11:58:59", "INVALID_FORMAT")
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_01/uuid_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_01/uuid_01.3.query.sqlpp
index 53d24c5..8bb6cda 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_01/uuid_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_01/uuid_01.3.query.sqlpp
@@ -17,8 +17,6 @@
* under the License.
*/
-use test;
-
{
'uuid1':test.uuid('02a199ca-bf58-412e-bd9f-60a0c975a8ac'),
'uuid2':test.uuid('8cea25ab-55f8-467e-929d-94888f754832'),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_02/uuid_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_02/uuid_02.1.query.sqlpp
index a11fd8b..944830f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_02/uuid_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_02/uuid_02.1.query.sqlpp
@@ -17,4 +17,11 @@
* under the License.
*/
-uuid('02a199ca-bf58-412e-bd9f-60a0c975a8a-'); // Invalid format
+// requesttype=application/json
+// param max-warnings:json=100
+
+{
+ "null_0": uuid('02a199ca-bf58-412e-bd9f-60a0c975a8a-'), // Invalid format -> NULL
+ "null_1": uuid('12345') // Invalid length -> NULL
+};
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_03/uuid_03.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_03/uuid_03.1.query.sqlpp
deleted file mode 100644
index a9fb88e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/constructor/uuid/uuid_03/uuid_03.1.query.sqlpp
+++ /dev/null
@@ -1,20 +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.
- */
-
-uuid('12345'); // Invalid length
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/bad-type-ddl/bad-type-ddl.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/bad-type-ddl/bad-type-ddl.2.ddl.sqlpp
new file mode 100644
index 0000000..effd74b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/bad-type-ddl/bad-type-ddl.2.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Duplicate field name
+ * Expected Res : Error
+ */
+
+drop dataverse experiments if exists;
+create dataverse experiments;
+use experiments;
+
+create type t1 as {
+ c : string,
+ c : string
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.1.ddl.sqlpp
index 55f3604..4776ab4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.1.ddl.sqlpp
@@ -25,7 +25,7 @@
CREATE FUNCTION listMetadata() {
SELECT "Dataset" AS en, d.DatasetName, d.DatatypeDataverseName, d.DatatypeName,
- d.MetatypeDataverseName, d.MetatypeName
+ d.MetatypeDataverseName, d.MetatypeName, d.InternalDetails.PrimaryKey, d.InternalDetails.KeySourceIndicator
FROM Metadata.`Dataset` d
WHERE d.DataverseName = "test"
UNION ALL
@@ -52,11 +52,11 @@
) CLOSED TYPE PRIMARY KEY c_custkey;
CREATE DATASET A_Customers_Open(
- c_custkey integer not unknown,
- c_name string not unknown,
+ c_custkey integer /* auto NOT UNKNOWN if used in PRIMARY KEY */,
+ c_name string /* auto NOT UNKNOWN if used in PRIMARY KEY */,
c_phone string,
c_comment string
-) OPEN TYPE PRIMARY KEY c_custkey;
+) OPEN TYPE PRIMARY KEY c_custkey, c_name;
/* External datasets */
@@ -117,16 +117,16 @@
PRIMARY KEY c_custkey ;
CREATE DATASET C_Customers_Meta_Closed(
- c_custkey integer not unknown,
+ c_custkey integer /* auto NOT UNKNOWN if used in PRIMARY KEY */,
c_name string not unknown,
c_phone string,
c_comment string
) CLOSED TYPE
WITH META(
- c_x integer not unknown,
+ c_x integer /* auto NOT UNKNOWN if used in PRIMARY KEY */,
c_y integer
) CLOSED TYPE
-PRIMARY KEY c_custkey;
+PRIMARY KEY c_custkey, META().c_x;
CREATE DATASET C_Customers_Meta_Open(
c_custkey integer not unknown,
@@ -135,7 +135,7 @@
c_comment string
) OPEN TYPE
WITH META(
- c_x integer not unknown,
- c_y integer
+ c_x integer /* auto NOT UNKNOWN if used in PRIMARY KEY */,
+ c_y integer /* auto NOT UNKNOWN if used in PRIMARY KEY */
) OPEN TYPE
-PRIMARY KEY c_custkey;
+PRIMARY KEY META().c_x, META().c_y;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.6.ddl.sqlpp
new file mode 100644
index 0000000..1c5f520
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.6.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* Create dataset that attempts to declare the same field twice
+ in inline type definition */
+
+USE test;
+
+CREATE DATASET Cust4X(
+ c_custkey integer not unknown,
+ c_name string,
+ c_name string
+) PRIMARY KEY c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-index/create-index-6/create-index-6.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-index/create-index-6/create-index-6.1.ddl.sqlpp
new file mode 100644
index 0000000..bcf65d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-index/create-index-6/create-index-6.1.ddl.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * 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 : Create index on a view
+ * Expected Res : Failure
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.LineItemType as
+ closed {
+ l_orderkey : bigint,
+ l_partkey : bigint,
+ l_suppkey : bigint,
+ l_linenumber : bigint,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey;
+
+create view LineItemView1 as LineItem;
+
+--- Failure: cannot create index on a view
+
+create index idx1 on LineItemView1(l_partkey : bigint);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-index/create-index-6/create-index-6.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-index/create-index-6/create-index-6.2.ddl.sqlpp
new file mode 100644
index 0000000..e86ce88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-index/create-index-6/create-index-6.2.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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 : Create index on a typed view
+ * Expected Res : Failure
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.LineItemType as
+ open {
+ l_orderkey : bigint
+};
+
+create type test.LineItemType2 as
+ closed {
+ l_orderkey : bigint?,
+ l_partkey : bigint?,
+ l_suppkey : bigint?,
+ l_linenumber : bigint?,
+ l_quantity : double?,
+ l_extendedprice : double?,
+ l_discount : double?,
+ l_tax : double?,
+ l_returnflag : string?,
+ l_linestatus : string?,
+ l_shipdate : string?,
+ l_commitdate : string?,
+ l_receiptdate : string?,
+ l_shipinstruct : string?,
+ l_shipmode : string?,
+ l_comment : string?
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey;
+
+create view LineItemView2(LineItemType2) default null as LineItem;
+
+--- Failure: cannot create index on a typed view
+
+create index idx1 on LineItemView2(l_partkey);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.004.query.sqlpp
index 5834c9c..14f2ebd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.004.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.004.query.sqlpp
@@ -21,4 +21,5 @@
SET `import-private-functions` `true`;
FROM DUMP_INDEX("test", "ds1", "idx1") AS v
-SELECT VALUE v;
\ No newline at end of file
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.005.query.sqlpp
index 0518ef0..9b674de 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.005.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-bad-fields/index-bad-fields.005.query.sqlpp
@@ -21,4 +21,5 @@
SET `import-private-functions` `true`;
FROM DUMP_INDEX("test", "ds2", "idx2") AS v
-SELECT VALUE v;
\ No newline at end of file
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.000.ddl.sqlpp
new file mode 100644
index 0000000..a26a005
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.000.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE DATASET ds1(id int not unknown, typed_f1 string, typed_f2 int) OPEN TYPE PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.001.ddl.sqlpp
new file mode 100644
index 0000000..ac134c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.001.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+// can only use CAST (DEFAULT NULL) with BTREE
+CREATE INDEX idx ON ds1(UNNEST a : string) CAST (DEFAULT NULL);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.002.ddl.sqlpp
new file mode 100644
index 0000000..afdd07b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.002.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+// cannot use ENFORCED and CAST (DEFAULT NULL)
+CREATE INDEX idx ON ds1(f: int?) ENFORCED CAST (DEFAULT NULL);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.003.ddl.sqlpp
new file mode 100644
index 0000000..ce5bf39
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.003.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+// the target type of the CAST is not specified
+CREATE INDEX idx ON ds1(typed_f1) CAST (DEFAULT NULL);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.004.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.004.ddl.sqlpp
new file mode 100644
index 0000000..dc14260
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.004.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+// cannot override the type of a closed field without using CAST
+CREATE INDEX idx ON ds1(typed_f2: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.005.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.005.ddl.sqlpp
new file mode 100644
index 0000000..bbabd10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null-negative/index-cast-null-negative.005.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+// invalid datetime property name
+CREATE INDEX idx_dt ON ds1(f_dt: datetime) CAST (DEFAULT NULL invalid_date 'MM/DD/YYYY');
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.000.ddl.sqlpp
new file mode 100644
index 0000000..5de3a72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.000.ddl.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * 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 : test that CAST (DEFAULT NULL) casts to the target type such that NULL is produced for invalid input
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+CREATE TYPE t1 AS { id: int, s_f: string, d_f: double, i_f: int, b_f: boolean };
+CREATE TYPE t2 AS { id: int, s_f1: string, s_f2: string?, i_f: int? };
+CREATE DATASET ds1(t1) PRIMARY KEY id %with%;
+CREATE DATASET ds2(t1) PRIMARY KEY id %with%;
+CREATE DATASET ds3(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds4(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds5(t2) PRIMARY KEY id %with%;
+
+CREATE INDEX ds2_o_idx1 ON ds2(o_s_f: int, o_i_f: string) INCLUDE UNKNOWN KEY CAST (DEFAULT NULL);
+CREATE INDEX ds2_o_idx2 ON ds2(o_s_f: double, o_d_f: string) CAST (DEFAULT NULL);
+
+CREATE INDEX ds2_o_idx4 ON ds2(a.s_f: int) CAST (DEFAULT NULL);
+
+CREATE INDEX ds2_o_idx5 ON ds2(a.any_f: int) CAST (DEFAULT NULL);
+CREATE INDEX ds2_o_idx6 ON ds2(a.any_f: string) CAST (DEFAULT NULL);
+CREATE INDEX ds2_o_idx7 ON ds2(a.any_f: double) CAST (DEFAULT NULL);
+
+// temporal
+CREATE INDEX ds3_o_idx_f_dt ON ds3(f_dt: datetime) CAST (DEFAULT NULL);
+CREATE INDEX ds3_o_idx_f_d ON ds3(f_d: date) CAST (DEFAULT NULL);
+CREATE INDEX ds3_o_idx_f_t ON ds3(f_t: time) CAST (DEFAULT NULL);
+
+// temporal with format
+CREATE INDEX ds3_o_idx_f_dt_fmt ON ds3(f_dt_fmt: datetime) CAST (DEFAULT NULL datetime 'MM/DD/YYYY hh:mm:ss.nnna');
+CREATE INDEX ds3_o_idx_f_d_fmt ON ds3(f_d_fmt: date) CAST (DEFAULT NULL date 'MM/DD/YYYY');
+CREATE INDEX ds3_o_idx_f_t_fmt ON ds3(f_t_fmt: time) CAST (DEFAULT NULL time 'hh:mm:ss.nnna');
+
+CREATE INDEX ds3_o_idx_invalid_fmt ON ds3(f_d_fmt: date) CAST (DEFAULT NULL date 'invalid_format');
+
+// on closed fields
+CREATE INDEX idx1 ON ds5(s_f1);
+CREATE INDEX idx2 ON ds5(s_f1: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds5(s_f1: int) CAST (DEFAULT NULL);
+
+CREATE INDEX idx4 ON ds5(s_f2);
+CREATE INDEX idx5 ON ds5(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx6 ON ds5(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE INDEX idx7 ON ds5(i_f: int) CAST (DEFAULT NULL);
+CREATE INDEX idx8 ON ds5(i_f: string) CAST (DEFAULT NULL);
+
+CREATE INDEX idx_exc1 ON ds5(s_f2: int) EXCLUDE UNKNOWN KEY CAST (DEFAULT NULL);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.001.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.001.update.sqlpp
new file mode 100644
index 0000000..fac0b31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.001.update.sqlpp
@@ -0,0 +1,72 @@
+/*
+ * 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 ds1 [
+{"id": 1, "s_f": "s", "d_f": 1.5, "i_f": 1, "b_f": true, "o_s_f": "s", "o_d_f": 1.5, "o_i_f": 1, "o_b_f": true, "a": {"s_f": "s", "any_f": 1}},
+{"id": 2, "s_f": "2", "d_f": 2.5, "i_f": 2, "b_f": false, "o_s_f": "2", "o_d_f": 2.5, "o_i_f": 2, "o_b_f": false, "a": {"s_f": "2", "any_f": 1.5}},
+{"id": 3, "s_f": "3.5", "d_f": 3.5, "i_f": 3, "b_f": true, "o_s_f": "3.5", "o_d_f": 3.5, "o_i_f": 3, "o_b_f": true, "a": {"s_f": "3.5", "any_f": "1"}},
+{"id": 4, "s_f": "true", "d_f": 4.5, "i_f": 4, "b_f": false, "o_s_f": "true", "o_d_f": 4.5, "o_i_f": 4, "o_b_f": false, "a": {"s_f": "true", "any_f": "1.5"}},
+{"id": 5, "s_f": "false", "d_f": 5.5, "i_f": 5, "b_f": false, "o_s_f": "false", "o_d_f": 5.5, "o_i_f": 5, "o_b_f": false, "a": {"s_f": "false", "any_f": "str"}},
+{"id": 6, "s_f": "6", "d_f": 6.5, "i_f": 6, "b_f": false, "o_s_f": "6", "o_d_f": 6.5, "o_i_f": 6, "o_b_f": false, "a": {"s_f": "6", "any_f": true}},
+{"id": 7, "s_f": "7.5", "d_f": 7.5, "i_f": 7, "b_f": false, "o_s_f": "7.5", "o_d_f": 7.5, "o_i_f": 7, "o_b_f": false, "a": {"s_f": "7.5", "any_f": false}},
+{"id": 8, "s_f": "false", "d_f": 8.5, "i_f": 8, "b_f": false, "o_s_f": "false", "o_d_f": 8.5, "o_i_f": 8, "o_b_f": false, "a": {"s_f": "false", "any_f": [1,2]}},
+{"id": 9, "s_f": "false", "d_f": 9.5, "i_f": 9, "b_f": false}
+];
+
+INSERT INTO ds2 [
+{"id": 1, "s_f": "s", "d_f": 1.5, "i_f": 1, "b_f": true, "o_s_f": "s", "o_d_f": 1.5, "o_i_f": 1, "o_b_f": true, "a": {"s_f": "s", "any_f": 1}},
+{"id": 2, "s_f": "2", "d_f": 2.5, "i_f": 2, "b_f": false, "o_s_f": "2", "o_d_f": 2.5, "o_i_f": 2, "o_b_f": false, "a": {"s_f": "2", "any_f": 1.5}},
+{"id": 3, "s_f": "3.5", "d_f": 3.5, "i_f": 3, "b_f": true, "o_s_f": "3.5", "o_d_f": 3.5, "o_i_f": 3, "o_b_f": true, "a": {"s_f": "3.5", "any_f": "1"}},
+{"id": 4, "s_f": "true", "d_f": 4.5, "i_f": 4, "b_f": false, "o_s_f": "true", "o_d_f": 4.5, "o_i_f": 4, "o_b_f": false, "a": {"s_f": "true", "any_f": "1.5"}},
+{"id": 5, "s_f": "false", "d_f": 5.5, "i_f": 5, "b_f": false, "o_s_f": "false", "o_d_f": 5.5, "o_i_f": 5, "o_b_f": false, "a": {"s_f": "false", "any_f": "str"}},
+{"id": 6, "s_f": "6", "d_f": 6.5, "i_f": 6, "b_f": false, "o_s_f": "6", "o_d_f": 6.5, "o_i_f": 6, "o_b_f": false, "a": {"s_f": "6", "any_f": true}},
+{"id": 7, "s_f": "7.5", "d_f": 7.5, "i_f": 7, "b_f": false, "o_s_f": "7.5", "o_d_f": 7.5, "o_i_f": 7, "o_b_f": false, "a": {"s_f": "7.5", "any_f": false}},
+{"id": 8, "s_f": "false", "d_f": 8.5, "i_f": 8, "b_f": false, "o_s_f": "false", "o_d_f": 8.5, "o_i_f": 8, "o_b_f": false, "a": {"s_f": "false", "any_f": [1,2]}},
+{"id": 9, "s_f": "false", "d_f": 9.5, "i_f": 9, "b_f": false}
+];
+
+INSERT INTO ds3 [
+{"id": 1, 'f_dt':'2020-02-20T01:40:41.001', 'f_d':'2020-02-20', 'f_t':'01:40:41.001', 'f_dt_fmt':'02/20/2020 01:40:41.001am', 'f_d_fmt':'02/20/2020', 'f_t_fmt':'01:40:41.001am'},
+{"id": 2, 'f_dt':'2021-11-25T22:12:51.999', 'f_d':'2021-11-25', 'f_t':'22:12:51.999', 'f_dt_fmt':'11/25/2021 10:12:51.999pm', 'f_d_fmt':'11/25/2021', 'f_t_fmt':'10:12:51.999pm'},
+/* Invalid value */
+{"id": 3, 'f_dt':'a', 'f_d':'b', 'f_t':'c', 'f_dt_fmt':'a', 'c_date':'b', 'f_t_fmt':'c'},
+{"id": 4, 'f_dt':'2019-01-20T14:40:41.001', 'f_d':'2019-01-20', 'f_t':'14:40:41.001', 'f_dt_fmt':'01/20/2019 02:40:41.001pm', 'f_d_fmt':'01/20/2019', 'f_t_fmt':'02:40:41.001pm'},
+{"id": 5, 'f_dt':null, 'f_d':null, 'f_t':null, 'f_dt_fmt':null, 'f_d_fmt':null, 'f_t_fmt':null},
+{"id": 6}
+];
+
+INSERT INTO ds4 [
+{"id": 1, 'f_dt':'2020-02-20T01:40:41.001', 'f_d':'2020-02-20', 'f_t':'01:40:41.001', 'f_dt_fmt':'02/20/2020 01:40:41.001am', 'f_d_fmt':'02/20/2020', 'f_t_fmt':'01:40:41.001am'},
+{"id": 2, 'f_dt':'2021-11-25T22:12:51.999', 'f_d':'2021-11-25', 'f_t':'22:12:51.999', 'f_dt_fmt':'11/25/2021 10:12:51.999pm', 'f_d_fmt':'11/25/2021', 'f_t_fmt':'10:12:51.999pm'},
+/* Invalid value */
+{"id": 3, 'f_dt':'a', 'f_d':'b', 'f_t':'c', 'f_dt_fmt':'a', 'c_date':'b', 'f_t_fmt':'c'},
+{"id": 4, 'f_dt':'2019-01-20T14:40:41.001', 'f_d':'2019-01-20', 'f_t':'14:40:41.001', 'f_dt_fmt':'01/20/2019 02:40:41.001pm', 'f_d_fmt':'01/20/2019', 'f_t_fmt':'02:40:41.001pm'},
+{"id": 5, 'f_dt':null, 'f_d':null, 'f_t':null, 'f_dt_fmt':null, 'f_d_fmt':null, 'f_t_fmt':null},
+{"id": 6}
+];
+
+INSERT INTO ds5 [
+{"id": 1, "s_f1": "s", "s_f2": "s", "i_f": 1 },
+{"id": 2, "s_f1": "2", "s_f2": "2", "i_f": 2 },
+{"id": 3, "s_f1": "3.5", "s_f2": "3.5", "i_f": 3 },
+{"id": 4, "s_f1": "4", "s_f2": "4", "i_f": 4 },
+{"id": 5, "s_f1": "5", "s_f2": null, "i_f": null },
+{"id": 6, "s_f1": "" }
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.002.ddl.sqlpp
new file mode 100644
index 0000000..8ddbebc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.002.ddl.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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 the bulk load path of CREATE INDEX
+ */
+
+USE test;
+
+CREATE INDEX ds1_o_idx1 ON ds1(o_s_f: int, o_i_f: string) INCLUDE UNKNOWN KEY CAST(DEFAULT NULL);
+CREATE INDEX ds1_o_idx2 ON ds1(o_s_f: double, o_d_f: string) CAST(DEFAULT NULL);
+
+CREATE INDEX ds1_o_idx4 ON ds1(a.s_f: int) CAST(DEFAULT NULL);
+
+CREATE INDEX ds1_o_idx5 ON ds1(a.any_f: int) CAST(DEFAULT NULL);
+CREATE INDEX ds1_o_idx6 ON ds1(a.any_f: string) CAST(DEFAULT NULL);
+CREATE INDEX ds1_o_idx7 ON ds1(a.any_f: double) CAST(DEFAULT NULL);
+
+// temporal
+CREATE INDEX ds4_o_idx_f_dt ON ds4(f_dt: datetime) CAST (DEFAULT NULL);
+CREATE INDEX ds4_o_idx_f_d ON ds4(f_d: date) CAST (DEFAULT NULL);
+CREATE INDEX ds4_o_idx_f_t ON ds4(f_t: time) CAST (DEFAULT NULL);
+
+// temporal with format
+CREATE INDEX ds4_o_idx_f_dt_fmt ON ds4(f_dt_fmt: datetime) CAST (DEFAULT NULL datetime 'MM/DD/YYYY hh:mm:ss.nnna');
+CREATE INDEX ds4_o_idx_f_d_fmt ON ds4(f_d_fmt: date) CAST (DEFAULT NULL date 'MM/DD/YYYY');
+CREATE INDEX ds4_o_idx_f_t_fmt ON ds4(f_t_fmt: time) CAST (DEFAULT NULL time 'hh:mm:ss.nnna');
+
+CREATE INDEX ds4_o_idx_invalid_fmt ON ds4(f_d_fmt: date) CAST (DEFAULT NULL date 'invalid_format');
+
+// on closed fields
+CREATE INDEX idx9 ON ds5(s_f1);
+CREATE INDEX idx10 ON ds5(s_f1: string) CAST (DEFAULT NULL);
+CREATE INDEX idx11 ON ds5(s_f1: int) CAST (DEFAULT NULL);
+
+CREATE INDEX idx12 ON ds5(s_f2);
+CREATE INDEX idx13 ON ds5(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx14 ON ds5(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE INDEX idx15 ON ds5(i_f: int) CAST (DEFAULT NULL);
+CREATE INDEX idx16 ON ds5(i_f: string) CAST (DEFAULT NULL);
+
+CREATE INDEX idx_exc2 ON ds5(s_f2: int) EXCLUDE UNKNOWN KEY CAST (DEFAULT NULL);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.003.query.sqlpp
new file mode 100644
index 0000000..3106b2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.003.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds2", "ds2_o_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.004.query.sqlpp
new file mode 100644
index 0000000..31e5fcd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.004.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds2", "ds2_o_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.005.query.sqlpp
new file mode 100644
index 0000000..1dee7d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.005.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds2", "ds2_o_idx4") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.006.query.sqlpp
new file mode 100644
index 0000000..6401d91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.006.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds2", "ds2_o_idx5") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.007.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.007.query.sqlpp
new file mode 100644
index 0000000..12cc91f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.007.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds2", "ds2_o_idx6") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.008.query.sqlpp
new file mode 100644
index 0000000..ee605f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.008.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds2", "ds2_o_idx7") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.009.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.009.query.sqlpp
new file mode 100644
index 0000000..7f0ddee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.009.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "ds1_o_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.010.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.010.query.sqlpp
new file mode 100644
index 0000000..afa0afa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.010.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "ds1_o_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.011.query.sqlpp
new file mode 100644
index 0000000..db14e84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.011.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "ds1_o_idx4") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.012.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.012.query.sqlpp
new file mode 100644
index 0000000..d7d6246
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.012.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "ds1_o_idx5") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.013.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.013.query.sqlpp
new file mode 100644
index 0000000..c072a76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.013.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "ds1_o_idx6") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.014.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.014.query.sqlpp
new file mode 100644
index 0000000..086720b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.014.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "ds1_o_idx7") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.015.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.015.query.sqlpp
new file mode 100644
index 0000000..e3c5f25
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.015.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+// check the index metadata
+USE test;
+
+FROM Metadata.`Index` v WHERE v.DatasetName = 'ds2' AND v.IndexName = 'ds2_o_idx2' SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.016.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.016.query.sqlpp
new file mode 100644
index 0000000..8881cb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.016.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds3", "ds3_o_idx_f_d_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.017.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.017.query.sqlpp
new file mode 100644
index 0000000..558f3b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.017.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds3", "ds3_o_idx_f_t_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.018.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.018.query.sqlpp
new file mode 100644
index 0000000..bf99064
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.018.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds3", "ds3_o_idx_f_dt_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.019.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.019.query.sqlpp
new file mode 100644
index 0000000..c37d984
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.019.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds3", "ds3_o_idx_f_d") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.020.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.020.query.sqlpp
new file mode 100644
index 0000000..e901b04
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.020.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds3", "ds3_o_idx_f_t") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.021.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.021.query.sqlpp
new file mode 100644
index 0000000..151d016
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.021.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds3", "ds3_o_idx_f_dt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.022.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.022.query.sqlpp
new file mode 100644
index 0000000..3fcff2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.022.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds3", "ds3_o_idx_invalid_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.023.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.023.query.sqlpp
new file mode 100644
index 0000000..2c6339d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.023.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds4", "ds4_o_idx_f_d_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.024.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.024.query.sqlpp
new file mode 100644
index 0000000..be5e2ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.024.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds4", "ds4_o_idx_f_t_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.025.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.025.query.sqlpp
new file mode 100644
index 0000000..99551de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.025.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds4", "ds4_o_idx_f_dt_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.026.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.026.query.sqlpp
new file mode 100644
index 0000000..7f407ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.026.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds4", "ds4_o_idx_f_d") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.027.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.027.query.sqlpp
new file mode 100644
index 0000000..d75187f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.027.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds4", "ds4_o_idx_f_t") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.028.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.028.query.sqlpp
new file mode 100644
index 0000000..6f2eb21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.028.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds4", "ds4_o_idx_f_dt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.029.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.029.query.sqlpp
new file mode 100644
index 0000000..e4d5557
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.029.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds4", "ds4_o_idx_invalid_fmt") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.030.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.030.query.sqlpp
new file mode 100644
index 0000000..aeaffe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.030.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+// check the index metadata
+USE test;
+
+FROM Metadata.`Index` v
+WHERE v.DatasetName = 'ds3' AND v.IsPrimary = false
+SELECT v.IndexName, v.`Cast`
+ORDER BY v.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.031.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.031.query.sqlpp
new file mode 100644
index 0000000..437de9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.031.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+// check the index metadata
+USE test;
+
+FROM Metadata.`Index` v
+WHERE v.DatasetName = 'ds4' AND v.IsPrimary = false
+SELECT v.IndexName, v.`Cast`
+ORDER BY v.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.032.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.032.query.sqlpp
new file mode 100644
index 0000000..13b56e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.032.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.033.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.033.query.sqlpp
new file mode 100644
index 0000000..4e3ea15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.033.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.034.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.034.query.sqlpp
new file mode 100644
index 0000000..43bd912
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.034.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx3") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.035.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.035.query.sqlpp
new file mode 100644
index 0000000..3129e17
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.035.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx4") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.036.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.036.query.sqlpp
new file mode 100644
index 0000000..7dac7ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.036.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx5") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.037.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.037.query.sqlpp
new file mode 100644
index 0000000..442e4c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.037.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx6") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.038.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.038.query.sqlpp
new file mode 100644
index 0000000..0e16524
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.038.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx7") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.039.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.039.query.sqlpp
new file mode 100644
index 0000000..73bcd31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.039.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx8") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.040.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.040.query.sqlpp
new file mode 100644
index 0000000..b8683cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.040.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx9") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.041.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.041.query.sqlpp
new file mode 100644
index 0000000..a694667
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.041.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx10") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.042.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.042.query.sqlpp
new file mode 100644
index 0000000..ba593c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.042.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx11") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.043.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.043.query.sqlpp
new file mode 100644
index 0000000..c5931ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.043.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx12") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.044.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.044.query.sqlpp
new file mode 100644
index 0000000..d168fb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.044.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx13") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.045.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.045.query.sqlpp
new file mode 100644
index 0000000..090b384
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.045.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx14") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.046.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.046.query.sqlpp
new file mode 100644
index 0000000..1d66487
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.046.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx15") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.047.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.047.query.sqlpp
new file mode 100644
index 0000000..b6455d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.047.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx16") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.048.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.048.query.sqlpp
new file mode 100644
index 0000000..e551576
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.048.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx_exc1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.049.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.049.query.sqlpp
new file mode 100644
index 0000000..6046ab4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.049.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds5", "idx_exc2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.050.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.050.query.sqlpp
new file mode 100644
index 0000000..cc547f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.050.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+FROM Metadata.`Index` v
+WHERE v.DatasetName = 'ds5' AND v.IsPrimary = false
+SELECT v.IndexName, v.SearchKey, v.SearchKeyType, v.`Cast`
+ORDER BY v.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.999.ddl.sqlpp
new file mode 100644
index 0000000..86a1b59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/index-cast-null/index-cast-null.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.1.ddl.sqlpp
new file mode 100644
index 0000000..1b1210f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Empty view name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create view `` as
+ select r from range(0, 2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.2.ddl.sqlpp
new file mode 100644
index 0000000..a3a0172
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.2.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * View name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create view ` a` as
+ select r from range(0, 2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.3.ddl.sqlpp
new file mode 100644
index 0000000..2774fe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.3.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Invalid dataverse name in CREATE VIEW -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create view ` invalid`.a as
+ select r from range(0, 2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.4.ddl.sqlpp
new file mode 100644
index 0000000..d38df4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.4.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Invalid dataverse name in CREATE VIEW -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create view `a/b`.a as
+ select r from range(0, 2) r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.5.ddl.sqlpp
new file mode 100644
index 0000000..6acc680
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-view-name/invalid-view-name.5.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Invalid dataverse name in DROP VIEW -> Error
+ */
+
+drop dataverse c.d if exists;
+create dataverse c.d;
+
+use c.d;
+
+create view e as
+ select r from range(0, 2) r;
+
+drop view `c/d`.e;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.01.ddl.sqlpp
new file mode 100644
index 0000000..89abc08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.01.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+// test that secondary indexes can include or exclude entries that have unknown keys
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE DATASET ds1(id int not unknown) OPEN TYPE PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.02.update.sqlpp
new file mode 100644
index 0000000..0ac8895
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.02.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 ds1 ([
+{"id": 1, "int_f": 1 , "str_f": "1"},
+{"id": 2, "int_f": null, "str_f": "2"},
+{"id": 3 , "str_f": "3"},
+{"id": 4, "int_f": 4 , "str_f": null},
+{"id": 5, "int_f": 5 },
+{"id": 6, "int_f": null },
+{"id": 7 , "str_f": null},
+{"id": 8, "int_f": null, "str_f": null},
+{"id": 9 ,"other_f": "other" }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.03.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.03.ddl.sqlpp
new file mode 100644
index 0000000..44546cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.03.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+CREATE INDEX exclude_unknown_idx1 ON ds1(int_f: int) exclude unknown key;
+CREATE INDEX exclude_unknown_idx2 ON ds1(int_f: int, str_f: string) exclude unknown key;
+
+CREATE INDEX include_unknown_idx1 ON ds1(int_f: int) include unknown key;
+CREATE INDEX include_unknown_idx2 ON ds1(int_f: int, str_f: string) include unknown key;
+
+CREATE INDEX idx1 ON ds1(int_f: int);
+CREATE INDEX idx2 ON ds1(int_f: int, str_f: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.04.query.sqlpp
new file mode 100644
index 0000000..02c8708
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.04.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.05.query.sqlpp
new file mode 100644
index 0000000..826c8a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.05.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.06.query.sqlpp
new file mode 100644
index 0000000..b3ed16f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.06.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.07.query.sqlpp
new file mode 100644
index 0000000..91543ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.07.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.08.query.sqlpp
new file mode 100644
index 0000000..14f2ebd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.08.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.09.query.sqlpp
new file mode 100644
index 0000000..845ee14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.09.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.10.query.sqlpp
new file mode 100644
index 0000000..edc8ece
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.10.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT COUNT(*) AS cnt FROM ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.11.query.sqlpp
new file mode 100644
index 0000000..8f4c38b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.11.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT i.DataverseName, i.IndexName, i.ExcludeUnknownKey FROM Metadata.`Index` i WHERE i.DataverseName = "test" ORDER BY i.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/create-index-unknown-key/create-index-unknown-key.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.01.ddl.sqlpp
new file mode 100644
index 0000000..9d3f4bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+// test that secondary indexes can include or exclude entries that have unknown keys
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE DATASET ds1(id int not unknown) OPEN TYPE PRIMARY KEY id;
+
+CREATE INDEX exclude_unknown_idx1 ON ds1(int_f: int) exclude unknown key;
+CREATE INDEX exclude_unknown_idx2 ON ds1(int_f: int, str_f: string) exclude unknown key;
+
+CREATE INDEX include_unknown_idx1 ON ds1(int_f: int) include unknown key;
+CREATE INDEX include_unknown_idx2 ON ds1(int_f: int, str_f: string) include unknown key;
+
+CREATE INDEX idx1 ON ds1(int_f: int);
+CREATE INDEX idx2 ON ds1(int_f: int, str_f: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.02.update.sqlpp
new file mode 100644
index 0000000..0ac8895
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.02.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 ds1 ([
+{"id": 1, "int_f": 1 , "str_f": "1"},
+{"id": 2, "int_f": null, "str_f": "2"},
+{"id": 3 , "str_f": "3"},
+{"id": 4, "int_f": 4 , "str_f": null},
+{"id": 5, "int_f": 5 },
+{"id": 6, "int_f": null },
+{"id": 7 , "str_f": null},
+{"id": 8, "int_f": null, "str_f": null},
+{"id": 9 ,"other_f": "other" }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.04.query.sqlpp
new file mode 100644
index 0000000..02c8708
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.04.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.05.query.sqlpp
new file mode 100644
index 0000000..826c8a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.05.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.06.query.sqlpp
new file mode 100644
index 0000000..b3ed16f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.06.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.07.query.sqlpp
new file mode 100644
index 0000000..91543ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.07.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.08.query.sqlpp
new file mode 100644
index 0000000..14f2ebd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.08.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.09.query.sqlpp
new file mode 100644
index 0000000..845ee14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.09.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.10.query.sqlpp
new file mode 100644
index 0000000..edc8ece
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.10.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT COUNT(*) AS cnt FROM ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.11.query.sqlpp
new file mode 100644
index 0000000..8f4c38b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.11.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT i.DataverseName, i.IndexName, i.ExcludeUnknownKey FROM Metadata.`Index` i WHERE i.DataverseName = "test" ORDER BY i.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-into-index-unknown-key/insert-into-index-unknown-key.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.01.ddl.sqlpp
new file mode 100644
index 0000000..9d3f4bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+// test that secondary indexes can include or exclude entries that have unknown keys
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE DATASET ds1(id int not unknown) OPEN TYPE PRIMARY KEY id;
+
+CREATE INDEX exclude_unknown_idx1 ON ds1(int_f: int) exclude unknown key;
+CREATE INDEX exclude_unknown_idx2 ON ds1(int_f: int, str_f: string) exclude unknown key;
+
+CREATE INDEX include_unknown_idx1 ON ds1(int_f: int) include unknown key;
+CREATE INDEX include_unknown_idx2 ON ds1(int_f: int, str_f: string) include unknown key;
+
+CREATE INDEX idx1 ON ds1(int_f: int);
+CREATE INDEX idx2 ON ds1(int_f: int, str_f: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.02.update.sqlpp
new file mode 100644
index 0000000..12aaddb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.02.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+
+LOAD DATASET ds1 USING localfs((`path`=`asterix_nc1://data/misc/1.adm`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.04.query.sqlpp
new file mode 100644
index 0000000..02c8708
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.04.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.05.query.sqlpp
new file mode 100644
index 0000000..826c8a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.05.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.06.query.sqlpp
new file mode 100644
index 0000000..b3ed16f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.06.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.07.query.sqlpp
new file mode 100644
index 0000000..91543ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.07.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.08.query.sqlpp
new file mode 100644
index 0000000..14f2ebd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.08.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.09.query.sqlpp
new file mode 100644
index 0000000..845ee14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.09.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.10.query.sqlpp
new file mode 100644
index 0000000..edc8ece
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.10.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT COUNT(*) AS cnt FROM ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.11.query.sqlpp
new file mode 100644
index 0000000..8f4c38b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.11.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT i.DataverseName, i.IndexName, i.ExcludeUnknownKey FROM Metadata.`Index` i WHERE i.DataverseName = "test" ORDER BY i.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/load-into-index-unknown-key/load-into-index-unknown-key.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.01.ddl.sqlpp
new file mode 100644
index 0000000..fd4edb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.01.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+// test that secondary indexes can include or exclude entries that have unknown keys
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE DATASET ds1(id int not unknown) OPEN TYPE PRIMARY KEY id;
+
+CREATE INDEX exclude_unknown_idx1 ON ds1(int_f: int) EXCLUDE UNKNOWN KEY;
+CREATE INDEX exclude_unknown_idx2 ON ds1(int_f: int, str_f: string) EXCLUDE UNKNOWN KEY;
+
+CREATE INDEX include_unknown_idx1 ON ds1(int_f: int) INCLUDE UNKNOWN KEY;
+CREATE INDEX include_unknown_idx2 ON ds1(int_f: int, str_f: string) INCLUDE UNKNOWN KEY;
+
+CREATE INDEX idx1 ON ds1(int_f: int);
+CREATE INDEX idx2 ON ds1(int_f: int, str_f: string);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.02.update.sqlpp
new file mode 100644
index 0000000..ddd4565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.02.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+UPSERT INTO ds1 ([
+{"id": 1, "int_f": 1 , "str_f": "1"},
+{"id": 2, "int_f": null, "str_f": "2"},
+{"id": 3 , "str_f": "3"},
+{"id": 4, "int_f": 4 , "str_f": null},
+{"id": 5, "int_f": 5 },
+{"id": 6, "int_f": null },
+{"id": 7 , "str_f": null},
+{"id": 8, "int_f": null, "str_f": null},
+{"id": 9 ,"other_f": "other" }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.04.query.sqlpp
new file mode 100644
index 0000000..02c8708
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.04.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.05.query.sqlpp
new file mode 100644
index 0000000..826c8a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.05.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "exclude_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.06.query.sqlpp
new file mode 100644
index 0000000..b3ed16f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.06.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.07.query.sqlpp
new file mode 100644
index 0000000..91543ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.07.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "include_unknown_idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.08.query.sqlpp
new file mode 100644
index 0000000..14f2ebd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.08.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx1") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.09.query.sqlpp
new file mode 100644
index 0000000..845ee14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.09.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SET `import-private-functions` `true`;
+FROM DUMP_INDEX("test", "ds1", "idx2") AS v
+SELECT VALUE v
+ORDER BY v.values;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.10.query.sqlpp
new file mode 100644
index 0000000..edc8ece
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.10.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT COUNT(*) AS cnt FROM ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.11.query.sqlpp
new file mode 100644
index 0000000..8f4c38b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.11.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+SELECT i.DataverseName, i.IndexName, i.ExcludeUnknownKey FROM Metadata.`Index` i WHERE i.DataverseName = "test" ORDER BY i.IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-into-index-unknown-key/upsert-into-index-unknown-key.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.000.ddl.sqlpp
new file mode 100644
index 0000000..d6bc862
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.000.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+// bad case: no auth method and no endpoint provided
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.099.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-no-endpoint/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.000.ddl.sqlpp
new file mode 100644
index 0000000..a665d05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.000.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
+("endpoint"="%azure-endpoint%"),
+("container"="public-access"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.001.query.sqlpp
new file mode 100644
index 0000000..8ec9cc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+select count(*) `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.099.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.000.ddl.sqlpp
new file mode 100644
index 0000000..22385f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.000.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+// bad case: no auth method is provided
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
+("endpoint"="%azure-endpoint%"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.001.query.sqlpp
new file mode 100644
index 0000000..8ec9cc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+select count(*) `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.099.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-not-allowed/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-auth-methods/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-auth-methods/test.000.ddl.sqlpp
index b3c8bc4..a2f680f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-auth-methods/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-auth-methods/test.000.ddl.sqlpp
@@ -28,10 +28,10 @@
// bad case: more than one authentication method is provided at once
drop dataset test if exists;
CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
-("accountName"="%azureblob-accountname%"),
-("%azureblob-credentialsname-1%"="%azureblob-credentialsvalue-1%"),
-("%azureblob-credentialsname-2%"="%azureblob-credentialsvalue-2%"),
-("blobEndpoint"="%azureblob-endpoint%"),
+("accountName"="%azure-accountname%"),
+("%azure-credentialsname-1%"="%azure-credentialsvalue-1%"),
+("%azure-credentialsname-2%"="%azure-credentialsvalue-2%"),
+("endpoint"="%azure-endpoint%"),
("container"="playground"),
("definition"="json-data/reviews/single-line/json"),
("format"="json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.000.ddl.sqlpp
deleted file mode 100644
index 43db107..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.000.ddl.sqlpp
+++ /dev/null
@@ -1,35 +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.
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use test;
-
-drop type test if exists;
-create type test as open {
-};
-
-// bad case: no auth method is provided
-drop dataset test if exists;
-CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
-("blobEndpoint"="%azureblob-endpoint%"),
-("container"="playground"),
-("definition"="json-data/reviews/single-line/json"),
-("format"="json")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/valid-auth-methods/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/valid-auth-methods/test.000.ddl.sqlpp
index afeaeae..ca750fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/valid-auth-methods/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/valid-auth-methods/test.000.ddl.sqlpp
@@ -27,9 +27,9 @@
drop dataset test if exists;
CREATE EXTERNAL DATASET test(test) USING AZUREBLOB (
-("accountName"="%azureblob-accountname%"),
-("%azureblob-credentialsname%"="%azureblob-credentialsvalue%"),
-("blobEndpoint"="%azureblob-endpoint%"),
+("accountName"="%azure-accountname%"),
+("%azure-credentialsname%"="%azure-credentialsvalue%"),
+("endpoint"="%azure-endpoint%"),
("container"="playground"),
("definition"="json-data/reviews/single-line/json"),
("format"="json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.000.ddl.sqlpp
new file mode 100644
index 0000000..69e42c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.000.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as { id: int, age: int };
+
+drop dataset test1 if exists;
+CREATE EXTERNAL DATASET test1(test) USING %adapter% (
+%template%,
+("container"="bom-file-container"),
+("format"="csv"),
+("include"="*.csv"),
+("header"=False),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.001.query.sqlpp
new file mode 100644
index 0000000..5aa5580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.001.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+select value test1 from test1 order by id asc;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.099.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/csv/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.000.ddl.sqlpp
new file mode 100644
index 0000000..ad6513f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.000.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test1 if exists;
+CREATE EXTERNAL DATASET test1(test) USING %adapter% (
+%template%,
+("container"="bom-file-container"),
+("format"="json"),
+("include"="*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.001.query.sqlpp
new file mode 100644
index 0000000..5aa5580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.001.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+select value test1 from test1 order by id asc;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.099.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/json/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.000.ddl.sqlpp
new file mode 100644
index 0000000..956e835
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.000.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as { id: int, age: int };
+
+drop dataset test1 if exists;
+CREATE EXTERNAL DATASET test1(test) USING %adapter% (
+%template%,
+("container"="bom-file-container"),
+("format"="tsv"),
+("include"="*.tsv"),
+("header"=False),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.001.query.sqlpp
new file mode 100644
index 0000000..5aa5580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.001.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+select value test1 from test1 order by id asc;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.099.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/azure_blob_storage/auth-methods/invalid-no-auth/test.099.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/byte_order_mark/tsv/test.099.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp
index bc6a0c4..04fb1db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-1/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("exclude"="*.json"),
("exclude1"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp
index a8b2c02..321d5dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-2/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("exclude"="*.json"),
("exclude#"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp
index 5497e13..e26f01b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/bad-name-3/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("exclude"="*.json"),
("exclude#hello"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp
index 2540c6a..428ae5e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/both/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("include"="*.json"),
("exclude"="*.json")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp
index e2883bf..e47fbd8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-1/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("exclude"="*.?sv")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp
index 25ae5af..85664b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-2/test.000.ddl.sqlpp
@@ -29,9 +29,9 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#1"="data/mixed/json/extension*"),
-("exclude#100"="data/mixed/json/EXTENSION*")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#1"="mixed-data/reviews/json/extension*"),
+("exclude#100"="mixed-data/reviews/json/EXTENSION*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp
index edeabbd..b127693 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-3/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#100"="data/mixed/json*201?*")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#100"="mixed-data/reviews/json*201?*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp
index c2ad561..b3f0bcf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-4/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#1"="data/mixed/json*bye*")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#1"="mixed-data/reviews/json*bye*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp
index bf4b1a0..ff12e07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-5/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
-("exclude"="data/mixed/?sv*"),
-("exclude#1"="data/mixed/json/extension/hello-world-2018.json")
+("exclude"="mixed-data/reviews/?sv*"),
+("exclude#1"="mixed-data/reviews/json/extension/hello-world-2018.json")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp
index 3896068..d4804b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-6/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("exclude"="*.[abct][abcs][abcv]")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp
index f0088c4..adef5d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/exclude-all/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("exclude"="*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp
index 3419631..9d31cd3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-1/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("include"="*2018*.json")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp
index 2aac2b2..f676230 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-10/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="csv"),
("header"=false),
("include"="*[abc][.*")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp
index c54635f..acf651c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-11/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="csv"),
("header"=false),
("include"="*.[a-c][a-z][a-z**||\\\\&&--~~]")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp
index 6996c56..30d3c63 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-12/test.000.ddl.sqlpp
@@ -31,7 +31,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="csv"),
("header"=false),
("include"="[][!][^]]]]*[![*a-zA--&&^$||0-9B$\\*&&]*&&[^a-b||0--9][[[")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp
index d37f273..ac496ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-2/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("include"="*201?*.json")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp
index cb42507..a367c04 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-3/test.000.ddl.sqlpp
@@ -31,7 +31,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="csv"),
("header"=false),
("include"="*201?*.csv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp
index 1356218..44ce34e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-4/test.000.ddl.sqlpp
@@ -31,7 +31,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="tsv"),
("header"=false),
("include"="*201?*.tsv")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp
index 52facb3..ebaa926 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-5/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("include"="*bye*.json")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp
index 89bbcee..baa137c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-6/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
-("include"="data/mixed/json/*EXTENSION*")
+("include"="mixed-data/reviews/json/*EXTENSION*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp
index de90340..7a569d8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-7/test.000.ddl.sqlpp
@@ -29,8 +29,8 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
-("include"="data/mixed/json/NO-EXTENSION*"),
-("include#0"="data/mixed/json/EXTENSION*")
+("include"="mixed-data/reviews/json/NO-EXTENSION*"),
+("include#0"="mixed-data/reviews/json/EXTENSION*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp
index 86a669d..75087f3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-8/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="csv"),
("header"=false),
("include"="*.[!xyt][!xyz][!xyz]")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp
index 0dc8472..ec8e59a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-9/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="csv"),
("header"=false),
("include"="*.[a-c][a-z][a-z]")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp
index 0255e87..6ebbe1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/include-exclude/include-all/test.000.ddl.sqlpp
@@ -29,7 +29,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("include"="*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
index 71130e2..0aceee0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
@@ -30,7 +30,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("exclude"="*")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp
index 9edc6e6..8b136cc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/no-files-returned/include-no-files/test.000.ddl.sqlpp
@@ -30,7 +30,7 @@
CREATE EXTERNAL DATASET test(test) USING %adapter% (
%template%,
("container"="include-exclude"),
-("definition"="data/mixed/"),
+("definition"="mixed-data/reviews/"),
("format"="json"),
("include"="*.notRealExtension")
);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.01.ddl.sqlpp
new file mode 100644
index 0000000..4e21d51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.01.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Array access pushdown
+* Expected Res : Success
+* Date : July 20th 2021
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.02.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.02.query.sqlpp
new file mode 100644
index 0000000..3b645f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.02.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down get-item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT p.entities.urls[0].display_url
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.query.sqlpp
new file mode 100644
index 0000000..c268b2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down get-item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT p.entities.urls[0].display_url
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.04.query.sqlpp
new file mode 100644
index 0000000..43a6c07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.04.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down get-item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT p.entities.urls[0].display_url
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.query.sqlpp
new file mode 100644
index 0000000..8187033
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down get-item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT p.entities.urls[0].display_url
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.06.query.sqlpp
new file mode 100644
index 0000000..9e3e02b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.06.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Push down p.entities.urls
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT p.entities.urls[*].display_url
+FROM ParquetDataset p
+WHERE p.entities.urls IS NOT MISSING
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.query.sqlpp
new file mode 100644
index 0000000..bf42c9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Push down p.entities.urls
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT p.entities.urls[*].display_url
+FROM ParquetDataset p
+WHERE p.entities.urls IS NOT MISSING
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.08.query.sqlpp
new file mode 100644
index 0000000..0148449
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.08.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Push down p.entities.urls
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT p.entities.urls[*].display_url
+FROM ParquetDataset p
+WHERE p.entities.urls IS NOT MISSING
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.query.sqlpp
new file mode 100644
index 0000000..cbb85b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Push down p.entities.urls
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT p.entities.urls[*].display_url
+FROM ParquetDataset p
+WHERE p.entities.urls IS NOT MISSING
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.10.query.sqlpp
new file mode 100644
index 0000000..cab4c18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT urls.display_url
+FROM ParquetDataset p, p.entities.urls urls
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.query.sqlpp
new file mode 100644
index 0000000..131ae3d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT urls.display_url
+FROM ParquetDataset p, p.entities.urls urls
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.12.query.sqlpp
new file mode 100644
index 0000000..509806c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT urls.display_url
+FROM ParquetDataset p, p.entities.urls urls
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.query.sqlpp
new file mode 100644
index 0000000..79d5558
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT urls.display_url
+FROM ParquetDataset p, p.entities.urls urls
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.14.query.sqlpp
new file mode 100644
index 0000000..811074e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest in a subplan
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT VALUE COUNT(*)
+FROM ParquetDataset p
+WHERE (EVERY ht in p.entities.urls SATISFIES ht.display_url = "string");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.query.sqlpp
new file mode 100644
index 0000000..45ee208
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest in a subplan
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT VALUE COUNT(*)
+FROM ParquetDataset p
+WHERE (EVERY ht in p.entities.urls SATISFIES ht.display_url = "string");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.16.query.sqlpp
new file mode 100644
index 0000000..108524f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.16.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest in a subplan
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT VALUE COUNT(*)
+FROM ParquetDataset p
+WHERE (EVERY ht in p.entities.urls SATISFIES ht.display_url = "string");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.query.sqlpp
new file mode 100644
index 0000000..407fd74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down scan-collection from unnest in a subplan
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT VALUE COUNT(*)
+FROM ParquetDataset p
+WHERE (EVERY ht in p.entities.urls SATISFIES ht.display_url = "string");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.18.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.18.query.sqlpp
new file mode 100644
index 0000000..fd9b28b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.18.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down nested get_item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT VALUE p.place.bounding_box.coordinates[0][0][0]
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.query.sqlpp
new file mode 100644
index 0000000..e2fb357
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down nested get_item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT VALUE p.place.bounding_box.coordinates[0][0][0]
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.20.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.20.query.sqlpp
new file mode 100644
index 0000000..3c01e49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.20.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down nested get_item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT VALUE p.place.bounding_box.coordinates[0][0][0]
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.query.sqlpp
new file mode 100644
index 0000000..57435ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down nested get_item
+ * Expected Res : Success
+ * Date : July 20th 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT VALUE p.place.bounding_box.coordinates[0][0][0]
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.01.ddl.sqlpp
new file mode 100644
index 0000000..9db6efb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.01.ddl.sqlpp
@@ -0,0 +1,68 @@
+/*
+ * 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 : Field access pushdown
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*id_age.parquet"),
+ ("format" = "parquet")
+);
+
+CREATE EXTERNAL DATASET ParquetDataset2(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*id_age.parquet"),
+ ("format" = "parquet")
+);
+
+CREATE EXTERNAL DATASET ParquetDataset3(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*id_name_comment.parquet"),
+ ("format" = "parquet")
+);
+
+CREATE EXTERNAL DATASET ParquetDataset4(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.02.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.02.query.sqlpp
new file mode 100644
index 0000000..d457e75
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.02.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.query.sqlpp
new file mode 100644
index 0000000..18669ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.04.query.sqlpp
new file mode 100644
index 0000000..4e31474
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.04.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.query.sqlpp
new file mode 100644
index 0000000..2aac7ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.06.query.sqlpp
new file mode 100644
index 0000000..127a914
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.06.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT p1, p2.id
+FROM ParquetDataset p1, ParquetDataset2 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.query.sqlpp
new file mode 100644
index 0000000..96035dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT p1, p2.id
+FROM ParquetDataset p1, ParquetDataset2 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.08.query.sqlpp
new file mode 100644
index 0000000..2bdfa95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.08.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT p1, p2.id
+FROM ParquetDataset p1, ParquetDataset2 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.query.sqlpp
new file mode 100644
index 0000000..5a0f903
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT p1, p2.id
+FROM ParquetDataset p1, ParquetDataset2 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.10.query.sqlpp
new file mode 100644
index 0000000..8dc44f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.10.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Access different fields when joining two datasets
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT p1.age, p2.name
+FROM ParquetDataset p1, ParquetDataset3 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.query.sqlpp
new file mode 100644
index 0000000..066fb53
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Access different fields when joining two datasets
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT p1.age, p2.name
+FROM ParquetDataset p1, ParquetDataset3 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.12.query.sqlpp
new file mode 100644
index 0000000..99bb5fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.12.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Access different fields when joining two datasets
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT p1.age, p2.name
+FROM ParquetDataset p1, ParquetDataset3 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.query.sqlpp
new file mode 100644
index 0000000..a17d617
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Access different fields when joining two datasets
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT p1.age, p2.name
+FROM ParquetDataset p1, ParquetDataset3 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.14.query.sqlpp
new file mode 100644
index 0000000..190d40d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down children of common fields access when requesting nested values
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT p.user.id, p.user.name
+FROM ParquetDataset4 p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.query.sqlpp
new file mode 100644
index 0000000..32aaf12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down children of common fields access when requesting nested values
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT p.user.id, p.user.name
+FROM ParquetDataset4 p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.16.query.sqlpp
new file mode 100644
index 0000000..1213808
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.16.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down children of common fields access when requesting nested values
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT p.user.id, p.user.name
+FROM ParquetDataset4 p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.query.sqlpp
new file mode 100644
index 0000000..2a47449
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Push down children of common fields access when requesting nested values
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT p.user.id, p.user.name
+FROM ParquetDataset4 p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.18.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.18.query.sqlpp
new file mode 100644
index 0000000..e42c9ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.18.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 : Test SELECT COUNT(*)
+ * Expected Res : Success
+ * Date : July 25nd 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT VALUE COUNT(*)
+FROM ParquetDataset4 p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.query.sqlpp
new file mode 100644
index 0000000..71624af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 : Test SELECT COUNT(*)
+ * Expected Res : Success
+ * Date : July 25nd 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT VALUE COUNT(*)
+FROM ParquetDataset4 p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.20.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.20.query.sqlpp
new file mode 100644
index 0000000..f8d1312
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.20.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 : Test SELECT COUNT(*). Should get empty records
+ * Expected Res : Success
+ * Date : July 25nd 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT VALUE COUNT(*)
+FROM ParquetDataset4 p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.query.sqlpp
new file mode 100644
index 0000000..c433eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 : Test SELECT COUNT(*). Should get empty records
+ * Expected Res : Success
+ * Date : July 25nd 2021
+ */
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT VALUE COUNT(*)
+FROM ParquetDataset4 p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.query.sqlpp
new file mode 100644
index 0000000..9ee38e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 : Ensure `compiler.external.field.pushdown` is set by default
+ * Expected Res : Success
+ * Date : July 7th 2021
+ */
+
+USE test;
+
+EXPLAIN
+SELECT p.user.id, p.user.name
+FROM ParquetDataset4 p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.1.ddl.sqlpp
new file mode 100644
index 0000000..f1c4e81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Heterogeneous access DDL
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*heterogeneous*"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.2.query.sqlpp
new file mode 100644
index 0000000..1ffdf5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.2.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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 : Accessing a heterogeneous value
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+
+SELECT VALUE (
+ CASE WHEN is_array(p.arrayOrObject) THEN
+ p.arrayOrObject[*].text
+ ELSE
+ p.arrayOrObject.text
+ END
+)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.3.query.sqlpp
new file mode 100644
index 0000000..428c496
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.3.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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 : Accessing a heterogeneous value
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+EXPLAIN
+SELECT VALUE (
+ CASE WHEN is_array(p.arrayOrObject) THEN
+ p.arrayOrObject[*].text
+ ELSE
+ p.arrayOrObject.text
+ END
+)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.4.query.sqlpp
new file mode 100644
index 0000000..f75c1f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.4.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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 : Accessing a heterogeneous value
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT VALUE (
+ CASE WHEN is_array(p.arrayOrObject) THEN
+ p.arrayOrObject[*].text
+ ELSE
+ p.arrayOrObject.text
+ END
+)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.5.query.sqlpp
new file mode 100644
index 0000000..4336a0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.5.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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 : Accessing a heterogeneous value
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT VALUE (
+ CASE WHEN is_array(p.arrayOrObject) THEN
+ p.arrayOrObject[*].text
+ ELSE
+ p.arrayOrObject.text
+ END
+)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-type/invalid-type.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-type/invalid-type.1.ddl.sqlpp
new file mode 100644
index 0000000..ad6cd0e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/invalid-type/invalid-type.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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 : Test type validation for Parquet
+* Expected Res : ASX1161: Type 'ParquetType' contains declared fields, which is not supported for 'parquet' format
+* Date : August 19th 2021
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE ParquetType as {
+ id: string,
+ text: string
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*id_age.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..51378ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.2.query.sqlpp
new file mode 100644
index 0000000..8372ce6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+SELECT p.not_a_field1 IS MISSING as f1, p.user.not_a_field2 IS MISSING as f2
+FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.3.query.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.4.query.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/missing-fields/missing-fields.3.query.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
new file mode 100644
index 0000000..23c357e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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 : Retrieve all fields from different Parquet files with different schemas
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*id_age.parquet"),
+ ("include#1"="*id_name.parquet"),
+ ("format" = "parquet")
+);
+
+CREATE EXTERNAL DATASET ParquetDataset2(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*id_age.parquet"),
+ ("include#1"="*id_age-string.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.2.query.sqlpp
new file mode 100644
index 0000000..e332ba7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 : Retrieve all fields from different Parquet files with different schemas
+ * Expected Res : Success
+ * Date : June 22nd 2020
+ */
+USE test;
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
new file mode 100644
index 0000000..a8bc91cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 : Retrieve all fields from different Parquet files with different schemas
+ with conflicting fields
+ * Expected Res : Success
+ * Date : July 7th 2021
+ */
+
+USE test;
+
+SELECT VALUE p
+FROM ParquetDataset2 p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.4.query.sqlpp
new file mode 100644
index 0000000..1e82c7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 : Retrieve all fields from different Parquet files with different schemas
+ with conflicting fields
+ * Expected Res : Success
+ * Date : July 7th 2021
+ */
+
+USE test;
+
+SELECT VALUE p.age
+FROM ParquetDataset2 p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/no-files/no-files.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/no-files/no-files.1.ddl.sqlpp
new file mode 100644
index 0000000..2fc3e62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/no-files/no-files.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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 : No files
+* Expected Res : Warning: The provided external dataset configuration returned no files from the external source
+* Date : November 14th 2021
+*/
+
+-- param max-warnings:json=1000
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="NOT_A_DEFINITION"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/no-files/no-files.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/no-files/no-files.2.query.sqlpp
new file mode 100644
index 0000000..37de163
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/no-files/no-files.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 : No files
+* Expected Res : Warning: The provided external dataset configuration returned no files from the external source
+* Date : November 14th 2021
+*/
+
+-- param max-warnings:json=1000
+
+USE test;
+
+SELECT VALUE COUNT(*)
+FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.1.ddl.sqlpp
new file mode 100644
index 0000000..c17cf79
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Concat two objects after pushdown
+* Expected Res : Success
+* Date : September 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.2.query.sqlpp
new file mode 100644
index 0000000..daacd24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.2.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 : Concat two objects after pushdown
+* Expected Res : Success
+* Date : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT VALUE object_concat(p.coordinates, p.user).name
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.3.query.sqlpp
new file mode 100644
index 0000000..7463ab1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 : Concat two objects after pushdown
+* Expected Res : Success
+* Date : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT VALUE object_concat(p.coordinates, p.user).name
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.4.query.sqlpp
new file mode 100644
index 0000000..c3a7586
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.4.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 : Concat two objects after pushdown
+* Expected Res : Success
+* Date : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+SELECT VALUE object_concat(p.coordinates, p.user)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.5.query.sqlpp
new file mode 100644
index 0000000..cbbc793
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/object-concat/object-concat.5.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Concat two objects after pushdown
+* Expected Res : Success
+* Date : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+
+EXPLAIN
+SELECT VALUE object_concat(p.coordinates, p.user)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.01.ddl.sqlpp
new file mode 100644
index 0000000..1913848
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.01.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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 : Test complex plans while enabling the value access pushdown
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset1(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
+
+CREATE EXTERNAL DATASET ParquetDataset2(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.query.sqlpp
new file mode 100644
index 0000000..dcaa9f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : No pushdown on p1
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT sum(object_length(p1))
+FROM ParquetDataset1 p1, ParquetDataset2 p2
+WHERE p2.id = p1.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.query.sqlpp
new file mode 100644
index 0000000..0b08465
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : No pushdown
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT DISTINCT VALUE p1
+FROM ParquetDataset1 p1
+WHERE p1.id > 10;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.query.sqlpp
new file mode 100644
index 0000000..2e7bb09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Should only pushdown "p1.entities.hashtags"
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT text, sum(array_distinct(p1.entities.hashtags))
+FROM ParquetDataset1 p1, p1.entities.hashtags ht
+WHERE p1.id > 10 AND lowercase(ht.text) = "string"
+GROUP BY ht.text AS text;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.query.sqlpp
new file mode 100644
index 0000000..61d00b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Get only "indices" and "text" from "p1.entities.hashtags"
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT text, sum(array_sum(ht.indices))
+FROM ParquetDataset1 p1, p1.entities.hashtags ht
+WHERE p1.id > 10 AND lowercase(ht.text) = "string"
+GROUP BY ht.text AS text;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.query.sqlpp
new file mode 100644
index 0000000..93b09d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 : Pushdown "p.entities.urls"
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT p.entities.urls[0].display_url
+FROM ParquetDataset1 p
+WHERE array_count(p.entities.urls) > 10
+LIMIT 10
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.query.sqlpp
new file mode 100644
index 0000000..b098547
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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 : Get "entities.hashtags.text" from "p1" and "p2"
+ and "user.name" from "p1"
+* Expected Res : Success
+* Date : July 23th 2021
+*/
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+EXPLAIN
+SELECT uname, COUNT(*) as cnt
+FROM ParquetDataset1 p1, p1.entities.hashtags ht1
+WHERE ht1.text IN (SELECT VALUE ht2.text
+ FROM ParquetDataset2 p2, p2.entities.hashtags ht2)
+GROUP BY p1.user.name AS uname
+ORDER BY cnt DESC
+LIMIT 10;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..be64bb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Retrieve all fields from a Parquet file
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+%template%,
+("container"="playground"),
+("definition"="parquet-data/reviews"),
+("include"="*dummy_tweet.parquet"),
+("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.2.query.sqlpp
new file mode 100644
index 0000000..3a0e80d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 : Retrieve all fields from a Parquet file
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.3.query.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.4.query.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-all-fields/select-all-fields.3.query.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp
new file mode 100644
index 0000000..5738fad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.2.query.sqlpp
new file mode 100644
index 0000000..7b022fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.2.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+SELECT VALUE count(p.text)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp
new file mode 100644
index 0000000..9ec9468
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE count(p.text)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp
new file mode 100644
index 0000000..a1e72c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 : Retrieve the number of users
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+SELECT VALUE count(p.user.name)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp
new file mode 100644
index 0000000..0a0e193
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 : Retrieve the number of users
+* Expected Res : Success
+* Date : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE count(p.user.name)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
new file mode 100644
index 0000000..4e4000a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Test Standard UTF-8
+* Expected Res : Success
+* Date : August 30th 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*id_name_comment.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.2.query.sqlpp
new file mode 100644
index 0000000..a957ac5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 : Test Standard UTF-8
+* Expected Res : Success
+* Date : August 30th 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "false";
+
+SELECT VALUE array_count(split(trim(p.comment),"𩸽"))
+FROM ParquetDataset p
+WHERE contains(p.comment, "𩸽");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.4.query.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.1.ddl.sqlpp
new file mode 100644
index 0000000..ab202b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Type mismatch DDL
+* Expected Res : Success
+* Date : July 20th 2021
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING %adapter%
+(
+ %template%,
+ ("container"="playground"),
+ ("definition"="parquet-data/reviews"),
+ ("include"="*dummy_tweet.parquet"),
+ ("format" = "parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.2.query.sqlpp
new file mode 100644
index 0000000..29008c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.2.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Accessing an array as an object
+* Expected Res : Warning
+* Date : July 20th 2021
+*/
+
+-- param max-warnings:json=1000
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE p.entities.urls.display_url IS MISSING
+FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.3.query.sqlpp
new file mode 100644
index 0000000..65f6bdf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Accessing an object as an array
+* Expected Res : Warning
+* Date : July 20th 2021
+*/
+
+-- param max-warnings:json=1000
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE p.entities[*].display_url IS MISSING
+FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.4.query.sqlpp
new file mode 100644
index 0000000..26ccbcb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/parquet/type-mismatch/type-mismatch.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Accessing an array as an object
+* Expected Res : Warning
+* Date : July 20th 2021
+*/
+
+-- param max-warnings:json=1000
+
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE p.place.bounding_box.coordinates[0][0].not_object IS MISSING
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/non-s3-region/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/non-s3-region/external_dataset.000.ddl.sqlpp
index 3b22f11..4d0941d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/non-s3-region/external_dataset.000.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/non-s3-region/external_dataset.000.ddl.sqlpp
@@ -18,7 +18,7 @@
*/
/*
- * Creating an external dataset should not fail when non-s3-region is used (some-new-region)
+ * Creating an external dataset should fail when non-s3-region is used (some-new-region)
*/
drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.00.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.00.ddl.sqlpp
new file mode 100644
index 0000000..2337ac4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.00.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ParquetType AS {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING S3 (
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="parquet-data/reviews"),
+("format"="parquet"),
+("include"="*dummy_tweet.parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.01.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.01.query.sqlpp
new file mode 100644
index 0000000..4631047
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.01.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+SELECT VALUE count(p.text)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.99.ddl.sqlpp
new file mode 100644
index 0000000..86a1b59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.00.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.00.ddl.sqlpp
new file mode 100644
index 0000000..15ec56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.00.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE ParquetType AS {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("sessionToken"="dummySessionToken"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="parquet-data/reviews"),
+("format"="parquet"),
+("include"="*dummy_tweet.parquet")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.01.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.01.query.sqlpp
new file mode 100644
index 0000000..4631047
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.01.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+SELECT VALUE count(p.text)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.99.ddl.sqlpp
new file mode 100644
index 0000000..86a1b59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.1.ddl.sqlpp
deleted file mode 100644
index a3e79a0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.1.ddl.sqlpp
+++ /dev/null
@@ -1,49 +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 : Create an external dataset that contains records stored with rc file format.
- * Build an index over the external dataset age attribute
- * Perform a query over the dataset using the index.
- * Expected Res : Success
- * Date : 3rd Jan 2014
-*/
-
-drop dataverse test if exists;
-create dataverse test;
-
-use test;
-
-
-create type test.EmployeeType as
- closed {
- id : bigint,
- name : string,
- age : bigint
-};
-
-create external dataset EmployeeDataset(EmployeeType) using `hdfs`(
-(`hdfs`=`hdfs://127.0.0.1:31888`),
-(`path`=`/asterix/external-indexing-test.rc`),
-(`input-format`=`rc-input-format`),
-(`format`=`hdfs-writable`),
-(`parser`=`hive-parser`),
-(`hive-serde`=`org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe`));
-
-create index EmployeeAgeIdx on EmployeeDataset (age) type btree;
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.2.update.sqlpp
deleted file mode 100644
index e2d598f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.2.update.sqlpp
+++ /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 : Create an external dataset that contains records stored with rc hdfs file format.
- * Build an index over the external dataset age attribute
- * Perform a query over the dataset using the index.
- * Expected Res : Success
- * Date : 3rd Jan 2014
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.3.query.sqlpp
deleted file mode 100644
index 58db233..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-indexing/rc-format/rc-format.3.query.sqlpp
+++ /dev/null
@@ -1,33 +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 : Create an external dataset that contains records stored with rc hdfs file format.
- * Build an index over the external dataset age attribute
- * Perform a query over the dataset using the index.
- * Expected Res : Success
- * Date : 3rd Jan 2014
-*/
-
-use test;
-
-
-select element emp
-from EmployeeDataset as emp
-where (emp.age = 22)
-;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.1.ddl.sqlpp
new file mode 100644
index 0000000..abe0710
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.1.ddl.sqlpp
@@ -0,0 +1,82 @@
+/*
+ * 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 : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+* This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date : Jun 16 2015
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.LineType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type test.OrderType as
+ closed {
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create type test.CustomerType as
+ closed {
+ c_custkey : integer,
+ c_name : string,
+ c_address : string,
+ c_nationkey : integer,
+ c_phone : string,
+ c_acctbal : double,
+ c_mktsegment : string,
+ c_comment : string
+};
+
+create external dataset Line(LineType) using `localfs`((`path`=`asterix_nc1://data/big-object/lineitem.tbl.big`),(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+create external dataset `Order`(OrderType) using `localfs`((`path`=`asterix_nc1://data/big-object/order.tbl.big`),(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+create external dataset Customer(CustomerType) using `localfs`((`path`=`asterix_nc1://data/big-object/customer.tbl.big`),(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.2.update.sqlpp
new file mode 100644
index 0000000..00a13e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+* This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date : Jun 16 2015
+*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.3.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.3.lib.sqlpp
new file mode 100644
index 0000000..7167161
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.3.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install test testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.4.ddl.sqlpp
new file mode 100644
index 0000000..28d8a90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+
+create function roundtrip(s) as "roundtrip",
+ "Tests.roundstr" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.5.query.sqlpp
new file mode 100644
index 0000000..f04a02c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_pyudf/big_object_pyudf.5.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+ This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date : Jun 16 2015
+*/
+
+USE test;
+
+
+SELECT ROUND(AVG(`string-length`(roundtrip(c.c_comment))),3) AS len_c_comment, ROUND(AVG(`string-length`(roundtrip(o.o_comment))),3) AS len_o_comment
+FROM Customer AS c,
+ `Order` AS o
+WHERE c.c_custkey = o.o_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.1.ddl.sqlpp
new file mode 100644
index 0000000..d3bf31c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+drop dataverse testdv2 if exists;
+create dataverse testdv2;
+use testdv2;
+
+create type testtype as closed {
+ id: int64,
+ name: string,
+ hobbies: {{string}}
+};
+
+create dataset testds(testtype) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.2.update.sqlpp
new file mode 100644
index 0000000..0353284
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 testdv2;
+
+set `compiler.sortmemory` "64MB";
+
+load dataset testds
+using localfs
+(("path"="asterix_nc1://target/data/big-object/big_object_20M.adm"),("format"="adm"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.3.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.3.lib.sqlpp
new file mode 100644
index 0000000..923dcb2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.3.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install testdv2 testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.4.ddl.sqlpp
new file mode 100644
index 0000000..8b304fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 testdv2;
+
+create function roundtrip(s) as "roundtrip",
+ "Tests.roundstr" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.5.query.sqlpp
new file mode 100644
index 0000000..fdb285e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.5.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 testdv2;
+
+select value roundtrip(d)
+from testds d
+where d.id=1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.0.ddl.sqlpp
new file mode 100644
index 0000000..5a5bbec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.0.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE externallibtest;
+USE externallibtest;
+
+create type CountryCapitalType if not exists as closed {
+country: string,
+capital: string
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.1.lib.sqlpp
new file mode 100644
index 0000000..cc6c855
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.1.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.2.ddl.sqlpp
new file mode 100644
index 0000000..4ef5d6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.2.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 externallibtest;
+
+create function getCapital(a: string) returns CountryCapitalType
+ as "org.apache.asterix.external.library.CapitalFinderFactory" at testlib;
+
+create function mysum(a, b) { a+b };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.3.query.sqlpp
new file mode 100644
index 0000000..cbf2a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 externallibtest;
+
+SELECT getCapital(country) as natCap, mysum(1,2) as sum
+FROM ["England","Italy","China","United States","India","Jupiter"] as country
+ORDER BY natCap.country;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.4.ddl.sqlpp
new file mode 100644
index 0000000..65733e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mixedudfs/mixedudfs.4.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+DROP DATAVERSE externallibtest;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp
index e17f03f..72dc83c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp
@@ -28,4 +28,5 @@
select element result
from Animals as test
with result as roundtrip(test)[0].class.fullClassification.lower.lower.lower.lower.lower.lower.Species
-order by result;
+order by result
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.2.ddl.sqlpp
index 8041b5e..ff61810 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.2.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.2.ddl.sqlpp
@@ -23,3 +23,6 @@
create function typeValidationNullCall(a, b, c, d, e, f, g, h)
as "roundtrip", "Tests.roundtrip" at testlib with {"null-call": true};
+
+create function stringTest(s) as "roundtrip",
+ "Tests.roundtrip" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.3.query.sqlpp
index 358b28a..e8b7fd3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.3.query.sqlpp
@@ -17,5 +17,5 @@
* under the License.
*/
use externallibtest;
-typeValidation(907, 9.07, "907", 9.07, true, date("2013-01-01"), datetime("1989-09-07T12:13:14.039Z"));
+typeValidation(907, 9.07, "907", 9.07, true, date("2013-01-01"), datetime("1989-09-07T12:13:14.039"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/select-let/select-let.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/select-let/select-let.2.query.sqlpp
new file mode 100644
index 0000000..5fbe641
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/select-let/select-let.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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: Test SELECT ... LET ... WHERE ... (no FROM clause)
+ */
+
+select value x
+let x = 2
+where x > 0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
index a1173de..342c74d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
@@ -51,6 +51,7 @@
let temp = (select * from Tweets f where f.user.id_str in (select value fw.followers from Followers fw where fw.twitter_id=user_id)[0]
and
spatial_intersect(create_point(f.place.bounding_box.coordinates[0][0][0],f.place.bounding_box.coordinates[0][0][1]),
-create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1])))
+create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1]))
+order by f.id)
where t.user.id_str = user_id)
};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-07-error/core-07-error.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-07-error/core-07-error.1.query.sqlpp
new file mode 100644
index 0000000..04a063c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/core-07-error/core-07-error.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Failure: GROUP BY without FROM clause
+ */
+
+SELECT x, y, COUNT(*) AS cnt
+GROUP BY x, y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp
new file mode 100644
index 0000000..10f5546
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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 : Testing group by over cross product (ASTERIXDB-2917)
+ * Expected : SUCCESS
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id integer not unknown) open type primary key id;
+
+create dataset t2(id integer not unknown) open type primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp
new file mode 100644
index 0000000..7fe3777
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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 t1 ([
+ { "id": 1, "x": "a" },
+ { "id": 2, "x": "a" },
+ { "id": 3, "x": "a" },
+ { "id": 4, "x": "b" },
+ { "id": 5, "x": "b" },
+ { "id": 6, "x": "b" }
+]);
+
+insert into t2 ([
+ { "id": 11, "y": "c" },
+ { "id": 12, "y": "c" }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp
new file mode 100644
index 0000000..34ec706
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+select t1.x, count(*) as cnt
+from t2, t1
+group by t1.x
+order by t1.x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/parquet/parquet.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/parquet/parquet.1.ddl.sqlpp
new file mode 100644
index 0000000..308d43d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/parquet/parquet.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : Test reading Parquet from HDFS
+ Check Parquet S3 for more tests
+* Expected Res : Success
+* Date : June 22nd 2020
+* Update Date : July 26th 2021
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+ ("hdfs"="hdfs://127.0.0.1:31888"),
+ ("path"="/asterix/id_age.parquet"),
+ ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/parquet/parquet.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/parquet/parquet.2.query.sqlpp
new file mode 100644
index 0000000..7e332e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/hdfs/parquet/parquet.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 : Test reading Parquet from HDFS
+ Check Parquet S3 for more tests
+* Expected Res : Success
+* Date : June 22nd 2020
+* Update Date : July 26th 2021
+*/
+USE test;
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.1.ddl.sqlpp
new file mode 100644
index 0000000..58d2013
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.1.ddl.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.
+ */
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE GenericType AS { _id: uuid, c: bigint };
+CREATE DATASET IndexDatasetA (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET IndexDatasetB (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET IndexDatasetC (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+CREATE DATASET ProbeDataset (GenericType)
+PRIMARY KEY _id AUTOGENERATED;
+
+CREATE INDEX indexA
+ON IndexDatasetA (k: int);
+CREATE INDEX indexB
+ON IndexDatasetB (k: int);
+CREATE INDEX indexC
+ON IndexDatasetC (k: int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.2.update.sqlpp
new file mode 100644
index 0000000..04b3c42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.2.update.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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 TestDataverse;
+
+INSERT INTO ProbeDataset [
+ { "c": 1 },
+ { "c": 2 },
+ { "c": 3 }
+];
+INSERT INTO IndexDatasetA [
+ { "c": 1, "k": 1 },
+ { "c": 2, "k": 2 },
+ { "c": 3, "k": 4 }
+];
+INSERT INTO IndexDatasetB [
+ { "c": 1, "k": 1 },
+ { "c": 2, "k": 2 },
+ { "c": 3, "k": 4 }
+];
+INSERT INTO IndexDatasetC [
+ { "c": 1, "k": 1 },
+ { "c": 2, "k": 2 },
+ { "c": 3, "k": 4 }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.3.query.sqlpp
new file mode 100644
index 0000000..9e65e5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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 TestDataverse;
+
+FROM ProbeDataset P,
+ IndexDatasetA A,
+ IndexDatasetB B
+WHERE P.c /* +indexnl */ = A.k AND
+ P.c /* +indexnl */ = B.k
+SELECT P.c
+ORDER BY P.c;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.4.query.sqlpp
new file mode 100644
index 0000000..f59cd69
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-join/btree-multiple-join/btree-multiple-join.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 TestDataverse;
+
+FROM ProbeDataset P,
+ IndexDatasetA A,
+ IndexDatasetB B,
+ IndexDatasetC C
+WHERE P.c /* +indexnl */ = A.k AND
+ P.c /* +indexnl */ = B.k AND
+ P.c /* +indexnl */ = C.k
+SELECT P.c
+ORDER BY P.c;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.01.ddl.sqlpp
new file mode 100644
index 0000000..c454cea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.01.ddl.sqlpp
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE t1 AS { id: int, s_f1: string, s_f2: string?, i_f: int? };
+
+CREATE DATASET ds1(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds2(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds3(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds4(id int not unknown, a string, b int) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds5(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds6(id int not unknown) OPEN TYPE PRIMARY KEY id %with%;
+CREATE DATASET ds7(t1) PRIMARY KEY id %with%;
+
+CREATE INDEX idx1 ON ds1(x: int);
+CREATE INDEX idx2 ON ds2(x: int) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds3(x: int) CAST (DEFAULT NULL);
+
+CREATE INDEX idx5_dt_fmt ON ds5(f_dt_fmt: datetime) CAST (DEFAULT NULL datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY');
+CREATE INDEX idx5_d_fmt ON ds5(f_d_fmt: date) CAST (DEFAULT NULL date 'MM/DD/YYYY');
+CREATE INDEX idx5_t_fmt ON ds5(f_t_fmt: time) CAST (DEFAULT NULL time 'hh:mm:ss.nnna');
+
+CREATE INDEX idx6_dt ON ds6(f_dt: datetime) CAST (DEFAULT NULL);
+CREATE INDEX idx6_d ON ds6(f_d: date) CAST (DEFAULT NULL);
+CREATE INDEX idx6_t ON ds6(f_t: time) CAST (DEFAULT NULL);
+
+CREATE INDEX idx1 ON ds7(s_f2);
+CREATE INDEX idx2 ON ds7(s_f2: string) CAST (DEFAULT NULL);
+CREATE INDEX idx3 ON ds7(s_f2: int) CAST (DEFAULT NULL);
+
+CREATE VIEW view1(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds1;
+CREATE VIEW view2(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds2;
+CREATE VIEW view3(id int, x string, y int) DEFAULT NULL AS SELECT id, x, y FROM ds3;
+CREATE VIEW view4(id int, x int, y int) DEFAULT NULL AS SELECT id, x, y FROM ds4;
+
+CREATE VIEW view5_dt_fmt(id int, f_dt_fmt datetime) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_dt_fmt FROM ds5;
+
+CREATE VIEW view5_d_fmt(id int, f_d_fmt date) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_d_fmt FROM ds5;
+
+CREATE VIEW view5_t_fmt(id int, f_t_fmt time) DEFAULT NULL
+datetime 'MM/DD/YYYY hh:mm:ss.nnna' date 'MM/DD/YYYY' time 'hh:mm:ss.nnna' AS SELECT id, f_t_fmt FROM ds5;
+
+CREATE VIEW view6_dt(id int, f_dt datetime) DEFAULT NULL AS SELECT id, f_dt FROM ds6;
+CREATE VIEW view6_d(id int, f_d date) DEFAULT NULL AS SELECT id, f_d FROM ds6;
+CREATE VIEW view6_t(id int, f_t time) DEFAULT NULL AS SELECT id, f_t FROM ds6;
+
+CREATE VIEW view7_1(id int, s_f2 string, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
+CREATE VIEW view7_2(id int, s_f2 int, i_f int) DEFAULT NULL AS SELECT id, s_f2, i_f FROM ds7;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.02.update.sqlpp
new file mode 100644
index 0000000..1a51639
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.02.update.sqlpp
@@ -0,0 +1,85 @@
+/*
+ * 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 ds1 [
+{"id": 1, "a": "s1", "b": 1, "x": 1, "y": 2},
+{"id": 2, "a": "s2", "b": 2, "x": 2, "y": 1},
+{"id": 3, "a": "s3", "b": 3, "x": 1.1, "y": 2.1},
+{"id": 4, "a": "s4", "b": 4, "x": 2.1, "y": 1.1},
+{"id": 5, "a": "s5", "b": 5, "y": 1.1},
+{"id": 6, "a": "s6", "b": 6, "x": 3, "y": 3}
+];
+
+INSERT INTO ds2 [
+{"id": 1, "a": "s1", "b": 1, "x": 1, "y": 2},
+{"id": 2, "a": "s2", "b": 2, "x": 2, "y": 1},
+{"id": 3, "a": "s3", "b": 3, "x": 1.1, "y": 2.1},
+{"id": 4, "a": "s4", "b": 4, "x": 2.1, "y": 1.1},
+{"id": 5, "a": "s5", "b": 5, "y": 1.1},
+{"id": 6, "a": "s6", "b": 6, "x": 33, "y": 3}
+];
+
+INSERT INTO ds3 [
+{"id": 1, "a": "s1", "b": 1, "x": 1, "y": 2},
+{"id": 2, "a": "s2", "b": 2, "x": 2, "y": 1},
+{"id": 3, "a": "s3", "b": 3, "x": 1.1, "y": 2.1},
+{"id": 4, "a": "s4", "b": 4, "x": 2.1, "y": 1.1},
+{"id": 5, "a": "s5", "b": 5, "y": 1.1},
+{"id": 6, "a": "s6", "b": 6, "x": 333, "y": 3}
+];
+
+INSERT INTO ds4 [
+{"id": 1, "a": "s1", "b": 1, "x": 1, "y": 2},
+{"id": 2, "a": "s2", "b": 2, "x": 2, "y": 1},
+{"id": 3, "a": "s3", "b": 3, "x": 1.1, "y": 2.1},
+{"id": 4, "a": "s4", "b": 4, "x": 2.1, "y": 1.1},
+{"id": 5, "a": "s5", "b": 5, "y": 1.1},
+{"id": 6, "a": "s6", "b": 6, "x": 3333, "y": 3}
+];
+
+INSERT INTO ds5 [
+{"id": 1, 'f_dt_fmt':'02/20/2020 01:40:41.001am', 'f_d_fmt':'02/20/2020', 'f_t_fmt':'01:40:41.001am'},
+{"id": 2, 'f_dt_fmt':'11/25/2021 10:12:51.999pm', 'f_d_fmt':'11/25/2021', 'f_t_fmt':'10:12:51.999pm'},
+/* Invalid value */
+{"id": 3, 'f_dt_fmt':'a', 'c_date':'b', 'f_t_fmt':'c'},
+{"id": 4, 'f_dt_fmt':'01/20/2019 02:40:41.001pm', 'f_d_fmt':'01/20/2019', 'f_t_fmt':'02:40:41.001pm'},
+{"id": 5, 'f_dt_fmt':null, 'f_d_fmt':null, 'f_t_fmt':null},
+{"id": 6}
+];
+
+INSERT INTO ds6 [
+{"id": 1, 'f_dt':'2020-02-20T01:40:41.001', 'f_d':'2020-02-20', 'f_t':'01:40:41.001'},
+{"id": 2, 'f_dt':'2021-11-25T22:12:51.999', 'f_d':'2021-11-25', 'f_t':'22:12:51.999'},
+/* Invalid value */
+{"id": 3, 'f_dt':'a', 'f_d':'b', 'f_t':'c'},
+{"id": 4, 'f_dt':'2019-01-20T14:40:41.001', 'f_d':'2019-01-20', 'f_t':'14:40:41.001'},
+{"id": 5, 'f_dt':null, 'f_d':null, 'f_t':null},
+{"id": 6}
+];
+
+INSERT INTO ds7 [
+{"id": 1, "s_f1": "s", "s_f2": "s", "i_f": 1 },
+{"id": 2, "s_f1": "2", "s_f2": "2", "i_f": 2 },
+{"id": 3, "s_f1": "3.5", "s_f2": "3.5", "i_f": 3 },
+{"id": 4, "s_f1": "4", "s_f2": "4", "i_f": 4 },
+{"id": 5, "s_f1": "5", "s_f2": null, "i_f": null },
+{"id": 6, "s_f1": "" }
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.03.query.sqlpp
new file mode 100644
index 0000000..56ad29e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.03.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// test that idx1 on x is not used since idx1 does not have the CAST modifier and the field x has cast
+SELECT id, x, y FROM view1 WHERE x <= 1 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.04.query.sqlpp
new file mode 100644
index 0000000..f9db494
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.04.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// test that idx2 on x is used since idx2 has the CAST modifier and the field x has cast
+SELECT id, x, y FROM view2 WHERE x <= 1 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.05.query.sqlpp
new file mode 100644
index 0000000..5180871
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.05.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// test that idx3 on x is not used since idx3 has the CAST modifier on int and the field x has cast as string
+SELECT id, x, y FROM view3 WHERE x <= "1" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.06.query.sqlpp
new file mode 100644
index 0000000..9508ece
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.06.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// test that idx3 on x is not used since idx3 has the CAST modifier and the field x does not have cast
+SELECT id, x, y FROM ds3 WHERE x <= 1 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.07.query.sqlpp
new file mode 100644
index 0000000..e5016c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.07.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// test that idx2 on view2(x) is used. both v4.x and v2.x are int and both v2.x and idx2 use int-default-null.
+SELECT v4.x AS v4x, v2.x AS v2x
+FROM view4 AS v4, view2 AS v2
+WHERE v4.x /*+ indexnl */ = v2.x ORDER BY v4x, v2x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.08.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.08.query.sqlpp
new file mode 100644
index 0000000..2242a50
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.08.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// test that idx3 on view3(x) is not used because v3.x uses string-default-null and idx3 uses int-default-null
+SELECT v4.x AS v4x, v3.x AS v3x
+FROM view4 AS v4, view3 AS v3
+WHERE v4.x /*+ indexnl */ = v3.x ORDER BY v4x, v3x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.09.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.09.query.sqlpp
new file mode 100644
index 0000000..5be7fc9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.09.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// test that idx1 is not used because idx1 does not have CAST modifier (idx1 is just int and v1.x uses int-default-null).
+SELECT v4.x AS v4x, v1.x AS v1x
+FROM view4 AS v4, view1 AS v1
+WHERE v4.x /*+ indexnl */ = v1.x ORDER BY v4x, v1x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.10.query.sqlpp
new file mode 100644
index 0000000..5be7fc9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.10.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// test that idx1 is not used because idx1 does not have CAST modifier (idx1 is just int and v1.x uses int-default-null).
+SELECT v4.x AS v4x, v1.x AS v1x
+FROM view4 AS v4, view1 AS v1
+WHERE v4.x /*+ indexnl */ = v1.x ORDER BY v4x, v1x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.11.query.sqlpp
new file mode 100644
index 0000000..59786c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.11.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+// test that idx2 is not used because idx2 uses int-default-null and ds2.x is just int.
+SELECT v4.x AS v4x, ds2.x AS ds2x
+FROM view4 AS v4, ds2 AS ds2
+WHERE v4.x /*+ indexnl */ = ds2.x ORDER BY v4x, ds2x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.12.query.sqlpp
new file mode 100644
index 0000000..e47124a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.12.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT VALUE v
+FROM view5_dt_fmt AS v
+WHERE v.f_dt_fmt < datetime('2020-12-20T00:00:00')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.13.query.sqlpp
new file mode 100644
index 0000000..faae08d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.13.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT VALUE v
+FROM view5_d_fmt AS v
+WHERE v.f_d_fmt < date('2020-12-20')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.14.query.sqlpp
new file mode 100644
index 0000000..3c775d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.14.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT VALUE v
+FROM view5_t_fmt AS v
+WHERE v.f_t_fmt < time('18:13:03')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.15.query.sqlpp
new file mode 100644
index 0000000..7b88c3d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.15.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT VALUE v
+FROM view6_dt AS v
+WHERE v.f_dt < datetime('2020-12-20T00:00:00')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.16.query.sqlpp
new file mode 100644
index 0000000..5871f2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.16.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT VALUE v
+FROM view6_d AS v
+WHERE v.f_d < date('2020-12-20')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.17.query.sqlpp
new file mode 100644
index 0000000..93f1c88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.17.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT VALUE v
+FROM view6_t AS v
+WHERE v.f_t < time('18:13:03')
+ORDER BY v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.18.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.18.query.sqlpp
new file mode 100644
index 0000000..29081f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.18.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+// testing that idx5_dt_fmt is used
+SELECT v6.f_dt AS v6f_dt, v5.f_dt_fmt AS v5f_dt_fmt
+FROM view6_dt AS v6, view5_dt_fmt AS v5
+WHERE v6.f_dt /*+ indexnl */ = v5.f_dt_fmt ORDER BY v6f_dt, v5f_dt_fmt;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.19.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.19.query.sqlpp
new file mode 100644
index 0000000..e632e91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.19.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// index idx2 should be used
+SELECT id, s_f2 FROM view7_1 WHERE s_f2 < "4" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.20.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.20.query.sqlpp
new file mode 100644
index 0000000..56296e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.20.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// no index is used (and no result is returned because of incompatible types. s_f2 string vs. 4 int)
+SELECT id, s_f2 FROM view7_1 WHERE s_f2 < 4 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.21.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.21.query.sqlpp
new file mode 100644
index 0000000..7e03ebf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.21.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// no index is used (and no result is returned because of incompatible types. s_f2 int vs. "4" string)
+SELECT id, s_f2 FROM view7_2 WHERE s_f2 < "4" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.22.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.22.query.sqlpp
new file mode 100644
index 0000000..f0eece2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.22.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+// index idx3 should be used
+SELECT id, s_f2 FROM view7_2 WHERE s_f2 < 4 ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.23.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.23.query.sqlpp
new file mode 100644
index 0000000..4fe0cae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/cast-default-null/cast-default-null.23.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+// index idx1 should be used
+set `compiler.indexonly` "false"; // there is an issue with using index only plan where nulls/missings are included
+SELECT id, s_f2 FROM ds7 WHERE s_f2 < "4" ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.1.ddl.sqlpp
new file mode 100644
index 0000000..dd75e79
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.Schema as
+ closed {
+ unique1 : bigint,
+ unique2 : bigint,
+ two : bigint,
+ four : bigint,
+ ten : bigint,
+ twenty : bigint,
+ onePercent : bigint,
+ tenPercent : bigint,
+ twentyPercent : bigint,
+ fiftyPercent : bigint,
+ unique3 : bigint,
+ evenOnePercent : bigint,
+ oddOnePercent : bigint,
+ stringu1 : string,
+ stringu2 : string,
+ string4 : string
+};
+
+create dataset onektup(Schema) primary key unique2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.2.update.sqlpp
new file mode 100644
index 0000000..e5d85fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+
+load dataset onektup using localfs ((`path`=`asterix_nc1://data/wisc/onektup.adm`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.3.query.sqlpp
new file mode 100644
index 0000000..6e29bdd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+ * Cross join: 100 * 100 -> 10,000 tuples
+ */
+
+use test;
+
+select count(*) as cnt
+from onektup cross join onektup as onektup2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.4.query.sqlpp
new file mode 100644
index 0000000..d0be10b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/*
+ * Cross join: 5 * 5 -> 25 tuples
+ */
+
+use test;
+
+select t1.unique1 x, t2.unique1 y
+from onektup t1 cross join onektup t2
+where t1.twenty = 0 and t2.twenty = 1
+order by x, y;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.5.query.sqlpp
new file mode 100644
index 0000000..2c542f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-01/cross-join-01.5.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Cross join: 2 * 2 -> 4 tuples
+ */
+
+use test;
+
+select x, y
+from [1,2] x cross join [3, 4] y
+order by x, y;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-02-negative/cross-join-02-negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-02-negative/cross-join-02-negative.1.query.sqlpp
new file mode 100644
index 0000000..92d4678
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/cross-join-02-negative/cross-join-02-negative.1.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/*
+ * Negative: left branch variable is not in scope
+ * for the right branch of the cross join
+ */
+
+select count(*) as cnt
+from [ [1,2], [3,4] ] as x cross join x as y;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash_join_missing/hash_join_missing.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash_join_missing/hash_join_missing.1.query.sqlpp
new file mode 100644
index 0000000..337226d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash_join_missing/hash_join_missing.1.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test hash join when values on both side are MISSING
+ */
+
+with
+R as (
+ from range(1, 50000) r
+ select (case when get_year(current_date()) > 0 then missing else r end) as r
+),
+
+S as (
+ from range(1, 50000) s
+ select (case when get_year(current_date()) > 0 then missing else s end) as s
+)
+
+select count(*) cnt
+from R, S
+where R.r = S.s;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash_join_missing/hash_join_missing.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash_join_missing/hash_join_missing.2.query.sqlpp
new file mode 100644
index 0000000..94de090
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/join/hash_join_missing/hash_join_missing.2.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test hash join when values on both side are NULL
+ */
+
+with
+R as (
+ from range(1, 50000) r
+ select (case when get_year(current_date()) > 0 then null else r end) as r
+),
+
+S as (
+ from range(1, 50000) s
+ select (case when get_year(current_date()) > 0 then null else s end) as s
+)
+
+select count(*) cnt
+from R, S
+where R.r = S.s;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.1.query.sqlpp
new file mode 100644
index 0000000..194983d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+-- param max-warnings:json=1000
+
+SELECT
+ parse_json('null') nullVal,
+ parse_json('1') intVal,
+ parse_json('"stringVal"') stringVal,
+ parse_json('3.75') doubleVal,
+ parse_json('{"id": 7, "name": "Jerry", "comment": "\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا"}') objectVal,
+ parse_json('[1,2,3]') arrayVal,
+ parse_json(null) nullArgVal,
+ parse_json(missing) missingArgVal,
+ parse_json(1) unexpectedType,
+ parse_json('') emptyString
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.2.query.sqlpp
new file mode 100644
index 0000000..81c7944
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.2.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+SELECT parse_json(v) multipleValues
+FROM ['', '{"a":1}', '[1, 2]', '{"c": "test"}', '{"a":3.57}', '["a", "b"]', '{"c":{"d":1}}'] v
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.3.query.sqlpp
new file mode 100644
index 0000000..cd4fc4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/json-parser/parse-json-function/parse-json-function.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+-- param max-warnings:json=1000
+
+SELECT parse_json(v) malformedVal
+FROM ['{"a":}', '[1, 2]', '{"c": "test"}', '{"a":}', '["a", "b"]', '{"c":{"d":1}}'] v
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/ordered-list-constructor_05/ordered-list-constructor_05.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/ordered-list-constructor_05/ordered-list-constructor_05.1.query.sqlpp
new file mode 100644
index 0000000..a86df20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/ordered-list-constructor_05/ordered-list-constructor_05.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+select
+ r,
+ ('foo','bar','foobar') a1,
+ (r,[r],null) a2,
+ (
+ (from [r, r+1] x select value count(x)),
+ (from [r, r+1] x select value sum(x))
+ ) a3,
+ (r,[r]) = [r,[r]] eq
+
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/ordered-list-constructor_06_negative/ordered-list-constructor_06_negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/ordered-list-constructor_06_negative/ordered-list-constructor_06_negative.1.query.sqlpp
new file mode 100644
index 0000000..24346b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/ordered-list-constructor_06_negative/ordered-list-constructor_06_negative.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * Failure expected: need parenthesis around each subquery
+ */
+
+select
+ (
+ from [r, r+1] x select value count(x),
+ from [r, r+1] x select value sum(x)
+ ) x
+
+from range(1,2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
index c4d77ca..219f66b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
@@ -48,4 +48,4 @@
};
create dataset Customers(CustomerType) primary key cid;
-create index customer_name_idx on Customers(name);
\ No newline at end of file
+create index customer_name_idx on Customers(name) exclude unknown key;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.1.ddl.sqlpp
new file mode 100644
index 0000000..f28ae6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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: views in a dataverse with a multipart name
+ */
+
+drop dataverse x.y if exists;
+create dataverse x.y;
+
+drop dataverse x.z if exists;
+create dataverse x.z;
+
+create view x.z.vz as
+ select r from range(0,2) r;
+
+create view x.y.vy as
+ select vz.* from x.z.vz;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.2.query.sqlpp
new file mode 100644
index 0000000..6293c28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.2.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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: views in a dataverse with a multipart name
+ */
+
+select value r
+from x.y.vy r
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.3.query.sqlpp
new file mode 100644
index 0000000..de77ed0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/view_1/view_1.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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: views in a dataverse with a multipart name
+ */
+
+select DataverseName, DatasetName,
+ ViewDetails.Dependencies Dependencies
+from Metadata.`Dataset`
+where DataverseName like "x%" and DatasetType = "VIEW"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.1.query.sqlpp
new file mode 100644
index 0000000..293f824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.1.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a, d.b ASC;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.10.query.sqlpp
new file mode 100644
index 0000000..38ec82c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.10.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS LAST, d.b DESC NULLS LAST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.11.query.sqlpp
new file mode 100644
index 0000000..bf17f9f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.11.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC, d.b DESC;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.12.query.sqlpp
new file mode 100644
index 0000000..acb7ea0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.12.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS FIRST, d.b DESC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.13.query.sqlpp
new file mode 100644
index 0000000..616c160
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.13.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS FIRST, d.b DESC NULLS LAST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.14.query.sqlpp
new file mode 100644
index 0000000..09cc382
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.14.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS LAST, d.b DESC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.15.query.sqlpp
new file mode 100644
index 0000000..09cc382
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.15.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS LAST, d.b DESC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.2.query.sqlpp
new file mode 100644
index 0000000..8c585c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.2.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS FIRST, d.b ASC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.3.query.sqlpp
new file mode 100644
index 0000000..31c41ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.3.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS FIRST, d.b ASC NULLS LAST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.4.query.sqlpp
new file mode 100644
index 0000000..af5c3c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.4.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS LAST, d.b ASC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.5.query.sqlpp
new file mode 100644
index 0000000..e48bbf7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.5.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS LAST, d.b ASC NULLS LAST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.6.query.sqlpp
new file mode 100644
index 0000000..5566c48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.6.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC, d.b DESC;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.7.query.sqlpp
new file mode 100644
index 0000000..edb760d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.7.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS FIRST, d.b DESC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.8.query.sqlpp
new file mode 100644
index 0000000..5fc05de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.8.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS FIRST, d.b DESC NULLS LAST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.9.query.sqlpp
new file mode 100644
index 0000000..96b33a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-2/order-by-2.9.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+WITH data AS [
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS LAST, d.b DESC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-3-negative/order-by-3-negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-3-negative/order-by-3-negative.1.query.sqlpp
new file mode 100644
index 0000000..1d9941f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-3-negative/order-by-3-negative.1.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+ * Failure: Syntax error
+ */
+
+WITH data AS
+[ {"a":1, "b":2},
+ {"a": NULL, "b": 7},
+ {"b":4},
+ {"a": "1", "b":8},
+ {"a": 3.0, "b":9}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-3-negative/order-by-3-negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-3-negative/order-by-3-negative.2.query.sqlpp
new file mode 100644
index 0000000..40a9cad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-3-negative/order-by-3-negative.2.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+ * Failure: Syntax error
+ */
+
+WITH data AS
+[ {"a":1, "b":2},
+ {"a": NULL, "b": 7},
+ {"b":4},
+ {"a": "1", "b":8},
+ {"a": 3.0, "b":9}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS NULLS;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.1.ddl.sqlpp
new file mode 100644
index 0000000..4c098ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type TestType as {
+ c: bigint
+};
+
+create dataset data(TestType) primary key c;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.10.query.sqlpp
new file mode 100644
index 0000000..a5af7c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.10.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS FIRST, d.b DESC NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.11.query.sqlpp
new file mode 100644
index 0000000..66978dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.11.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS LAST, d.b DESC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.12.query.sqlpp
new file mode 100644
index 0000000..f0c70cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.12.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS LAST, d.b DESC NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.13.query.sqlpp
new file mode 100644
index 0000000..3b6d826
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.13.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC, d.b DESC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.14.query.sqlpp
new file mode 100644
index 0000000..ec1ec83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.14.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS FIRST, d.b DESC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.15.query.sqlpp
new file mode 100644
index 0000000..0521b07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.15.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS FIRST, d.b DESC NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.16.query.sqlpp
new file mode 100644
index 0000000..8e85354
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.16.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS LAST, d.b DESC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.17.query.sqlpp
new file mode 100644
index 0000000..583b5cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.17.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS LAST, d.b DESC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.2.update.sqlpp
new file mode 100644
index 0000000..a642f52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.2.update.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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 data (
+[
+ {"a":1,"c":1},
+ {"a":1,"b":1,"c":2},
+ {"a":1,"b":NULL,"c":3},
+ {"a":1,"b":2,"c":4},
+
+ {"a":NULL,"c":5},
+ {"a":NULL,"b":3,"c":6},
+ {"a":NULL,"b":NULL,"c":7},
+ {"a":NULL,"b":4,"c":8},
+
+ {"c":9},
+ {"b":5,"c":10},
+ {"b":NULL,"c":11},
+ {"b":6,"c":12},
+
+ {"a":"1","c":13},
+ {"a":"1","b":7,"c":14},
+ {"a":"1","b":NULL,"c":15},
+ {"a":"1","b":8,"c":16},
+
+ {"a":3.0,"c":17},
+ {"a":3.0,"b":9,"c":18},
+ {"a":3.0,"b":NULL,"c":19},
+ {"a":3.0,"b":10,"c":20}
+]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.3.query.sqlpp
new file mode 100644
index 0000000..d66429f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a, d.b ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.4.query.sqlpp
new file mode 100644
index 0000000..04644fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS FIRST, d.b ASC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.5.query.sqlpp
new file mode 100644
index 0000000..6173a3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.5.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS FIRST, d.b ASC NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.6.query.sqlpp
new file mode 100644
index 0000000..e25a0c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.6.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS LAST, d.b ASC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.7.query.sqlpp
new file mode 100644
index 0000000..b630030
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.7.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS LAST, d.b ASC NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.8.query.sqlpp
new file mode 100644
index 0000000..fadd49e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.8.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC, d.b DESC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.9.query.sqlpp
new file mode 100644
index 0000000..aab89f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.9.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS FIRST, d.b DESC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.1.ddl.sqlpp
index 9ac564e..d58c3eb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.1.ddl.sqlpp
@@ -17,8 +17,6 @@
* under the License.
*/
-
-
drop dataverse test if exists;
create dataverse test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.3.query.sqlpp
index 3758ff2..44d2a73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.3.query.sqlpp
@@ -19,8 +19,6 @@
USE test;
-
SELECT VALUE d
FROM data AS d
-ORDER BY d.a;
-
+ORDER BY d.a;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.4.query.sqlpp
new file mode 100644
index 0000000..538870f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.5.query.sqlpp
new file mode 100644
index 0000000..e4fe10d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.5.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.6.query.sqlpp
new file mode 100644
index 0000000..243b0f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.6.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.7.query.sqlpp
new file mode 100644
index 0000000..2a3ad56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.7.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS FIRST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.8.query.sqlpp
new file mode 100644
index 0000000..5b5c067
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by-from-dataset/order-by-from-dataset.8.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS LAST;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.1.query.sqlpp
index 567ca72..16f939d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.1.query.sqlpp
@@ -17,7 +17,6 @@
* under the License.
*/
-
WITH data AS
[ {"a":1, "b":2},
{"a": NULL, "b": 7},
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.2.query.sqlpp
new file mode 100644
index 0000000..0c70a22
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+WITH data AS
+[ {"a":1, "b":2},
+ {"a": NULL, "b": 7},
+ {"b":4},
+ {"a": "1", "b":8},
+ {"a": 3.0, "b":9}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.3.query.sqlpp
new file mode 100644
index 0000000..0a77b97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+WITH data AS
+[ {"a":1, "b":2},
+ {"a": NULL, "b": 7},
+ {"b":4},
+ {"a": "1", "b":8},
+ {"a": 3.0, "b":9}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a ASC NULLS LAST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.4.query.sqlpp
new file mode 100644
index 0000000..5c345cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+WITH data AS
+[ {"a":1, "b":2},
+ {"a": NULL, "b": 7},
+ {"b":4},
+ {"a": "1", "b":8},
+ {"a": 3.0, "b":9}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.5.query.sqlpp
new file mode 100644
index 0000000..efbeb55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.5.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+WITH data AS
+[ {"a":1, "b":2},
+ {"a": NULL, "b": 7},
+ {"b":4},
+ {"a": "1", "b":8},
+ {"a": 3.0, "b":9}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS FIRST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.6.query.sqlpp
new file mode 100644
index 0000000..2f26dc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/order-by/order-by.6.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+WITH data AS
+[ {"a":1, "b":2},
+ {"a": NULL, "b": 7},
+ {"b":4},
+ {"a": "1", "b":8},
+ {"a": 3.0, "b":9}
+]
+
+SELECT VALUE d
+FROM data AS d
+ORDER BY d.a DESC NULLS LAST;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/round-half-up/round-half-up.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/round-half-up/round-half-up.1.query.sqlpp
new file mode 100644
index 0000000..2a2c2ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/round-half-up/round-half-up.1.query.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.
+ */
+
+with lst as [
+ float(10.51),
+ float(20.59),
+ double(30.51),
+ double(40.59),
+ float("INF"),
+ double("-INF"),
+ int8(8),
+ int16(16),
+ int32(32),
+ int64(64)
+]
+
+select r, round_half_up(v, 1) v1, round_half_up(-v, 1) v2
+from range( 0, len(lst)-1 ) r
+let v = lst[r]
+order by r;
+
+
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 bb51b39..2675c40 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
@@ -126,6 +126,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="objects">
+ <compilation-unit name="object_concat_with_array">
+ <output-dir compare="Text">object_concat_with_array</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="objects">
<compilation-unit name="object_length">
<output-dir compare="Text">object_length</output-dir>
</compilation-unit>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.2.query.sqlpp
new file mode 100644
index 0000000..775c79a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.2.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE
+[
+ is_null(object_concat([])),
+ is_null(object_concat([null])),
+ is_missing(object_concat([missing])),
+ is_null(object_concat([{"a":1}, null])),
+ is_missing(object_concat([{"a":1}, null, missing])),
+ is_null(object_concat([{"a":1}, 1])),
+ is_null(object_concat([{"a":1}, []])),
+ object_concat([{"a":1, "b":"x"}]),
+ object_concat([{"a":1, "b":"x" }, {"c":true, "d":false}, {"e":null}] ),
+ object_concat([{"a":1, "b":"x", "c":true }, {"a":2, "b":"y" }, {"b":null}]),
+ object_concat([{"a":1, "b": { "x":2, "y":3 } }, {"a":10, "b": { "x":4, "y":5 } }, {"a":100}]),
+ object_concat([{"a":1, "b": { "x":2, "y":3 } }, {"a":10, "b": { "x":4, "y":5 } }, {"a":100, "b": { "x":400, "y":500 } }])
+]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.3.query.sqlpp
new file mode 100644
index 0000000..ddfa0d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat/object_concat.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+SELECT VALUE
+object_concat((
+ SELECT VALUE object_add({}, i.id, i.label)
+ FROM [{"id":"test","label":"val"},{"id":"test2","label":"val1"}, {"id":"test2","label":"val2"}] i
+))
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.01.ddl.sqlpp
new file mode 100644
index 0000000..26e1cf3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.01.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+// test that object_concat() accepts and processes a single list of records argument
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE flat_t AS {a: int, b: string};
+CREATE TYPE nesting_t AS {x: {a: int, b: string}, y: [flat_t]};
+
+CREATE TYPE t1 AS {id: int, array_nesting_rec: [nesting_t], array_flat_rec: [flat_t]};
+CREATE DATASET ds(t1) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.02.update.sqlpp
new file mode 100644
index 0000000..6051c9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.02.update.sqlpp
@@ -0,0 +1,78 @@
+/*
+ * 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 ds [
+{"id": 1 ,"array_nesting_rec": [{"x": {"a": 3, "b": "3"}, "y": [{"a": 3, "b": "3"}]},
+ {"x": {"a": 2, "b": "2"}, "y": [{"a": 2, "b": "2"}]},
+ {"x": {"a": 1, "b": "1"}, "y": [{"a": 1, "b": "1"}]}]
+ ,"array_flat_rec": [{"a": 3, "b": "3"},
+ {"a": 2, "b": "2"},
+ {"a": 1, "b": "1"}]
+ ,"optional_field1": [{"x": {"a": 3, "b": "3"}, "y": [{"a": 3, "b": "3"}]},
+ {"x": {"a": 2, "b": "2"}, "y": [{"a": 2, "b": "2"}]},
+ {"x": {"a": 1, "b": "1"}, "y": [{"a": 1, "b": "1"}]}]
+ ,"optional_field2": [{"a": 3, "b": "3"},
+ {"a": 2, "b": "2"},
+ {"a": 1, "b": "1"}]
+},
+{"id": 2 ,"array_nesting_rec": [{"x": {"a": 3, "b": "3"}, "y": [{"a": 3, "b": "3"}]},
+ {"x": {"a": 1, "b": "1"}, "y": [{"a": 1, "b": "1"}]},
+ {"x": {"a": 2, "b": "2"}, "y": [{"a": 2, "b": "2"}]}]
+ ,"array_flat_rec": [{"a": 3, "b": "3"},
+ {"a": 1, "b": "1"},
+ {"a": 2, "b": "2"}]
+ ,"optional_field1": [{"x3": {"a": 3, "b": "3"}, "y3": [{"a": 3, "b": "3"}]},
+ {"x1": {"a": 1, "b": "1"}, "y1": [{"a": 1, "b": "1"}]},
+ {"x2": {"a": 2, "b": "2"}, "y2": [{"a": 2, "b": "2"}]}]
+ ,"optional_field2": [1,
+ {"a": 1, "b": "1"},
+ "3"]
+},
+{"id": 3 ,"array_nesting_rec": [{"x": {"a": 1, "b": "1"}, "y": [{"a": 1, "b": "1"}]},
+ {"x": {"a": 2, "b": "2"}, "y": [{"a": 2, "b": "2"}]},
+ {"x": {"a": 3, "b": "3"}, "y": [{"a": 3, "b": "3"}]}]
+ ,"array_flat_rec": [{"a": 1, "b": "1"},
+ {"a": 2, "b": "2"},
+ {"a": 3, "b": "3"}]
+ ,"optional_field1": 5
+ ,"optional_field2": [{"x1": {"a": 1, "b": "1"}, "y1": [{"a": 1, "b": "1"}]},
+ {"x2": {"a": 2, "b": "2"}, "y2": [{"a": 2, "b": "2"}]},
+ {"x3": {"a": 3, "b": "3"}, "y3": [{"a": 3, "b": "3"}]}]
+},
+{"id": 4 ,"array_nesting_rec": [{"x": {"a": 1, "b": "1"}, "y": [{"a": 1, "b": "1"}]},
+ {"x": {"a": 2, "b": "2"}, "y": [{"a": 2, "b": "2"}]},
+ {"x": {"a": 4, "b": "4"}, "y": [{"a": 4, "b": "4"}]}]
+ ,"array_flat_rec": [{"a": 1, "b": "1"},
+ {"a": 2, "b": "2"},
+ {"a": 4, "b": "4"}]
+ ,"optional_field1": {"x": {"a": 4, "b": "4"}, "y": [{"a": 4, "b": "4"}]}
+ ,"optional_field2": {"a": 4, "b": "4"}
+},
+{"id": 5 ,"array_nesting_rec": [{"x": {"a": 1, "b": "1"}, "y": [{"a": 1, "b": "1"}]},
+ {"x": {"a": 2, "b": "2"}, "y": [{"a": 2, "b": "2"}]},
+ {"x": {"a": 5, "b": "5"}, "y": [{"a": 5, "b": "5"}]}]
+ ,"array_flat_rec": [{"a": 1, "b": "1"},
+ {"a": 2, "b": "2"},
+ {"a": 5, "b": "5"}]
+ ,"optional_field1": null
+ /*"optional_field2": missing*/
+}
+];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.03.query.sqlpp
new file mode 100644
index 0000000..89bc6f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.03.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+
+FROM ds SELECT id, object_concat(array_nesting_rec) AS oc ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.04.query.sqlpp
new file mode 100644
index 0000000..b877817
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.04.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+
+FROM ds SELECT id, object_concat(array_flat_rec) AS oc ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.05.query.sqlpp
new file mode 100644
index 0000000..b933712
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.05.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+
+FROM ds SELECT id, object_concat(optional_field1) AS oc ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.06.query.sqlpp
new file mode 100644
index 0000000..c5d5ab4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.06.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+
+FROM ds SELECT id, object_concat(optional_field2) AS oc ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.07.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.07.query.sqlpp
new file mode 100644
index 0000000..5611a1d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.07.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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;
+
+FROM ds SELECT id, object_concat(optional_field1, optional_field2) AS oc ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.99.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.99.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_concat_with_array/object_concat_with_array.99.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.1.ddl.sqlpp
deleted file mode 100644
index 8b1303f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.1.ddl.sqlpp
+++ /dev/null
@@ -1,53 +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 : Do not pushdown if the whole record was requested
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/id_age.parquet"),
- ("input-format"="parquet-input-format")
-);
-
-CREATE EXTERNAL DATASET ParquetDataset2(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/id_age.parquet"),
- ("input-format"="parquet-input-format")
-);
-
-CREATE EXTERNAL DATASET ParquetDataset3(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/id_name_comment.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.10.query.sqlpp
deleted file mode 100644
index 828b598..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.10.query.sqlpp
+++ /dev/null
@@ -1,31 +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 : Get only name and age
-* Expected Res : Success
-* Date : August 30th 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-EXPLAIN
-SELECT p1.age, p2.name
-FROM ParquetDataset p1, ParquetDataset3 p2
-WHERE p1.id = p2.id
-ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.11.ddl.sqlpp
deleted file mode 100644
index e9a8b3f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.11.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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 : Get common field
-* Expected Res : Success
-* Date : August 30 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/dummy_tweet.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.12.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.12.update.sqlpp
deleted file mode 100644
index c5796e0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.12.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Get common field
-* Expected Res : Success
-* Date : August 30 2020
-*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.13.query.sqlpp
deleted file mode 100644
index 10d0788..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.13.query.sqlpp
+++ /dev/null
@@ -1,30 +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 : Get common field
-* Expected Res : Success
-* Date : August 30 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-
-SELECT p.user.id, p.user.name
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.14.query.sqlpp
deleted file mode 100644
index f726e8d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.14.query.sqlpp
+++ /dev/null
@@ -1,30 +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 : Get common field
-* Expected Res : Success
-* Date : August 30 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-EXPLAIN
-SELECT p.user.id, p.user.name
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.2.update.sqlpp
deleted file mode 100644
index 8e71a42..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.2.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Do not pushdown if the whole record was requested
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.3.query.sqlpp
deleted file mode 100644
index 4ba4649..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.3.query.sqlpp
+++ /dev/null
@@ -1,29 +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 : Do not pushdown if the whole record was requested
- This case, the expression 'p.id' should not be pushed down
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SELECT VALUE p
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.4.query.sqlpp
deleted file mode 100644
index cea66ff..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.4.query.sqlpp
+++ /dev/null
@@ -1,31 +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 : Do not pushdown if the whole record was requested
- This case, the expression 'p.id' should not be pushed down
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-
-SELECT VALUE p
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.5.query.sqlpp
deleted file mode 100644
index 9492cdb..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.5.query.sqlpp
+++ /dev/null
@@ -1,31 +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 : Do not pushdown if the whole record was requested
- This case, the expression 'p.id' should not be pushed down
-* Expected Res : Success
-* Date : August 30th 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-EXPLAIN
-SELECT VALUE p
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.6.query.sqlpp
deleted file mode 100644
index d24c258..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.6.query.sqlpp
+++ /dev/null
@@ -1,30 +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 : Do not pushdown if the whole record was requested
- In this join, 'p2.id' should be pushed down, but not 'p1.id'
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SELECT p1, p2.id
-FROM ParquetDataset p1, ParquetDataset2 p2
-WHERE p1.id = p2.id
-ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.7.query.sqlpp
deleted file mode 100644
index eb2d8e9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.7.query.sqlpp
+++ /dev/null
@@ -1,32 +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 : Do not pushdown if the whole record was requested
- In this join, 'p2.id' should be pushed down, but not 'p1.id'
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-
-SELECT p1, p2.id
-FROM ParquetDataset p1, ParquetDataset2 p2
-WHERE p1.id = p2.id
-ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.8.query.sqlpp
deleted file mode 100644
index 4e60665..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.8.query.sqlpp
+++ /dev/null
@@ -1,32 +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 : Do not pushdown if the whole record was requested
- In this join, 'p2.id' should be pushed down, but not 'p1.id'
-* Expected Res : Success
-* Date : August 30th 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-EXPLAIN
-SELECT p1, p2.id
-FROM ParquetDataset p1, ParquetDataset2 p2
-WHERE p1.id = p2.id
-ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.9.query.sqlpp
deleted file mode 100644
index 069c26a..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.9.query.sqlpp
+++ /dev/null
@@ -1,31 +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 : Get only name and age
-* Expected Res : Success
-* Date : August 30th 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-
-SELECT p1.age, p2.name
-FROM ParquetDataset p1, ParquetDataset3 p2
-WHERE p1.id = p2.id
-ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.1.ddl.sqlpp
deleted file mode 100644
index 1e24678..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.1.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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 : Requesting non-existing fields should not fail
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/dummy_tweet.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.2.update.sqlpp
deleted file mode 100644
index 48ae519..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.2.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Requesting non-existing fields should not fail
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.3.query.sqlpp
deleted file mode 100644
index 1715bba..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.3.query.sqlpp
+++ /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 : Requesting non-existing fields should not fail
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SELECT p.not_a_field1 IS MISSING as f1, p.user.not_a_field2 IS MISSING as f2
-FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
deleted file mode 100644
index 7fc6176..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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 : Retrieve all fields from different Parquet files with different schemas
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/id_name.parquet,/asterix/id_age.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.2.update.sqlpp
deleted file mode 100644
index a4fe4c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.2.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Retrieve all fields from different Parquet files with different schemas
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
deleted file mode 100644
index 910f20c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
+++ /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 : Retrieve all fields from different Parquet files with different schemas
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SELECT VALUE p
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.1.ddl.sqlpp
deleted file mode 100644
index db6c4d9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.1.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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 : Concat two objects after pushdown
-* Expected Res : Success
-* Date : September 22nd 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/dummy_tweet.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.2.update.sqlpp
deleted file mode 100644
index 334beeb..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.2.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Concat two objects after pushdown
-* Expected Res : Success
-* Date : September 22nd 2020
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.3.query.sqlpp
deleted file mode 100644
index a429b73..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.3.query.sqlpp
+++ /dev/null
@@ -1,29 +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 : Concat two objects after pushdown
-* Expected Res : Success
-* Date : September 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-SELECT VALUE object_concat(p.coordinates, p.user).name
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.4.query.sqlpp
deleted file mode 100644
index dd114e6..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.4.query.sqlpp
+++ /dev/null
@@ -1,30 +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 : Concat two objects after pushdown
-* Expected Res : Success
-* Date : September 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-EXPLAIN
-SELECT VALUE object_concat(p.coordinates, p.user).name
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.5.query.sqlpp
deleted file mode 100644
index 06bd91c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.5.query.sqlpp
+++ /dev/null
@@ -1,29 +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 : Concat two objects after pushdown
-* Expected Res : Success
-* Date : September 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-SELECT VALUE object_concat(p.coordinates, p.user)
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.6.query.sqlpp
deleted file mode 100644
index 0b42622..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.6.query.sqlpp
+++ /dev/null
@@ -1,30 +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 : Concat two objects after pushdown
-* Expected Res : Success
-* Date : September 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-EXPLAIN
-SELECT VALUE object_concat(p.coordinates, p.user)
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp
deleted file mode 100644
index 20454cb..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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 : Retrieve all fields from a Parquet file
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/dummy_tweet.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.2.update.sqlpp
deleted file mode 100644
index 6b0c1e9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.2.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Retrieve all fields from a Parquet file
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.3.query.sqlpp
deleted file mode 100644
index 51a7b3e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.3.query.sqlpp
+++ /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 : Retrieve all fields from a Parquet file
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SELECT VALUE p
-FROM ParquetDataset p
-ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp
deleted file mode 100644
index 17ebc63..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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 : Retrieve the number of texts in all tweets
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/dummy_tweet.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.2.update.sqlpp
deleted file mode 100644
index c39aabc..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.2.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Retrieve the number of texts in all tweets
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp
deleted file mode 100644
index 491d789..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp
+++ /dev/null
@@ -1,29 +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 : Retrieve the number of texts in all tweets (with expression pushdown)
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-
-SELECT VALUE count(p.text)
-FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp
deleted file mode 100644
index 84639bf..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp
+++ /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 : Retrieve the number of texts in all tweets
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SELECT VALUE count(p.text)
-FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp
deleted file mode 100644
index dddb9b3..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp
+++ /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 : Retrieve the number of users
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SELECT VALUE count(p.user.name)
-FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.6.query.sqlpp
deleted file mode 100644
index 76ac0ce..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.6.query.sqlpp
+++ /dev/null
@@ -1,29 +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 : Retrieve the number of users (with expression pushdown)
-* Expected Res : Success
-* Date : June 22nd 2020
-*/
-USE test;
-
-SET `compiler.external.field.pushdown` "true";
-
-SELECT VALUE count(p.user.name)
-FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
deleted file mode 100644
index 650e3a0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
+++ /dev/null
@@ -1,39 +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 : Test Standard UTF-8
-* Expected Res : Success
-* Date : August 30th 2020
-*/
-
-DROP DATAVERSE test IF EXISTS;
-CREATE DATAVERSE test;
-
-USE test;
-
-
-CREATE TYPE ParquetType as {
-};
-
-CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
-(
- ("hdfs"="hdfs://127.0.0.1:31888"),
- ("path"="/asterix/id_name_comment.parquet"),
- ("input-format"="parquet-input-format")
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.2.update.sqlpp
deleted file mode 100644
index 01270f3..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.2.update.sqlpp
+++ /dev/null
@@ -1,23 +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 : Test Standard UTF-8
-* Expected Res : Success
-* Date : August 30th 2020
-*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp
deleted file mode 100644
index f849a1f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp
+++ /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 : Test Standard UTF-8
-* Expected Res : Success
-* Date : August 30th 2020
-*/
-USE test;
-
-SELECT VALUE array_count(split(trim(p.comment),"𩸽"))
-FROM ParquetDataset p
-WHERE contains(p.comment, "𩸽");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.1.ddl.sqlpp
new file mode 100644
index 0000000..fb19ba8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE TestDataverse IF EXISTS;
+CREATE DATAVERSE TestDataverse;
+USE TestDataverse;
+
+CREATE TYPE TestType AS {
+ _id: uuid
+};
+
+CREATE DATASET Dataset1 (TestType)
+PRIMARY KEY _id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.2.update.sqlpp
new file mode 100644
index 0000000..97cda55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 TestDataverse;
+
+INSERT INTO Dataset1 [
+ { "items": [], "other_items": [1, 2, 3] },
+ { "items": null, "other_items": [1, 2, 3] },
+ { "other_items": [1, 2, 3] },
+ { "items": [1, 2, 3], "other_items": [1, 2, 3 ] },
+ { "items": [1, 2], "other_items": [1, 2, 3] },
+ { "items": [1, 2, 3, 4], "other_items": [1, 2, 3] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.3.query.sqlpp
new file mode 100644
index 0000000..1c87dbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 TestDataverse;
+FROM Dataset1 D
+WHERE ANY AND EVERY I IN D.items
+ SATISFIES I BETWEEN 1 AND 3
+SELECT D.items
+ORDER BY D.items;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.4.query.sqlpp
new file mode 100644
index 0000000..85bf8c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 TestDataverse;
+FROM Dataset1 D
+WHERE ANY AND EVERY I IN D.items, J IN D.other_items
+ SATISFIES I BETWEEN 1 AND 3 AND J BETWEEN 1 AND 3
+SELECT D.items, D.other_items
+ORDER BY D.items, D.other_items;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.1.ddl.sqlpp
new file mode 100644
index 0000000..0ab290f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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: This test case is to verify the fix for ASTERIXDB-2947
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id uuid not unknown) open type primary key id autogenerated;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.2.update.sqlpp
new file mode 100644
index 0000000..5dcf4b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.2.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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 t1
+([
+ {"x":1, "y":"A" },
+ {"x":1, "y":"B" },
+ {"x":1, "y":"C" },
+
+ {"x":2, "y":"D" },
+ {"x":2, "y":"E" },
+ {"x":2, "y":"F" },
+
+ {"x":3, "y":"G" },
+ {"x":3, "y":"H" },
+ {"x":3, "y":"I" }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.query.sqlpp
new file mode 100644
index 0000000..712b8d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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: This test case is to verify the fix for ASTERIXDB-2947
+ */
+
+use test;
+
+select x, array_sort(array_agg({y})) yy
+from t1
+group by x
+having some r in array_agg({y}) satisfies r.y in ["B", "H"]
+order by x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.1.sto.cmd b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.1.sto.cmd
index 7ddaa20..ee4684a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.1.sto.cmd
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.1.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-nc:asterix_nc1 /addReplica 0 asterix_nc2
\ No newline at end of file
+nc:asterix_nc1 /addReplica 1 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.2.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.2.pollget.http
index c007931..178c5a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.2.pollget.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.2.pollget.http
@@ -19,4 +19,4 @@
//polltimeoutsecs=30
//prettifyjsonresult=true
-nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
+nc:asterix_nc1 /admin/storage/partition/1
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http
index 86363c0..cae5bd5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http
@@ -16,6 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-# param partition=0
+# param partition=1
nc:asterix_nc1 /admin/storage/release
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.4.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.4.get.http
index 53d75bc..bb4a566 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.4.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.4.get.http
@@ -18,4 +18,4 @@
*/
//prettifyjsonresult=true
-nc:asterix_nc1:19004 /admin/storage/partition/0
\ No newline at end of file
+nc:asterix_nc1:19004 /admin/storage/partition/1
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/scan/alltypes_01/alltypes_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/scan/alltypes_01/alltypes_01.4.query.sqlpp
new file mode 100644
index 0000000..21f37f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/scan/alltypes_01/alltypes_01.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+{
+ "t1_array_of_unknown": [ null, missing ],
+ "t2_multiset_of_unknown": {{ null, missing }}
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.ddl.sqlpp
new file mode 100644
index 0000000..aefab51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.2.update.sqlpp
new file mode 100644
index 0000000..a56ea7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.3.query.sqlpp
new file mode 100644
index 0000000..4c521a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.ddl.sqlpp
new file mode 100644
index 0000000..aefab51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.2.update.sqlpp
new file mode 100644
index 0000000..a56ea7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.3.query.sqlpp
new file mode 100644
index 0000000..f123788
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.1.ddl.sqlpp
new file mode 100644
index 0000000..5ba0acb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE RectangleType AS CLOSED {
+ id: uuid,
+ name: string,
+ g: rectangle
+};
+
+CREATE DATASET RectangleDataset(RectangleType) PRIMARY KEY id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.2.update.sqlpp
new file mode 100644
index 0000000..1876c09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 RectangleDataset {"name":"r1", "g":create_rectangle(create_point(0.0,0.0),create_point(1.0,1.0))};
+INSERT INTO RectangleDataset {"name":"r1", "g":create_rectangle(create_point(0.0,0.0),create_point(1.0,1.0))};
+INSERT INTO RectangleDataset {"name":"r2", "g":create_rectangle(create_point(2.0,2.0),create_point(3.0,3.0))};
+INSERT INTO RectangleDataset {"name":"r2", "g":create_rectangle(create_point(2.0,2.0),create_point(3.0,3.0))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.3.query.sqlpp
new file mode 100644
index 0000000..d07707c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_projection_check/spatial_join_projection_check.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT r1.name
+FROM RectangleDataset r1, RectangleDataset r2
+WHERE spatial_intersect(r1.g, r2.g)
+ORDER BY r1.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.ddl.sqlpp
new file mode 100644
index 0000000..aefab51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.2.update.sqlpp
new file mode 100644
index 0000000..a56ea7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.3.query.sqlpp
new file mode 100644
index 0000000..4c521a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.ddl.sqlpp
new file mode 100644
index 0000000..5550d51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+ id: int32,
+ geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Type for the geometry datasets
+CREATE TYPE GeomType AS{
+ id : int,
+ geom : geometry
+};
+
+-- Create datasets
+CREATE DATASET ParkSetG (GeomType) primary key id;
+CREATE DATASET LakeSetG (GeomType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.2.update.sqlpp
new file mode 100644
index 0000000..2834e20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
+-- make geometries and insert into new datasets
+insert into LakeSetG select id, st_make_envelope(get_x (get_points (geom)[0]) ,get_y (get_points (geom)[0]) ,get_x (get_points (geom)[1]) ,get_y (get_points (geom)[1]) , 4326) geom from LakeSet;
+insert into ParkSetG select id, st_make_envelope(get_x (get_points (geom)[0]) ,get_y (get_points (geom)[0]) ,get_x (get_points (geom)[1]) ,get_y (get_points (geom)[1]) , 4326) geom from ParkSet;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.3.query.sqlpp
new file mode 100644
index 0000000..2b59fc5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+FROM ParkSetG AS ps LEFT OUTER JOIN LakeSetG AS ls
+ON st_intersects(ps.geom, ls.geom)
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.1.query.sqlpp
new file mode 100644
index 0000000..9f94406
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test IN (scalar) in SQL-compat mode
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select value r
+from range(0, 3) r
+where r in (2)
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.2.query.sqlpp
new file mode 100644
index 0000000..9a9c282
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test IN scalar in SQL-compat mode
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select value r
+from range(0, 3) r
+where r in 2
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.3.query.sqlpp
new file mode 100644
index 0000000..c6be9b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test NOT IN (scalar) in SQL-compat mode
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select value r
+from range(0, 3) r
+where r not in (2)
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.4.query.sqlpp
new file mode 100644
index 0000000..e706c07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/in_non_list_01/in_non_list_01.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test NOT IN scalar in SQL-compat mode
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select value r
+from range(0, 3) r
+where r not in 2
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..909b74c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* Test that outer join emits NULLs for non-matching values */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
+
+create dataset ds2(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp
new file mode 100644
index 0000000..65e9d54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 ds1 ({"id": 1, "a": 1, "b": 1 });
+insert into ds1 ({"id": 2, "a": 1, "b": 2 });
+insert into ds1 ({"id": 3, "a": 2, "b": 3 });
+insert into ds1 ({"id": 4, "a": 2, "b": 4 });
+insert into ds1 ({"id": 5, "a": 3, "b": 5 });
+insert into ds1 ({"id": 6, "a": 3, "b": 6 });
+insert into ds1 ({"id": 7, "a": 4, "b": 7 });
+
+insert into ds2 ({"id": 100, "x": 2, "y": 100 });
+insert into ds2 ({"id": 101, "x": 2, "y": 101 });
+insert into ds2 ({"id": 102, "x": 3, "y": 102 });
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp
new file mode 100644
index 0000000..c2edfe6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / Hash */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on ds1.a = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp
new file mode 100644
index 0000000..3f85f0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / Hash */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on ds1.a = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp
new file mode 100644
index 0000000..6ae5a89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on (ds1.a - ds2.x >= 0 and ds2.x - ds1.a >= 0)
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp
new file mode 100644
index 0000000..e4a5ef5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on (ds1.a - ds2.x >= 0 and ds2.x - ds1.a >= 0)
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp
new file mode 100644
index 0000000..a58785a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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;
+
+create index idx_x on ds2(x:int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp
new file mode 100644
index 0000000..279ed0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / Index-NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on bigint(ds1.a) /* +indexnl */ = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp
new file mode 100644
index 0000000..e6b0c54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / Index-NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on bigint(ds1.a) /* +indexnl */ = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp
new file mode 100644
index 0000000..5f5c3fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/* Test that outer unnest emits NULLs if there are no items in the input list */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp
new file mode 100644
index 0000000..1fb895a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 ds1 ({"id": 1, "a": [1] });
+insert into ds1 ({"id": 2, "a": [1, 2] });
+insert into ds1 ({"id": 3, "a": [] });
+insert into ds1 ({"id": 4, "a": null });
+insert into ds1 ({"id": 5 /*a:missing*/ });
+insert into ds1 ({"id": 6, "a": [1,2,3] });
+insert into ds1 ({"id": 7, "a": [1, missing, null, 4] });
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp
new file mode 100644
index 0000000..ec5c8af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Test that outer unnest emits NULLs if there are no items in the input list */
+
+use test;
+
+select ds1.id, ai
+from ds1 left outer unnest ds1.a as ai
+order by ds1.id, ai;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp
new file mode 100644
index 0000000..02b34a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Test that outer unnest emits NULLs for positional variable as well */
+
+use test;
+
+select ds1.id, ai, pi
+from ds1 left outer unnest ds1.a as ai at pi
+order by ds1.id, ai;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/select_star_01/select_star_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/select_star_01/select_star_01.1.query.sqlpp
new file mode 100644
index 0000000..87d2207
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/select_star_01/select_star_01.1.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+/*
+ * Test SELECT * in sql-compat mode
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+with T as (
+ select r x, -r y
+ from range(1, 2) r
+)
+
+select *
+from T t
+order by x;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/select_star_02_negative/select_star_02_negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/select_star_02_negative/select_star_02_negative.1.query.sqlpp
new file mode 100644
index 0000000..c1b4824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/select_star_02_negative/select_star_02_negative.1.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+ * Test that SELECT * fails in sql-compat mode if
+ * FROM clause has more than one variable
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+with T as (
+ select r x, -r y
+ from range(1, 2) r
+)
+
+select *
+from T t1, T t2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.1.query.sqlpp
new file mode 100644
index 0000000..681abe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.1.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * SELECT subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1, (select max(r2) from range(0, r1) r2) r3
+from range(1,3) r1
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.10.query.sqlpp
new file mode 100644
index 0000000..4b8d3be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * SELECT subquery with ORDER BY and LIMIT
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1, (select r2 as r3 from range(0, r1) r2 order by r3 desc limit 1) r3
+from range(1,3) r1
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.2.query.sqlpp
new file mode 100644
index 0000000..de72fe5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.2.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * Subquery that does not return a single tuple is coerced to MISSING
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1, (select r2 from range(0, r1) r2 where r2 > 1) is missing r3
+from range(1,3) r1
+order by r1;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.3.query.sqlpp
new file mode 100644
index 0000000..5776ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.3.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * WHERE subquery ...
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1,3) r1
+where (select max(r2) > 0 from range(0, r1) r2)
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.4.query.sqlpp
new file mode 100644
index 0000000..6c0bc36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.4.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * GROUP BY subquery ...
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select x, array_sort(array_agg(r1)) r1
+from range(1,3) r1
+group by ((select max(r2) > 1 from range(0, r1) r2) x)
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.5.query.sqlpp
new file mode 100644
index 0000000..f3fe9ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.5.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * HAVING subquery ...
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select k
+from range(1,6) r1
+group by r1 % 3 k group as g
+having (select max(gi.r1) != 6 from g gi)
+order by k;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.6.query.sqlpp
new file mode 100644
index 0000000..6d8c526
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.6.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * Subquery in arithmetic operator
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1,
+ (select max(r2) from range(0, r1+1) r2) - (select max(r3) from range(0, r1) r3) r2
+from range(1,3) r1
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.7.query.sqlpp
new file mode 100644
index 0000000..5bf519f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.7.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ *
+ * WITH var AS subquery, LET var = subquery -- no coercion
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+with r4 as (select max(r5) from range(0, 2) r5)
+select r1, r3, r4
+from range(1,3) r1
+let r3 = (select max(r2) from range(0, r1) r2)
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.8.query.sqlpp
new file mode 100644
index 0000000..682ba12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * SELECT VALUE subquery is not coerced
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1, (select value max(r2) from range(0, r1) r2) r3
+from range(1,3) r1
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.9.query.sqlpp
new file mode 100644
index 0000000..a6f397e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.9.query.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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * Subquery has UNION ALL with ORDER BY / LIMIT
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1,
+(
+ select max(r2) m2
+ from range(0, r1+1) r2
+ union all
+ select max(r3) m2
+ from range(0, r1) r3
+ order by m2
+ limit 1
+) r3
+from range(1,3) r1
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.1.query.sqlpp
new file mode 100644
index 0000000..f379f5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.1.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * FAILURE: subquery returns more than one field
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1, (select min(r2), max(r2) from range(0, r1) r2) r3
+from range(1,3) r1
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.2.query.sqlpp
new file mode 100644
index 0000000..57aab66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.2.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * FAILURE: SELECT * is not supported in subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1, (select * from range(0, r1) r2, range(0, r1+1) r3)
+from range(1,3) r1
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.3.query.sqlpp
new file mode 100644
index 0000000..63a4a20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.3.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * FAILURE: UNION ALL between SELECT and SELECT VALUE
+ * (this is currently not supported)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1,
+(
+ select r2 from range(0, 1) r2 where r2 < 1
+ union all
+ select value r3 from range(0, 1) r3 where r3 > 1
+)
+from range(1, 3) r1
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.4.query.sqlpp
new file mode 100644
index 0000000..08c8bce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_02_scalar_negative/subquery_coercion_02_scalar_negative.4.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's
+ * outside of IN/NOT IN, comparison operators,
+ * FROM/JOIN/UNNEST clauses
+ *
+ * FAILURE: UNION ALL between SELECT and subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1,
+(
+ select r2 from range(0, 1) r2 where r2 < 1
+ union all
+ (let x = 1 select x)
+)
+from range(1, 3) r1
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.1.query.sqlpp
new file mode 100644
index 0000000..06beeea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.1.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 3) r1
+where r1 = (select null_if(max(r2), 2) from range(0, r1) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.10.query.sqlpp
new file mode 100644
index 0000000..1b533f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.10.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * Test when the subquery returns more than one tuple
+ * (coercion rewriting should produce MISSING and therefore an empty result)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*) cnt
+from [
+ { "r": 1, "x": [ { "y": 0 }, { "y": 1 } ] },
+ { "r": 2, "x": [ { "y": 0 }, { "y": 1 }, { "y": 2 } ] },
+ { "r": 3, "x": [ { "y": 0 }, { "y": 1 }, { "y": 2 }, { "y": 3 } ] }
+] v
+where v.x = (select y from range(0, v.r) y);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.11.query.sqlpp
new file mode 100644
index 0000000..adcd52e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.11.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = subquery, rhs = expr
+ *
+ * Test when the subquery returns more than one tuple
+ * (coercion rewriting should produce MISSING)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*) cnt
+from [
+ { "r": 1, "x": [ { "y": 0 }, { "y": 1 } ] },
+ { "r": 2, "x": [ { "y": 0 }, { "y": 1 }, { "y": 2 } ] },
+ { "r": 3, "x": [ { "y": 0 }, { "y": 1 }, { "y": 2 }, { "y": 3 } ] }
+] v
+where (select y from range(0, v.r) y) is not distinct from missing;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.12.query.sqlpp
new file mode 100644
index 0000000..01de9f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.12.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * NO COERCION for SELECT VALUE
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 3) r1
+let y = [{"x": r1}]
+where y = (select value { "x" : null_if(max(r2), 2) } from range(0, r1) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.13.query.sqlpp
new file mode 100644
index 0000000..f3f3277
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.13.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit array constructor expression
+ *
+ * lhs = (e1,e2), rhs = subquery
+ *
+ * NO COERCION for SELECT VALUE
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where ({"x": r1, "y": r1+1}, {"x": r1+1, "y": r1+2})
+ = (select value { "x": r2 - 1, "y": r2 } from range(2, 3) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.14.query.sqlpp
new file mode 100644
index 0000000..d41e202
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.14.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit array constructor expression
+ *
+ * lhs = (e1,e2), rhs = subquery
+ *
+ * NO COERCION for UNION ALL of SELECT VALUE
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where ({"x": r1, "y": r1+1}, {"x": r1+1, "y": r1+2})
+ = (
+ select value { "x": r2 - 1, "y": r2 } from range(2, 3) r2 where r2 < 3
+ union all
+ select value { "x": r2 - 1, "y": r2 } from range(2, 3) r2 where r2 > 2
+ )
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.15.query.sqlpp
new file mode 100644
index 0000000..326ebd2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.15.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery with ORDER BY and LIMIT
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 3) r1
+where r1 = (select r2 as r3 from range(0, r1) r2 order by r3 desc limit 1)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.2.query.sqlpp
new file mode 100644
index 0000000..00e2a27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.2.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = subquery, rhs = expr
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 3) r1
+where (select null_if(max(r2), 2) from range(0, r1) r2) = r1
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.3.query.sqlpp
new file mode 100644
index 0000000..820e920
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.3.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = subquery, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 3) r1
+where
+ (select null_if(max(r2), 2) from range(0, r1) r2)
+ =
+ (select null_if(max(-r2), 2) from range(-r1, 0) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.4.query.sqlpp
new file mode 100644
index 0000000..b4f402e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.4.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit array constructor expression
+ *
+ * lhs = (e1,e2), rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where (r1, r1+1) = (select r2 - 1, r2 from range(2, 3) r2 where r2 < 3)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.5.query.sqlpp
new file mode 100644
index 0000000..99d120c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.5.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit array constructor expression
+ *
+ * lhs = subquery, rhs = (e1,e2)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where (select r2 - 1, r2 from range(2, 3) r2 where r2 < 3) = (r1, r1+1)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.6.query.sqlpp
new file mode 100644
index 0000000..c5ebf84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.6.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit array constructor expression
+ *
+ * lhs = [e1,e2], rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where [r1, r1+1] = (select r2 - 1, r2 from range(2, 3) r2 where r2 < 3)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.7.query.sqlpp
new file mode 100644
index 0000000..93338c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit array constructor expression
+ *
+ * lhs = subquery, rhs = [e1,e2]
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where (select r2 - 1, r2 from range(2, 3) r2 where r2 < 3) = [r1, r1+1]
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.8.query.sqlpp
new file mode 100644
index 0000000..13c2439
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit multiset constructor expression
+ *
+ * lhs = {{e1,e2}}, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where {{r1, r1+1}} = (select r2 - 1, r2 from range(2, 3) r2 where r2 < 3)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.9.query.sqlpp
new file mode 100644
index 0000000..c131067
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.9.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into an array when it's compared
+ * with an explicit multiset constructor expression
+ *
+ * lhs = subquery, rhs = {{e1,e2}}
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(0, 1) r1
+where (select r2 - 1, r2 from range(2, 3) r2 where r2 < 3) = {{r1, r1+1}}
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.1.query.sqlpp
new file mode 100644
index 0000000..b50b5fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.1.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: subquery returns more than one field
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 = (select r2, r2+1 from range(0, r1) r2);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.2.query.sqlpp
new file mode 100644
index 0000000..73bec19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.2.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = subquery, rhs = expr
+ *
+ * FAILURE: subquery returns more than one field
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where (select r2, r2+1 from range(0, r1) r2) = r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.3.query.sqlpp
new file mode 100644
index 0000000..d15b588
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.3.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: UNION ALL between SELECT and SELECT VALUE
+ * (this is currently not supported)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 = (
+ select r2 from range(0, 1) r2 where r2 < 1
+ union all
+ select value r3 from range(0, 1) r3 where r3 > 1
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.4.query.sqlpp
new file mode 100644
index 0000000..8b61916
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.4.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: UNION ALL between SELECT VALUE and SELECT
+ * (this is currently not supported)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 =
+(
+ select value r3 from range(0, 1) r3 where r3 > 1
+ union all
+ select r2 from range(0, 1) r2 where r2 < 1
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.5.query.sqlpp
new file mode 100644
index 0000000..ad05ecf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.5.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: UNION ALL between SELECT and subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 = (
+ select r2 from range(0, 1) r2 where r2 < 1
+ union all
+ (let x = 1 select x)
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.6.query.sqlpp
new file mode 100644
index 0000000..5f55753
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.6.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: SELECT * is not supported in subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 = (select * from range(0, r1) r2);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.7.query.sqlpp
new file mode 100644
index 0000000..3e5dc56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.7.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when it's compared
+ * with an expression which is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: SELECT * is not supported in subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where
+ (select * from range(0, r1) r2) = (select * from range(0, r1) r3);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.8.query.sqlpp
new file mode 100644
index 0000000..2b2998e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_04_cmp_negative/subquery_coercion_04_cmp_negative.8.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into array when it's compared
+ * with an expression which is an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: SELECT * is not supported in subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where
+ (r1, r1+1) =
+ (select * from range(0, r1) r2, range(0, r1+1) r3);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.1.query.sqlpp
new file mode 100644
index 0000000..7f9d531
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.1.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where r1 IN (select r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.10.query.sqlpp
new file mode 100644
index 0000000..ab2f0dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.10.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the NOT IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery UNION ALL subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 6) r1
+where r1 NOT IN (
+ select r2 from range(1, 2) r2
+ UNION ALL
+ select r3 from range(5, 6) r3
+)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.11.query.sqlpp
new file mode 100644
index 0000000..d061efe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.11.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is an explicit array constructor
+ * and subquery contains an ORDER BY clause
+ *
+ * lhs = expr, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+select r1
+from range(1, 5) r1
+where r1 IN (select r2 as r3 from range(1, 5) r2 order by r3 desc limit 2)
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.12.query.sqlpp
new file mode 100644
index 0000000..0940aa8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.12.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is not an explicit array constructor
+ * and subquery contains ORDER BY and UNION ALL clauses
+ *
+ * lhs = expr, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+select r1
+from range(1, 5) r1
+where r1 IN (
+ select r2 as r3 from [1, 5] r2
+ union all
+ select r4 as r3 from [2, 4] r4
+ order by r3 desc
+ limit 2
+)
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.13.query.sqlpp
new file mode 100644
index 0000000..c6eabaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.13.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is an explicit array constructor
+ * and subquery contains an ORDER BY clause
+ *
+ * lhs = expr, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+select r1
+from range(1, 5) r1
+where (r1, r1+1) IN (
+ select r2 as r3, r2+1 from range(1, 5) r2 order by r3 desc limit 2
+ )
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.14.query.sqlpp
new file mode 100644
index 0000000..82df0ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.14.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is an explicit array constructor
+ * and subquery contains ORDER BY and UNION ALL clauses
+ *
+ * lhs = expr, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+select r1
+from range(1, 5) r1
+where (r1, r1+1) IN (
+ select r2 as r3, r2+1 r5 from [1, 5] r2
+ union all
+ select r4 as r3, r4+1 r5 from [2, 4] r4
+ order by r3 desc
+ limit 2
+)
+order by r1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.2.query.sqlpp
new file mode 100644
index 0000000..0eb8e05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.2.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is an explicit array constructor
+ *
+ * lhs = (e1,e2), rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where (r1-1, r1) IN (select r2-1, r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.3.query.sqlpp
new file mode 100644
index 0000000..b1d48fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.3.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is an explicit array constructor
+ *
+ * lhs = [e1,e2], rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where [r1-1, r1] IN (select r2-1, r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.4.query.sqlpp
new file mode 100644
index 0000000..0d8edc9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.4.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is an explicit multiset constructor
+ *
+ * lhs = {{e1,e2}}, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where {{r1-1, r1}} IN (select r2-1, r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.5.query.sqlpp
new file mode 100644
index 0000000..3886216
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.5.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery UNION ALL subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 6) r1
+where r1 IN (
+ select r2 from range(1, 2) r2
+ UNION ALL
+ select r3 from range(5, 6) r3
+)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.6.query.sqlpp
new file mode 100644
index 0000000..b04b0e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.6.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the NOT IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where r1 NOT IN (select r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.7.query.sqlpp
new file mode 100644
index 0000000..9d8f18e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.7.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the NOT IN operator
+ * and lhs expression is an explicit array constructor
+ *
+ * lhs = (e1,e2), rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where (r1-1, r1) NOT IN (select r2-1, r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.8.query.sqlpp
new file mode 100644
index 0000000..cdde8e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.8.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the NOT IN operator
+ * and lhs expression is an explicit array constructor
+ *
+ * lhs = [e1,e2], rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where [r1-1, r1] NOT IN (select r2-1, r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.9.query.sqlpp
new file mode 100644
index 0000000..db0c322
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.9.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the NOT IN operator
+ * and lhs expression is an explicit multiset constructor
+ *
+ * lhs = {{e1,e2}}, rhs = subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r1
+from range(1, 5) r1
+where {{r1-1, r1}} NOT IN (select r2-1, r2 from range(0, r1-(r1 % 2)) r2)
+order by r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.1.query.sqlpp
new file mode 100644
index 0000000..72a003e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.1.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: subquery returns more than one field
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 IN (select r2, r2+1 from range(0, r1) r2);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.2.query.sqlpp
new file mode 100644
index 0000000..f28d860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.2.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the NOT IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: subquery returns more than one field
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 NOT IN
+ (select r2, r2+1 from range(0, r1) r2);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.3.query.sqlpp
new file mode 100644
index 0000000..9642dff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.3.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: UNION ALL between SELECT and SELECT VALUE
+ * (this is currently not supported)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 IN (
+ select r2 from range(0, 1) r2 where r2 < 1
+ union all
+ select value r3 from range(0, 1) r3 where r3 > 1
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.4.query.sqlpp
new file mode 100644
index 0000000..d7bf919
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.4.query.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the NOT IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: UNION ALL between SELECT VALUE and SELECT
+ * (this is currently not supported)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 NOT IN
+(
+ select value r2 from range(0, 1) r2 where r2 < 1
+ union all
+ select r3 from range(0, 1) r3 where r3 > 1
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.5.query.sqlpp
new file mode 100644
index 0000000..8cfb6f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.5.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: UNION ALL between SELECT and subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 IN (
+ select r2 from range(0, 1) r2 where r2 < 1
+ union all
+ (let x = 1 select x)
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.6.query.sqlpp
new file mode 100644
index 0000000..575b6d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.6.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into a scalar value when
+ * its the rhs argument of the IN operator
+ * and lhs expression is not an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: SELECT * is not supported in subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where r1 IN (select * from range(0, r1) r2);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.7.query.sqlpp
new file mode 100644
index 0000000..bee7d0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_06_in_negative/subquery_coercion_06_in_negative.7.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test subquery coercion into array when
+ * its the rhs argument of the IN operator
+ * and lhs expression is an explicit array constructor
+ *
+ * lhs = expr, rhs = subquery
+ *
+ * FAILURE: SELECT * is not supported in subquery
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select count(*)
+from range(1, 3) r1
+where
+ (r1, r1+1) IN (select * from range(0, r1) r2, range(0, r1+1) r3);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.1.query.sqlpp
new file mode 100644
index 0000000..c1feeb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test no subquery coercion if the subquery is in the FROM clause
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r2.r1
+from (select r1 from range(1, 3) r1) r2
+order by r2.r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.2.query.sqlpp
new file mode 100644
index 0000000..bdefbdd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.2.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test no subquery coercion if the subquery is in the JOIN clause
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r2.nr1, r4.nr3
+from (select r1, -r1 as nr1 from range(1, 4) r1) r2
+join (select r3, -r3 as nr3 from range(2, 5) r3) r4 on r2.r1 = r4.r3
+order by r2.nr1 desc;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.3.query.sqlpp
new file mode 100644
index 0000000..36b88d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test no subquery coercion if the subquery is in the UNNEST clause
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select v1.x, v2.y
+from (select x, [x, x+1, x+2] as y from range(1, 3) x) v1
+unnest (select y from v1.y) v2
+order by v1.x, v2.y;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.1.query.sqlpp
new file mode 100644
index 0000000..689d5d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.1.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test no subquery coercion if the subquery is in the IN clause
+ * of a quantified expression (SOME)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r2.r1
+from (select r1 from range(1, 3) r1) r2
+where some r3 in (select r4 from range(0, 2) r4) satisfies r3.r4 = r2.r1
+order by r2.r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.2.query.sqlpp
new file mode 100644
index 0000000..c96057e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.2.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test no subquery coercion if the subquery is in the IN clause
+ * of a quantified expression (EVERY)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r2.r1
+from (select r1 from range(1, 3) r1) r2
+where every r3 in (select r4 from range(r2.r1, r2.r1) r4) satisfies r3.r4 = r2.r1
+order by r2.r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.3.query.sqlpp
new file mode 100644
index 0000000..01b1b40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test no subquery coercion if the subquery is in the EXISTS clause
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r2.r1, (select value { "a": r4, "b": r4 + 1 } from range(2, r2.r1) r4) x
+from (select r1 from range(1, 3) r1) r2
+where exists (select r4 a, r4 + 1 b from range(2, r2.r1) r4) -- 0, 1, or 2 tuples
+order by r2.r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.4.query.sqlpp
new file mode 100644
index 0000000..6709b34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.4.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * SQL-compat mode.
+ * Test no subquery coercion if the subquery is in the NOT EXISTS clause
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+select r2.r1, (select value { "a": r4, "b": r4 + 1 } from range(2, r2.r1) r4) x
+from (select r1 from range(1, 3) r1) r2
+where not exists (select r4 a, r4 + 1 b from range(2, r2.r1) r4) -- 0, 1, or 2 tuples
+order by r2.r1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.1.ddl.sqlpp
new file mode 100644
index 0000000..83754e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
+
+create dataset ds2(id int) open type primary key id;
+
+create dataset ds3(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.2.update.sqlpp
new file mode 100644
index 0000000..d23a7f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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 ds1 ({"id": 1, "a": 1, "b": 1 });
+insert into ds1 ({"id": 2, "a": 1, "b": 2 });
+insert into ds1 ({"id": 3, "a": 2, "b": 3 });
+insert into ds1 ({"id": 4, "a": 2, "b": 4 });
+insert into ds1 ({"id": 5, "a": 3, "b": 5 });
+insert into ds1 ({"id": 6, "a": 3, "b": 6 });
+insert into ds1 ({"id": 7, "a": 4, "b": 7 });
+
+insert into ds2 ({"id": 100, "c": 2, "d": 100 });
+insert into ds2 ({"id": 101, "c": 2, "d": 101 });
+insert into ds2 ({"id": 102, "c": 3, "d": 102 });
+
+insert into ds3 ({"id": 200, "e": 3, "f": 200 });
+insert into ds3 ({"id": 201, "e": 3, "f": 201 });
+insert into ds3 ({"id": 202, "e": 4, "f": 202 });
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.3.query.sqlpp
new file mode 100644
index 0000000..748dce4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.3.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL -> SELECT list rewriting
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select id, a, b from ds1
+union all
+select id, c, d from ds2
+union all
+select id, e, f from ds3
+order by id desc;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.4.query.sqlpp
new file mode 100644
index 0000000..bc557bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.4.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL -> SELECT list rewriting
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select a, b from ds1
+union all
+select c, d from ds2
+union all
+select e, f from ds3
+order by a desc, b desc;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.5.query.sqlpp
new file mode 100644
index 0000000..7682176
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.5.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * SELECT VALUE inside UNION ALL -> no rewriting
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select value { id, a, b } from ds1
+union all
+select value { id, c, d } from ds2
+union all
+select value { id, e, f } from ds3
+order by id desc;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.6.query.sqlpp
new file mode 100644
index 0000000..4812fa3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.6.query.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.
+ */
+
+/*
+ * UNION ALL in subquery -> SELECT list rewriting and subquery coercion
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select x, y
+from range(2, 4) x, [3, 100, 200] y
+where (x, y) in (
+ select a, b from ds1
+ union all
+ select c, d from ds2
+ union all
+ select e, f from ds3
+)
+order by x, y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.7.query.sqlpp
new file mode 100644
index 0000000..9524ba8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.7.query.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.
+ */
+
+/*
+ * UNION ALL in subquery -> SELECT list rewriting and subquery coercion
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select x
+from [0, 1, 101, 201, 301] x
+where x in (
+ select b from ds1
+ union all
+ select d from ds2
+ union all
+ select f from ds3
+)
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.8.query.sqlpp
new file mode 100644
index 0000000..8fd5852
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_01/union_all_01.8.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL in subquery -> SELECT list rewriting and subquery coercion
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select x,
+ (
+ select a from ds1
+ where id = 101
+ union all
+ select c from ds2
+ where id = 101
+ union all
+ select e from ds3
+ where id = 101
+ ) y
+from range(0, 1) x
+order by x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.1.ddl.sqlpp
new file mode 100644
index 0000000..83754e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
+
+create dataset ds2(id int) open type primary key id;
+
+create dataset ds3(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.2.query.sqlpp
new file mode 100644
index 0000000..a78aea6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.2.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL failure. Both SELECT and SELECT VALUE are present
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select id, a, b from ds1
+union all
+select value { id, c, d } from ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.3.query.sqlpp
new file mode 100644
index 0000000..3ba52d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL failure. Different number of projected fields.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select id, a, b from ds1
+union all
+select id, c from ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.4.query.sqlpp
new file mode 100644
index 0000000..f20bd55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.4.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL failure. Unsupported projection kind (SELECT *)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select * from ds1
+union all
+select * from ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.5.query.sqlpp
new file mode 100644
index 0000000..15a44d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.5.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL failure. Unsupported projection kind (SELECT v.*)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+
+select x.* from ds1 x
+union all
+select y.* from ds2 y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.6.query.sqlpp
new file mode 100644
index 0000000..0941248
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/union_all_02_negative/union_all_02_negative.6.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * UNION ALL failure. Unsupported projection kind (SELECT *, v.* mixed)
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+use test;
+
+select id from ds1
+union all
+select * from ds2
+union all
+select v3.* from ds3 v3;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-literal1/string-literal1.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-literal1/string-literal1.1.query.sqlpp
new file mode 100644
index 0000000..6c0c952
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/string-literal1/string-literal1.1.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+with strings as [
+ 'xs0',
+ 'xs1''ys1',
+ 'xs2''ys2''zs2',
+ E'xs3',
+
+ "xd0",
+ "xd1""yd1",
+ "xd2""yd2""zd2",
+ E"xd3"
+]
+
+select i, s
+from range(0, len(strings)-1) i
+let s = strings[i]
+order by i;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.query.sqlpp
new file mode 100644
index 0000000..22105a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 : Test the issue described in ASTERIXDB-2949
+ * Success : Yes
+ */
+
+
+SELECT SUBSTR("•\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789\n•\tabcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789", 0, 1000) AS s;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.1.ddl.sqlpp
new file mode 100644
index 0000000..bb5c8f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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: This test case is to verify the fix for ASTERIXDB-3006
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset ds1(id int not unknown) open type primary key id;
+create dataset ds2(id int not unknown) open type primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.2.update.sqlpp
new file mode 100644
index 0000000..fd8e76b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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 ds1 ([
+ { "id": 1, "a": [ { "b": 1 }, { "b": 11 }, { "b": 111 } ] },
+ { "id": 2, "a": [ { "b": 2 }, { "b": 22 }, { "b": 222 } ] },
+ { "id": 3, "a": [ { "b": 3 }, { "b": 33 }, { "b": 333 } ] }
+]);
+
+insert into ds2 ([
+ { "id": 101, "x": [ { "y": 33 }, { "y": 11 } ] },
+ { "id": 102, "x": [ { "y": 22 }, { "y": 11 } ] }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.3.query.sqlpp
new file mode 100644
index 0000000..ad662ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test inner join inside a subquery
+ */
+use test;
+
+select value ds1
+from ds1
+where exists(
+ select 1
+ from ds1.a[*].b b
+ join ds2 on ds2.x[0].y = b
+)
+order by ds1.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/TemporalQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/TemporalQueries.xml
index 4ec2303..2547e7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/TemporalQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/TemporalQueries.xml
@@ -33,6 +33,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="temporal">
+ <compilation-unit name="current_date_time_01">
+ <output-dir compare="Text">current_date_time_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
<compilation-unit name="overlap_bins_gby_1">
<output-dir compare="Text">overlap_bins_gby_1</output-dir>
</compilation-unit>
@@ -58,21 +63,41 @@
</compilation-unit>
</test-case>
<test-case FilePath="temporal">
- <compilation-unit name="parse_02">
- <output-dir compare="Text">parse_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="temporal">
<compilation-unit name="parse_01">
<output-dir compare="Text">parse_01</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="temporal">
+ <compilation-unit name="parse_02">
+ <output-dir compare="Text">parse_02</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="parse_03">
+ <output-dir compare="Text">parse_03</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="print_01">
+ <output-dir compare="Text">print_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
<compilation-unit name="day_of_week_01">
<output-dir compare="Text">day_of_week_01</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="temporal">
+ <compilation-unit name="day_of_year_01">
+ <output-dir compare="Text">day_of_year_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="quarter_of_year_01">
+ <output-dir compare="Text">quarter_of_year_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="temporal">
<compilation-unit name="interval_bin">
<output-dir compare="Text">interval_bin</output-dir>
</compilation-unit>
@@ -154,6 +179,11 @@
<output-dir compare="Text">duration_comps</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="week_of_year_01">
+ <output-dir compare="Text">week_of_year_01</output-dir>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="temporal/interval_joins">
<compilation-unit name="interval_after">
<output-dir compare="Text">interval_after</output-dir>
@@ -219,6 +249,11 @@
<output-dir compare="Text">interval_starts</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="temporal/interval_joins">
+ <compilation-unit name="interval_join_projection_check">
+ <output-dir compare="Text">interval_join_projection_check</output-dir>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="temporal/interval_joins_spilling">
<compilation-unit name="interval_after">
<output-dir compare="Text">interval_after</output-dir>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/current_date_time_01/current_date_time_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/current_date_time_01/current_date_time_01.1.query.sqlpp
new file mode 100644
index 0000000..61751c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/current_date_time_01/current_date_time_01.1.query.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+select
+ d = d_from_dt t1,
+ t = t_from_dt t2,
+ day_from_dt = day_from_d t3,
+ milli_from_dt = milli_from_t t4,
+ count(*) cnt
+from range(1, 1000) r
+let
+ dt = current_datetime(),
+ d = current_date(),
+ d_from_dt = date(dt),
+ t = current_time(),
+ t_from_dt = time(dt),
+ day_from_dt = get_day(dt),
+ day_from_d = get_day(d),
+ milli_from_dt = get_millisecond(dt),
+ milli_from_t = get_millisecond(t)
+group by dt, d, d_from_dt, t, t_from_dt, day_from_dt, day_from_d, milli_from_dt, milli_from_t
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/current_date_time_01/current_date_time_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/current_date_time_01/current_date_time_01.2.query.sqlpp
new file mode 100644
index 0000000..c0073a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/current_date_time_01/current_date_time_01.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+select
+ get_year(current_datetime_immediate()) > 0 t1,
+ get_year(current_date_immediate()) > 0 t2,
+ get_hour(current_datetime_immediate()) >= 0 t3,
+ get_hour(current_time_immediate()) >= 0 t4
+from range(1, 2) r
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/datetime_functions/datetime_functions.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/datetime_functions/datetime_functions.4.query.sqlpp
new file mode 100644
index 0000000..ae10e6f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/datetime_functions/datetime_functions.4.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+select
+ tz,
+ string(s1) s1,
+ string(s2) s2,
+ string(m1) m1,
+ string(m2) m2,
+ unix_time_from_datetime_in_secs(s1) = ets us1,
+ unix_time_from_datetime_in_secs(s2, tz) = ets us2,
+ unix_time_from_datetime_in_ms(m1) = etm um1,
+ unix_time_from_datetime_in_ms(m2, tz) = etm um2
+from range(0, 2) r
+let ets = 1631671261, /* UTC:2021-09-15T02:01:01.000 */
+ etm = ets * 1000,
+ tzs = [ "UTC", "America/Los_Angeles", "Asia/Tokyo" ],
+ tz = tzs[r],
+ s1 = datetime_from_unix_time_in_secs(ets),
+ s2 = datetime_from_unix_time_in_secs(ets, tz),
+ m1 = datetime_from_unix_time_in_ms(etm),
+ m2 = datetime_from_unix_time_in_ms(etm, tz)
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_week_01/day_of_week_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_week_01/day_of_week_01.3.query.sqlpp
index 89c7ea7..3e97dc8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_week_01/day_of_week_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_week_01/day_of_week_01.3.query.sqlpp
@@ -24,4 +24,4 @@
use test;
-{'1970-01-01':test.`day-of-week`(test.date('1970-01-01')),'2013-08-06':test.`day-of-week`(test.date('2013-08-06')),'-2013-08-06':test.`day-of-week`(test.date('-2013-08-06')),'1913-08-06T15:53:28Z':test.`day-of-week`(test.datetime('1913-08-06T15:53:28Z')),'-1913-08-10T15:53:28Z':test.`day-of-week`(test.datetime('-1913-08-10T15:53:28Z')),'null':test.`day-of-week`(null)};
+{'1970-01-01':test.`day-of-week`(test.date('1970-01-01'), "Mon"),'2013-08-06':test.`day-of-week`(test.date('2013-08-06'), "MON"),'-2013-08-06':test.`day-of-week`(test.date('-2013-08-06'), "Monday"),'1913-08-06T15:53:28Z':test.`day-of-week`(test.datetime('1913-08-06T15:53:28Z'), "MONDAY"),'-1913-08-10T15:53:28Z':test.`day-of-week`(test.datetime('-1913-08-10T15:53:28Z'), 2),'null':test.`day-of-week`(null)};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_week_01/day_of_week_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_week_01/day_of_week_01.4.query.sqlpp
new file mode 100644
index 0000000..05aa88b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_week_01/day_of_week_01.4.query.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+/**
+ * day-of-week test case: test the day-of-week function
+ * Expected result: success
+ **/
+
+select r, array_min(dows) mn, array_max(dows) mx
+from range(1, 7) r
+let
+ d = date("2021-09-05"),
+ dt = datetime_from_date_time(d, time("01:01:01")),
+ days = [ "sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday" ],
+ dows =
+ [
+ day_of_week(d, int8(r)),
+ day_of_week(dt, int16(r)),
+ day_of_week(d, int32(r)),
+ day_of_week(dt, int64(r)),
+ day_of_week(d, float(r)),
+ day_of_week(dt, double(r)),
+ day_of_week(d, days[r-1]),
+ day_of_week(dt, upper(days[r-1])),
+ day_of_week(d, lower(days[r-1])),
+ day_of_week(dt, substr(days[r-1], 0, 3)),
+ day_of_week(d, upper(substr(days[r-1], 0, 3))),
+ day_of_week(dt, lower(substr(days[r-1], 0, 3)))
+ ]
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_year_01/day_of_year_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_year_01/day_of_year_01.1.query.sqlpp
new file mode 100644
index 0000000..e065c98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_year_01/day_of_year_01.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+select r, array_min(doys) mn, array_max(doys) mx
+from range(1, 7) r
+let
+ d = date("2021-01-0" || string(r)),
+ dt = datetime_from_date_time(d, time("01:01:01")),
+ doys =
+ [
+ day_of_year(d),
+ day_of_year(dt)
+ ]
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_year_01/day_of_year_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_year_01/day_of_year_01.2.query.sqlpp
new file mode 100644
index 0000000..76715a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/day_of_year_01/day_of_year_01.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+select sum(doy1) s1, sum(doy2) s2
+from range(0, 364) r
+let
+ d = date_from_unix_time_in_days(r),
+ dt = datetime_from_date_time(d, time("01:01:01")),
+ doy1 = day_of_year(d),
+ doy2 = day_of_year(dt);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.1.ddl.sqlpp
new file mode 100644
index 0000000..c403987
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE IntervalType AS CLOSED {
+ id: uuid,
+ name: string,
+ i:interval
+};
+
+CREATE DATASET IntervalDataset(IntervalType) PRIMARY KEY id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.2.update.sqlpp
new file mode 100644
index 0000000..bcc3f98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.2.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 IntervalDataset {
+"name":"i1","i":interval(date("2020-11-06"), date("2020-11-22"))
+};
+INSERT INTO IntervalDataset {
+"name":"i1","i":interval(date("2020-11-06"), date("2020-11-22"))
+};
+INSERT INTO IntervalDataset {
+"name":"i2","i":interval(date("2021-11-06"), date("2021-11-22"))
+};
+INSERT INTO IntervalDataset {
+"name":"i2","i":interval(date("2021-11-06"), date("2021-11-22"))
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.3.query.sqlpp
new file mode 100644
index 0000000..6bf7c95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+SELECT i1.name
+FROM IntervalDataset i1, IntervalDataset i2
+WHERE /*+ range [date("2020-11-06"), date("2021-11-22")] */ interval_overlapping(i1.i, i2.i)
+ORDER BY i1.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_01/parse_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_01/parse_01.3.query.sqlpp
index 5a16610..0ee2fd1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_01/parse_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_01/parse_01.3.query.sqlpp
@@ -24,4 +24,4 @@
use test;
-select element {'date1':test.`parse-date`('2013-8-23','YY-M-D'),'date2':test.`parse-date`('Aug 12 -12','MMM D Y'),'date3':test.`parse-date`('-1234-01-01','YYYY-MM-DD'),'date4':test.`parse-date`('09/11/-1980','D/M/Y'),'date5':test.`parse-date`('09/11/-1990','YY-M-D|MMM D Y|D/M/Y'),'date6':test.`parse-date`('Mon Aug 19 2013','W MMM D Y'),'data7':test.`parse-date`('SKIPMEPLEASE Mon Aug SKIPME1ALSO 19 2013','O W MMM O D Y'),'time1':test.`parse-time`('8:23:49','h:m:s'),'time2':test.`parse-time`('8.19.23:32','h.m.s:nn'),'time3':test.`parse-time`('08.19.23:32 pm','h.m.s:nn a'),'time4':test.`parse-time`('6:30:40.948 pm PST','h:mm:ss.nnn a z'),'time5':test.`parse-time`('6:30:40.948 pm PST','h:m:s|h.m.s:nn|h.m.s:nn a|h:mm:ss.nnn a z'),'datetime1':test.`parse-datetime`('Dec 30 -1203 3:48:27 PM','MMM DD YYYY h:m:s a'),'datetime2':test.`parse-datetime`('12/30/-1203 03:48:27.392 PM Asia/Shanghai','MM/DD/YYY hh:mm:ss.nnn a z'),'datetime3':test.`parse-datetime`('1723-12-03T23:59:23.392Z','YYYY-MM-DDThh:mm:ss.nnnz'),'datetime4':test.`parse-datetime`('1723-12-03T23:59:23.392-04:00','YYYY-MM-DDThh:mm:ss.nnnz'),'datetime5':test.`parse-datetime`('1723-12-03T23:59:23.392-04:00','MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DDThh:mm:ss.nnnz'),'datetime6':test.`parse-datetime`('1970-01-01 Thu 23:59:23.392-04:00','MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD W hh:mm:ss.nnnz'),'datetime7':test.`parse-datetime`('1723-12-03 What3v3r STRINGHERE 23:59:23.392-04:00','MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD O O hh:mm:ss.nnnz')};
+select element {'date1':test.`parse-date`('2013-8-23','YY-M-D'),'date2':test.`parse-date`('Aug 12 -12','MMM D Y'),'date3':test.`parse-date`('-1234-01-01','YYYY-MM-DD'),'date4':test.`parse-date`('09/11/-1980','D/M/Y'),'date5':test.`parse-date`('09/11/-1990','YY-M-D|MMM D Y|D/M/Y'),'date6':test.`parse-date`('Mon Aug 19 2013','EEE MMM D Y'),'data7':test.`parse-date`('SKIPMEPLEASE Mon Aug SKIPME1ALSO 19 2013','O EEE MMM O D Y'),'time1':test.`parse-time`('8:23:49','h:m:s'),'time2':test.`parse-time`('8.19.23:32','h.m.s:nn'),'time3':test.`parse-time`('08.19.23:32 pm','h.m.s:nn a'),'time4':test.`parse-time`('6:30:40.948 pm PST','h:mm:ss.nnn a z'),'time5':test.`parse-time`('6:30:40.948 pm PST','h:m:s|h.m.s:nn|h.m.s:nn a|h:mm:ss.nnn a z'),'datetime1':test.`parse-datetime`('Dec 30 -1203 3:48:27 PM','MMM DD YYYY h:m:s a'),'datetime2':test.`parse-datetime`('12/30/-1203 03:48:27.392 PM Asia/Shanghai','MM/DD/YYY hh:mm:ss.nnn a z'),'datetime3':test.`parse-datetime`('1723-12-03T23:59:23.392Z','YYYY-MM-DDThh:mm:ss.nnnz'),'datetime4':test.`parse-datetime`('1723-12-03T23:59:23.392-04:00','YYYY-MM-DDThh:mm:ss.nnnz'),'datetime5':test.`parse-datetime`('1723-12-03T23:59:23.392-04:00','MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DDThh:mm:ss.nnnz'),'datetime6':test.`parse-datetime`('1970-01-01 Thu 23:59:23.392-04:00','MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD EEE hh:mm:ss.nnnz'),'datetime7':test.`parse-datetime`('1723-12-03 What3v3r STRINGHERE 23:59:23.392-04:00','MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD O O hh:mm:ss.nnnz')};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_02/parse_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_02/parse_02.3.query.sqlpp
index e76f1b2..5426205 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_02/parse_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_02/parse_02.3.query.sqlpp
@@ -24,4 +24,4 @@
use test;
-{'date-string-1':test.`print-date`(test.date('-0123-01-30'),'YY/M/D'),'date-string-2':test.`print-date`(test.date('-0123-01-30'),'MMM DD, YYYY'),'date-string-3':test.`print-date`(test.date('-0123-01-30'),'YYYY/MM/DD'),'time-string-1':test.`print-time`(test.time('08:07:29.030Z'),'h.m.s.nn a z'),'time-string-2':test.`print-time`(test.time('08:07:29.030Z'),'hh.mm.ss.nnn a z'),'datetime-string-1':test.`print-datetime`(test.datetime('0137-12-31T23:59:59.999+08:00'),'MMM DD h:m:s.nnn a YY z'),'datetime-string-2':test.`print-datetime`(test.datetime('0137-12-31T23:59:59.999+08:00'),'YYYY/MMM/DD h:m:s.nnnz a'),'datetime-string-3':test.`print-datetime`(test.datetime('0137-12-31T23:59:59.999+08:00'),'YYYY-MM-DDThh:mm:ss.nnnz'),'datetime-string-4':test.`print-datetime`(test.datetime('1000-10-10T10:10:10.100+00:00'),'YYYY-MM-DDThh:mm:ss.nnnz')};
+{'date-string-1':test.`print-date`(test.date('-0123-01-30'),'YY/M/D'),'date-string-2':test.`print-date`(test.date('-0123-01-30'),'MMM DD, YYYY'),'date-string-3':test.`print-date`(test.date('-0123-01-30'),'YYYY/MM/DD'),'time-string-1':test.`print-time`(test.time('08:07:29.030Z'),'h.m.s.nn a'),'time-string-2':test.`print-time`(test.time('08:07:29.030Z'),'hh.mm.ss.nnn a'),'datetime-string-1':test.`print-datetime`(test.datetime('0137-12-31T23:59:59.999+08:00'),'MMM DD h:m:s.nnn a YY'),'datetime-string-2':test.`print-datetime`(test.datetime('0137-12-31T23:59:59.999+08:00'),'YYYY/MMM/DD h:m:s.nnn a'),'datetime-string-3':test.`print-datetime`(test.datetime('0137-12-31T23:59:59.999+08:00'),'YYYY-MM-DDThh:mm:ss.nnn'),'datetime-string-4':test.`print-datetime`(test.datetime('1000-10-10T10:10:10.100+00:00'),'YYYY-MM-DDThh:mm:ss.nnn')};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.1.query.sqlpp
new file mode 100644
index 0000000..125dfad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * Parse milliseconds using alternative format character (S)
+ */
+
+select
+ dt, string(parse_datetime(dt, "YYYY-MM-DDThh:mm:ss.SSS")) dt2,
+ t, string(parse_time(t, "hh:mm:ss.SSS")) t2
+from [0, 1, 9, 10, 99, 100, 999, 1999, 10999] r
+let
+ dt = string(datetime_from_unix_time_in_ms(r)),
+ t = string(time_from_unix_time_in_ms(r))
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.2.query.sqlpp
new file mode 100644
index 0000000..d982426
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.2.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * Parse (and ignore) weekday
+ */
+
+with weekdays as ["Sunday", "Monday", "Tuesday", "Wednesday", "Thursday", "Friday", "Saturday"]
+
+select s3, d3, s4, d4
+from range(0, 6) r
+let s = "2021-07-2" || string(r),
+ s3 = s || " " || substr(weekdays[r], 0, 3),
+ s4 = s || " " || weekdays[r],
+ d3 = parse_date(s3,"YYYY-MM-DD EEE"),
+ d4 = parse_date(s4,"YYYY-MM-DD EEEE")
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.3.query.sqlpp
new file mode 100644
index 0000000..e3c6a47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.3.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * Parse (and ignore) weekday
+ */
+
+with weekdays as ["Sunday", "Monday", "Tuesday", "Wednesday", "Thursday", "Friday", "Saturday"]
+
+select s3, d3, s4, d4
+from range(0, 6) r
+let s = "2021-07-2" || string(r),
+ s3 = substr(weekdays[r], 0, 3) || " " || s,
+ s4 = weekdays[r] || " " || s,
+ d3 = parse_date(s3,"EEE YYYY-MM-DD"),
+ d4 = parse_date(s4,"EEEE YYYY-MM-DD")
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.4.query.sqlpp
new file mode 100644
index 0000000..35db859
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.4.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+/*
+ * Parse month name
+ */
+
+ with months as [
+ "January", "February", "March", "April", "May", "June",
+ "July", "August", "September", "October", "November", "December"
+]
+select
+ s3m, parse_date(s3m, "YYYY-MMM") d3m,
+ s3d, parse_date(s3d, "YYYY-MMM-DD") d3d,
+ s4m, parse_date(s4m, "YYYY-MMMM") d4m,
+ s4d, parse_date(s4d, "YYYY-MMMM-DD") d4d
+from range(0, 11) r
+let
+ s3m = "2020-" || substr(months[r], 0, 3),
+ s3d = "2020-" || substr(months[r], 0, 3) || "-02",
+ s4m = "2020-" || months[r],
+ s4d = "2020-" || months[r] || "-02"
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.5.query.sqlpp
new file mode 100644
index 0000000..3dc2b10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/*
+ * Parse standalone year
+ */
+
+ {
+ "t1": parse_date("2020", "YYYY"),
+ "t2": parse_datetime("2020", "YYYY")
+ }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.6.query.sqlpp
new file mode 100644
index 0000000..522a8dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/parse_03/parse_03.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/*
+ * Parse year and quarter
+ */
+
+select r, d, dt
+from range(1, 4) r
+let d = parse_date("2020-" || string(r) , "YYYY-Q"),
+ dt = parse_datetime("2021-0" || string(r), "YYYY-QQ")
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.1.query.sqlpp
new file mode 100644
index 0000000..1579c95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.1.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/* Print quarter */
+
+select p1, p2, count(*) cnt
+from range(0, 365) r
+let d = date_from_unix_time_in_days(unix_time_from_date_in_days(date("2020-01-01")) + r),
+ p1 = print_date(d, "YYYY-Q"),
+ p2 = print_date(d, "YYYY-QQ")
+group by p1, p2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.2.query.sqlpp
new file mode 100644
index 0000000..81b04d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/* Print month and its full name */
+
+select p, count(*) cnt
+from range(0, 365) r
+let d = date_from_unix_time_in_days(unix_time_from_date_in_days(date("2020-01-01")) + r),
+ p = print_date(d, "YYYY-MM-MMMM")
+group by p;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.3.query.sqlpp
new file mode 100644
index 0000000..26e5743
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+/* Print day of year */
+
+select min(ddd) mn, max(ddd) mx, sum(ddd) sm
+from range(0, 365) r
+let d = date_from_unix_time_in_days(unix_time_from_date_in_days(date("2020-01-01")) + r),
+ p = print_date(d, "DDD"),
+ ddd = int(p);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.4.query.sqlpp
new file mode 100644
index 0000000..6893a80
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Print day of week
+ */
+
+select r,
+ print_date(d, "EEE") d3, print_date(d, "EEEE") d4,
+ print_datetime(dt, "EEE") dt3, print_datetime(dt, "EEEE") dt4
+from range(0, 6) r
+let
+ d = date_from_unix_time_in_days(unix_time_from_date_in_days(date("2021-09-19")) + r),
+ dt = datetime_from_date_time(d, time("01:01:01"))
+order by r;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.5.query.sqlpp
new file mode 100644
index 0000000..8aeb8bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/print_01/print_01.5.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * Print milliseconds using alternative format character (S)
+ */
+
+select r ms,
+ print_datetime(dt, "s.S") dt1,
+ print_datetime(dt, "s.SS") dt2,
+ print_datetime(dt, "s.SSS") dt3,
+ print_time(t, "s.S" ) t1,
+ print_time(t, "s.SS" ) t2,
+ print_time(t, "s.SSS" ) t3
+from [0, 1, 9, 10, 99, 100, 999] r
+let
+ dt = datetime_from_unix_time_in_ms(r),
+ t = time_from_unix_time_in_ms(r)
+order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/quarter_of_year_01/quarter_of_year_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/quarter_of_year_01/quarter_of_year_01.1.query.sqlpp
new file mode 100644
index 0000000..a11b3c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/quarter_of_year_01/quarter_of_year_01.1.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * Quarter of year
+ */
+
+select month, qoy1, qoy2, count(*) cnt
+from range(0, 365) r
+let
+ s = unix_time_from_date_in_days(date("2020-01-01")),
+ d = date_from_unix_time_in_days(s + r),
+ dt = datetime_from_date_time(d, time("01:01:01")),
+ month = get_month(d),
+ qoy1 = quarter_of_year(d),
+ qoy2 = quarter_of_year(dt)
+group by qoy1, qoy2, month
+order by qoy1, qoy2, month;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.1.query.sqlpp
new file mode 100644
index 0000000..86276447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.1.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * Start of the week: Sunday (default)
+ */
+
+select month, woy1, woy2, count(*) cnt
+from range(0, 364) r
+let
+ s = unix_time_from_date_in_days(date("2021-01-01")),
+ d = date_from_unix_time_in_days(s + r),
+ dt = datetime_from_date_time(d, time("01:01:01")),
+ month = get_month(d),
+ woy1 = week_of_year(d),
+ woy2 = week_of_year(dt)
+group by woy1, woy2, month
+order by woy1, woy2, month;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.2.query.sqlpp
new file mode 100644
index 0000000..b76dba6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.2.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+/*
+ * Start of the week: Monday
+ */
+
+select month, woy1, woy2, count(*) cnt
+from range(0, 364) r
+let
+ s = unix_time_from_date_in_days(date("2021-01-01")),
+ d = date_from_unix_time_in_days(s + r),
+ dt = datetime_from_date_time(d, time("01:01:01")),
+ month = get_month(d),
+ woy1 = week_of_year(d, "Monday"),
+ woy2 = week_of_year(dt, "Monday")
+group by woy1, woy2, month
+order by woy1, woy2, month;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.3.query.sqlpp
new file mode 100644
index 0000000..15b02cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/week_of_year_01/week_of_year_01.3.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+select r, array_min(dows) mn, array_max(dows) mx
+from range(1, 7) r
+let
+ d = date("2021-01-10"),
+ dt = datetime_from_date_time(d, time("01:01:01")),
+ days = [ "sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday" ],
+ dows =
+ [
+ week_of_year(d, int8(r)),
+ week_of_year(dt, int16(r)),
+ week_of_year(d, int32(r)),
+ week_of_year(dt, int64(r)),
+ week_of_year(d, float(r)),
+ week_of_year(dt, double(r)),
+ week_of_year(d, days[r-1]),
+ week_of_year(dt, upper(days[r-1])),
+ week_of_year(d, lower(days[r-1])),
+ week_of_year(dt, substr(days[r-1], 0, 3)),
+ week_of_year(d, upper(substr(days[r-1], 0, 3))),
+ week_of_year(dt, lower(substr(days[r-1], 0, 3)))
+ ]
+order by r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.2.query.sqlpp
new file mode 100644
index 0000000..b4c4e4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.2.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+FROM [
+ { "f": "a" },
+ { "f": "b" },
+ { "f": "c" }
+] AS D
+SELECT GET_TYPE(D.f) AS dType;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.3.ddl.sqlpp
new file mode 100644
index 0000000..5606270
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.3.ddl.sqlpp
@@ -0,0 +1,80 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.AllType as
+{
+ t1: [string],
+ t2: binary,
+ t3: boolean,
+ t4: circle,
+ t5: date,
+ t6: datetime,
+ t7: duration,
+ t8: interval,
+ t9: line,
+ t10: {{string}},
+ t11: point,
+ t12: polygon,
+ t13: rectangle,
+ t14: string,
+ t15: time,
+ t16: uuid
+};
+
+create dataset `All` (AllType) primary key t16 autogenerated;
+
+insert into `All` [{
+ "t1": ["a", "b"],
+ "t2": hex("ABCDEF0123456789"),
+ "t3": false,
+ "t4": circle("30.0,70.0 5.0"),
+ "t5": date("-19700101"),
+ "t6": datetime("2013-01-01T12:12:12.039Z"),
+ "t7": duration("P100Y12MT12M"),
+ "t8": interval(date("2013-01-01"), date("20130505")),
+ "t9": line("30.0,70.0 50.0,90.0"),
+ "t10": {{"a", "b"}},
+ "t11": point("1,2"),
+ "t12": polygon("1.0,1.0 2.0,2.0 3.0,3.0 4.0,4.0"),
+ "t13": rectangle("30.0,70.0 50.0,90.0"),
+ "t14": "abcde",
+ "t15": time("12:12:12.039Z")
+},
+{
+ "t1": ["a", "b"],
+ "t2": hex("ABCDEF0123456789"),
+ "t3": false,
+ "t4": circle("30.0,70.0 5.0"),
+ "t5": date("-19700101"),
+ "t6": datetime("2013-01-01T12:12:12.039Z"),
+ "t7": duration("P100Y12MT12M"),
+ "t8": interval(date("2013-01-01"), date("20130505")),
+ "t9": line("30.0,70.0 50.0,90.0"),
+ "t10": {{"a", "b"}},
+ "t11": point("1,2"),
+ "t12": polygon("1.0,1.0 2.0,2.0 3.0,3.0 4.0,4.0"),
+ "t13": rectangle("30.0,70.0 50.0,90.0"),
+ "t14": "abcde",
+ "t15": time("12:12:12.039Z")
+}];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.4.query.sqlpp
new file mode 100644
index 0000000..dbec25f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/gettype/gettype.4.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+SELECT VALUE {
+ "t1": get_type(t1),
+ "t2": get_type(t2),
+ "t3": get_type(t3),
+ "t4": get_type(t4),
+ "t5": get_type(t5),
+ "t6": get_type(t6),
+ "t7": get_type(t7),
+ "t8": get_type(t8),
+ "t9": get_type(t9),
+ "t10": get_type(t10),
+ "t11": get_type(t11),
+ "t12": get_type(t12),
+ "t13": get_type(t13),
+ "t14": get_type(t14),
+ "t15": get_type(t15),
+ "t16": get_type(t16)
+}
+FROM `All`;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/query-ASTERIXDB-2950/query-ASTERIXDB-2950.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/query-ASTERIXDB-2950/query-ASTERIXDB-2950.1.query.sqlpp
new file mode 100644
index 0000000..0e4bdb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/query-ASTERIXDB-2950/query-ASTERIXDB-2950.1.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/*
+ * Type-mismatch
+ */
+
+(select value 49)[0][0];
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/query-ASTERIXDB-2950/query-ASTERIXDB-2950.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/query-ASTERIXDB-2950/query-ASTERIXDB-2950.2.query.sqlpp
new file mode 100644
index 0000000..d6748ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/query-ASTERIXDB-2950/query-ASTERIXDB-2950.2.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/*
+ * Type-mismatch
+ */
+
+from Metadata.`Dataverse` as dv
+select count(dv.DataverseName)[0];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_bigint_02/to_bigint_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_bigint_02/to_bigint_02.1.query.sqlpp
index 13f482f..f83710f9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_bigint_02/to_bigint_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_bigint_02/to_bigint_02.1.query.sqlpp
@@ -16,6 +16,10 @@
* specific language governing permissions and limitations
* under the License.
*/
+
+// requesttype=application/json
+// param max-warnings:json=100
+
{
"t": to_bigint(date("2017-06-30"))
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_boolean_02/to_boolean_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_boolean_02/to_boolean_02.1.query.sqlpp
index 5be147e..78bc0d2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_boolean_02/to_boolean_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_boolean_02/to_boolean_02.1.query.sqlpp
@@ -16,6 +16,10 @@
* specific language governing permissions and limitations
* under the License.
*/
+
+// requesttype=application/json
+// param max-warnings:json=100
+
{
"t": to_boolean(date("2017-06-30"))
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_double_02/to_double_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_double_02/to_double_02.1.query.sqlpp
index 4a31e1c..78baae8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_double_02/to_double_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_double_02/to_double_02.1.query.sqlpp
@@ -16,6 +16,10 @@
* specific language governing permissions and limitations
* under the License.
*/
+
+// requesttype=application/json
+// param max-warnings:json=100
+
{
"t": to_double(date("2017-06-30"))
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_number_02/to_number_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_number_02/to_number_02.1.query.sqlpp
index 837750e..45190af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_number_02/to_number_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_number_02/to_number_02.1.query.sqlpp
@@ -16,6 +16,10 @@
* specific language governing permissions and limitations
* under the License.
*/
+
+// requesttype=application/json
+// param max-warnings:json=100
+
{
"t": to_number(date("2017-06-30"))
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_string_02/to_string_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_string_02/to_string_02.1.query.sqlpp
index cdfec14..3075bf4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_string_02/to_string_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/to_string_02/to_string_02.1.query.sqlpp
@@ -16,6 +16,10 @@
* specific language governing permissions and limitations
* under the License.
*/
+
+// requesttype=application/json
+// param max-warnings:json=100
+
{
"t": to_string(date("2017-06-30"))
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.1.ddl.sqlpp
new file mode 100644
index 0000000..66965a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.1.ddl.sqlpp
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test2;
+
+--- test2
+
+create synonym DataverseSyn2 for Metadata.`Dataverse`;
+
+create function f2() {
+ select count(*) as f2 from DataverseSyn2
+ where DataverseName like "t%"
+};
+
+create view v2 as select sum(r) as v2 from range(1,2) r;
+
+--- test1
+
+create synonym test1.DataverseSyn1 for Metadata.`Dataverse`;
+
+create view test1.v1 as select sum(r) as v1 from range(1,3) r;
+
+create function test1.f1() {
+ select count(*) as f1 from DataverseSyn1
+ where DataverseName like "t%"
+};
+
+create view test1.vA as
+ select v1.*, v2.*, f1.*, f2.*, s1.*, s2.*, d1.*
+ from
+ v1 v1,
+ test2.v2 v2,
+ f1() f1,
+ test2.f2() f2,
+ (
+ select count(*) as s1 from DataverseSyn1
+ where DataverseName like "te%"
+ ) s1,
+ (
+ select count(*) as s2 from test2.DataverseSyn2
+ where DataverseName like "tes%"
+ ) s2,
+ (
+ select count(*) as d1 from Metadata.`Dataverse`
+ where DataverseName like "test%"
+ ) d1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.10.ddl.sqlpp
new file mode 100644
index 0000000..073842f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.10.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+-- test "or replace"
+
+create or replace view test1.vA as
+ select r from range(3,4) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.11.query.sqlpp
new file mode 100644
index 0000000..443d9eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.11.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 test1;
+
+select value vA from vA order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.12.query.sqlpp
new file mode 100644
index 0000000..80aa2dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.12.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+select DataverseName, DatasetName,
+ DatatypeDataverseName, DatatypeName, ViewDetails
+from Metadata.`Dataset`
+where DataverseName like "test%" and DatasetType = "VIEW" and DatasetName = "vA"
+order by DataverseName, DatasetName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.2.query.sqlpp
new file mode 100644
index 0000000..d7b182f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 test1;
+
+select value vA from vA;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.3.query.sqlpp
new file mode 100644
index 0000000..2cf093f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+select DataverseName, DatasetName,
+ DatatypeDataverseName, DatatypeName, ViewDetails
+from Metadata.`Dataset`
+where DataverseName like "test%" and DatasetType = "VIEW"
+order by DataverseName, DatasetName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.4.ddl.sqlpp
new file mode 100644
index 0000000..237100e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.4.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+-- test "if not exists" when the view already exists
+
+create view test1.vA if not exists as
+ select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.5.query.sqlpp
new file mode 100644
index 0000000..636f5da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.5.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+--- same result as before
+
+use test1;
+
+select value vA from vA;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.6.query.sqlpp
new file mode 100644
index 0000000..a204c97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- same result as before
+
+select DataverseName, DatasetName,
+ DatatypeDataverseName, DatatypeName, ViewDetails
+from Metadata.`Dataset`
+where DataverseName like "test%" and DatasetType = "VIEW" and DatasetName = "vA"
+order by DataverseName, DatasetName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.7.ddl.sqlpp
new file mode 100644
index 0000000..8d6169f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.7.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+-- test "if not exists" when the view does not yet exist
+
+create view test1.vB if not exists as
+ select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.8.query.sqlpp
new file mode 100644
index 0000000..bb1f529
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.8.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 test1;
+
+select value vB from vB order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.9.query.sqlpp
new file mode 100644
index 0000000..84038a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-1/create-view-1.9.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+select DataverseName, DatasetName,
+ DatatypeDataverseName, DatatypeName, ViewDetails
+from Metadata.`Dataset`
+where DataverseName like "test%" and DatasetType = "VIEW" and DatasetName = "vB"
+order by DataverseName, DatasetName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.1.ddl.sqlpp
new file mode 100644
index 0000000..f8224ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.1.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+--- Negative: unknown dataverse
+
+drop dataverse test if exists;
+
+create view test.v1 as select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.10.ddl.sqlpp
new file mode 100644
index 0000000..9a8fa71
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.10.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+--- Negative: recursive definitions (view/function)
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as
+ select * from range(1,2) r;
+
+create function test.f2() {
+ select * from v1 r
+};
+
+create or replace view test.v1 as
+ select * from f2() r;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.11.ddl.sqlpp
new file mode 100644
index 0000000..5aab3ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.11.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+--- Negative: recursive definitions (function/view)
+
+drop dataverse test if exists;
+create dataverse test;
+
+create function test.f1() {
+ select * from range(1,2) r
+};
+
+create view test.v2 as
+ select * from f1() r
+;
+
+create or replace function test.f1() {
+ select * from v2 r
+};
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.12.ddl.sqlpp
new file mode 100644
index 0000000..b5acfc7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.12.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: untyped view cannot declare primary key
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 primary key (r) not enforced as
+ select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.13.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.13.ddl.sqlpp
new file mode 100644
index 0000000..7ad5c5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.13.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+--- Negative: untyped view cannot declare foreign key
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as
+ select r from range(1,2) r;
+
+create view test.v2 foreign key (r) references v1 not enforced as
+ select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.2.ddl.sqlpp
new file mode 100644
index 0000000..0340d2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.2.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+--- Negative: invalid view definition (syntax)
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as select * from range(1,2) r order by;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.3.ddl.sqlpp
new file mode 100644
index 0000000..93cde91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.3.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+--- Negative: invalid view definition (unknown used function)
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as select * from undefined_range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.4.ddl.sqlpp
new file mode 100644
index 0000000..6969d78
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.4.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: create view fails because view with this name already exists
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as select * from range(1,2) r;
+create view test.v1 as select * from range(3,4) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.5.ddl.sqlpp
new file mode 100644
index 0000000..da541d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.5.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+--- Negative: create view fails because dataset with this name already exists
+
+drop dataverse test if exists;
+create dataverse test;
+
+create dataset test.ds1(
+ c_custkey integer not unknown,
+ c_name string not unknown
+) primary key c_custkey;
+
+create view test.ds1 as select * from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.6.ddl.sqlpp
new file mode 100644
index 0000000..f6aefe9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.6.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+--- Negative: create or replace view fails because dataset with this name already exists
+
+drop dataverse test if exists;
+create dataverse test;
+
+create dataset test.ds2(
+ c_custkey integer not unknown,
+ c_name string not unknown
+) primary key c_custkey;
+
+create or replace view test.ds2 as select * from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.7.ddl.sqlpp
new file mode 100644
index 0000000..56e24e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.7.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+--- Negative: create dataset fails because view with this name already exists
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.ds1 as select * from range(1,2) r;
+
+create dataset test.ds1(
+ c_custkey integer not unknown,
+ c_name string not unknown
+) primary key c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.8.ddl.sqlpp
new file mode 100644
index 0000000..97a7709
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.8.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: "or replace" with "if not exists"
+
+drop dataverse test if exists;
+create dataverse test;
+
+create or replace view test.v1 if not exists as
+ select * from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.9.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.9.ddl.sqlpp
new file mode 100644
index 0000000..a9ef305
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-2-negative/create-view-2-negative.9.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+--- Negative: recursive definitions (view/view)
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as
+ select * from range(1,2) r;
+
+create view test.v2 as
+ select * from v1 r;
+
+create or replace view test.v1 as
+ select * from v2 r;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.1.ddl.sqlpp
new file mode 100644
index 0000000..5c4ea0e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.1.ddl.sqlpp
@@ -0,0 +1,125 @@
+/*
+ * 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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset t1(c_id int32 not unknown) open type primary key c_id;
+
+create dataset t2(c_id int32 not unknown) open type primary key c_id;
+
+/* inline type */
+create view v1(
+ c_id int32,
+ c_i8 int8, c_i16 int16, c_i32 int32, c_i64 int64, c_f float, c_d double,
+ c_b boolean, c_s string,
+ c_datetime datetime, c_date date, c_time time,
+ c_dur duration, c_ymdur year_month_duration, c_dtdur day_time_duration
+) default null as t1;
+
+create type t2 as closed {
+ c_id:int32?,
+ c_i8:int8?, c_i16:int16?, c_i32:int32?, c_i64:int64?, c_f:float?, c_d:double?,
+ c_b:boolean?, c_s:string?,
+ c_datetime:datetime?, c_date:date?, c_time:time?,
+ c_dur:duration?, c_ymdur:year_month_duration?, c_dtdur:day_time_duration?
+};
+
+/* type reference, query body */
+create view v2_ref_type(t2) default null as
+ select c_id,
+ c_i8, c_i16, c_i32, c_i64, c_f, c_d,
+ c_b, c_s,
+ c_datetime, c_date, c_time,
+ c_dur, c_ymdur, c_dtdur
+ from t1
+;
+
+/* custom date/time format */
+create view v3_datetime_format(
+ c_id int32,
+ c_datetime datetime, c_date date, c_time time
+) default null
+ datetime 'MM/DD/YYYY hh:mm:ss.nnna'
+ date 'MM/DD/YYYY'
+ time 'hh:mm:ss.nnna'
+as t2;
+
+/* custom date format */
+create view v4_date_format_only(
+ c_id int32,
+ c_datetime datetime, c_date date, c_time time
+) default null
+ date 'MM/DD/YYYY'
+as t2;
+
+/* primary key (not enforced) */
+
+create view v5_pk(
+ c_id int32 not unknown,
+ c_datetime datetime
+) default null
+ datetime 'MM/DD/YYYY hh:mm:ss.nnna'
+ primary key (c_id) not enforced
+as t2;
+
+/* primary key (not enforced), check that invalid tuples are eliminated */
+/* also check that NOT UNKNOWN type quantifier is automatically assumed for primary keys */
+
+create view v6_pk_no_nulls(
+ c_i64 int64,
+ c_id int32
+) default null
+ primary key (c_i64) not enforced
+as t1;
+
+/* no primary key, check that invalid tuples are eliminated if target field type is declared as not unknown */
+
+create view v7_no_nulls(
+ c_i64 int64 not unknown,
+ c_id int32
+) default null
+as t1;
+
+/* no primary key, check that invalid tuples are eliminated if target field type is declared as not unknown */
+
+create view v8_no_nulls_multi(
+ c_id int32,
+ c_x int64 not unknown,
+ c_y int64 not unknown
+) default null
+as
+ select
+ c_id,
+ case when to_bigint(c_i32) >= 0 then to_bigint(c_i32) when to_bigint(c_i32) < 0 then null else 0 end as c_x,
+ case when to_bigint(c_i64) >= 0 then null when to_bigint(c_i64) < 0 then to_bigint(c_i64) else 0 end as c_y
+ from t1;
+
+/* composite pk */
+
+create view v9_pk_composite(
+ c_id1 int32 not unknown,
+ c_id2 int32 not unknown
+) default null
+ primary key (c_id1, c_id2) not enforced
+as
+ select c_id as c_id1, -c_id as c_id2
+ from t1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.10.query.sqlpp
new file mode 100644
index 0000000..62dddff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.10.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test1;
+
+select c_id, c_x, c_y
+from v8_no_nulls_multi
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.11.query.sqlpp
new file mode 100644
index 0000000..71bb255
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.11.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test1;
+
+select c_id1, c_id2
+from v9_pk_composite
+order by c_id1, c_id2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.12.query.sqlpp
new file mode 100644
index 0000000..96ab466
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.12.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+select d.DataverseName, d.DatasetName, d.ViewDetails, dt.Derived.Record.Fields as DatatypeFields
+from Metadata.`Dataset` d
+join Metadata.`Datatype` dt on d.DatatypeDataverseName = dt.DataverseName and d.DatatypeName = dt.DatatypeName
+where d.DatasetType='VIEW'
+order by d.DataverseName, d.DatasetName;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.2.update.sqlpp
new file mode 100644
index 0000000..4c91491
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.2.update.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * 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 test1;
+
+insert into t1 ([
+ {
+ 'c_id':0,
+ 'c_i8':'8','c_i16':'16','c_i32':'32','c_i64':'64','c_f':'1.5','c_d':'2.25',
+ 'c_b':false,'c_s':'abc',
+ 'c_datetime':'2020-02-03T10:11:12.001','c_date':'2020-02-03','c_time':'10:11:12.001',
+ 'c_dur':'P30Y10M25DT13H12M50S','c_ymdur':'P30Y10M','c_dtdur':'P25DT13H12M50S'
+ },
+
+ {
+ 'c_id':1,
+ 'c_i8':'-8','c_i16':'-16','c_i32':'-32','c_i64':'-64','c_f':'-1.5','c_d':'-2.25',
+ 'c_b':true,'c_s':'xyz',
+ 'c_datetime':'2021-04-05T01:02:03.999','c_date':'2021-04-05','c_time':'01:02:03.999',
+ 'c_dur':'P1Y2M3DT4H5M6S','c_ymdur':'P1Y2M','c_dtdur':'P3DT4H5M6S'
+ },
+
+ /* Null values */
+ {
+ 'c_id':2,
+ 'c_i8':null,'c_i16':null,'c_i32':null,'c_i64':null,'c_f':null,'c_d':null,
+ 'c_b':null,'c_s':null,
+ 'c_datetime':null,'c_date':null,'c_time':null,
+ 'c_dur':null,'c_ymdur':null,'c_dtdur':null
+ },
+
+ /* Missing values */
+ {
+ 'c_id':3
+ },
+
+ /* Invalid values */
+ {
+ 'c_id':4,
+ 'c_i8':'a','c_i16':'b','c_i32':'c','c_i64':'d','c_f':'e','c_d':'f',
+ 'c_b':99,'c_s':null,
+ 'c_datetime':'a','c_date':'b','c_time':'c',
+ 'c_dur':'x','c_ymdur':'y','c_dtdur':'z',
+ 'c_something_else':'something_else'
+ }
+]);
+
+/* Custom date/time format */
+insert into t2 ([
+ {
+ 'c_id':0,
+ 'c_datetime':'02/20/2020 11:40:41.001pm','c_date':'02/20/2020','c_time':'11:40:41.001pm'
+ },
+ {
+ 'c_id':1,
+ 'c_datetime':'11/25/2021 11:50:51.999am','c_date':'11/25/2021','c_time':'11:50:51.999am'
+ },
+ /* Invalid values */
+ {
+ 'c_id':2,
+ 'c_datetime':'a','c_date':'b','c_time':'c'
+ }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.3.query.sqlpp
new file mode 100644
index 0000000..c8126cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 test1;
+
+select c_id,
+ c_i8, c_i16, c_i32, c_i64, c_f, c_d,
+ c_b, c_s,
+ c_datetime, c_date, c_time,
+ c_dur, c_ymdur, c_dtdur
+from v1
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.4.query.sqlpp
new file mode 100644
index 0000000..59a713a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 test1;
+
+select c_id,
+ c_i8, c_i16, c_i32, c_i64, c_f, c_d,
+ c_b, c_s,
+ c_datetime, c_date, c_time,
+ c_dur, c_ymdur, c_dtdur
+from v2_ref_type
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.5.query.sqlpp
new file mode 100644
index 0000000..19be7a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test1;
+
+select c_id,
+ c_datetime, c_date, c_time
+from v3_datetime_format
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.6.query.sqlpp
new file mode 100644
index 0000000..fd96010
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.6.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test1;
+
+select c_id, c_date
+from v4_date_format_only
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.7.query.sqlpp
new file mode 100644
index 0000000..6513352
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.7.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test1;
+
+select c_id, c_date
+from v5_pk
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.8.query.sqlpp
new file mode 100644
index 0000000..35a7799
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.8.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test1;
+
+select c_id, c_i64
+from v6_pk_no_nulls
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.9.query.sqlpp
new file mode 100644
index 0000000..6f14510
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-3-typed/create-view-3-typed.9.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test1;
+
+select c_id, c_i64
+from v7_no_nulls
+order by c_id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.1.ddl.sqlpp
new file mode 100644
index 0000000..2c39dd1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset t1(c_id int32 not unknown) open type primary key c_id;
+
+create view v1(
+ c_id int32,
+ c_i8 int8, c_i16 int16, c_i32 int32, c_i64 int64, c_f float, c_d double,
+ c_b boolean, c_s string,
+ c_datetime datetime, c_date date, c_time time,
+ c_dur duration, c_ymdur year_month_duration, c_dtdur day_time_duration
+) default null as t1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.2.update.sqlpp
new file mode 100644
index 0000000..644893f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.2.update.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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 test1;
+
+insert into t1 ([
+ /* Invalid format (string values) */
+ {
+ 'c_id':0,
+ 'c_i8':'a','c_i16':'b','c_i32':'c','c_i64':'d','c_f':'e','c_d':'f',
+ 'c_b':null,'c_s':null,
+ 'c_datetime':'g','c_date':'h','c_time':'j',
+ 'c_dur':'k','c_ymdur':'m','c_dtdur':'n'
+ },
+ /* Invalid type */
+ {
+ 'c_id':1,
+ 'c_i8':duration('P1M8S'),'c_i16':duration('P1M16S'),'c_i32':duration('P1M32S'),'c_i64':duration('P1M64S'),'c_f':duration('P2M32S'),'c_d':duration('P2M64S'),
+ 'c_b':duration('P3M1S'),'c_s':null,
+ 'c_datetime':duration('P4M1S'),'c_date':duration('P5M1S'),'c_time':duration('P6M1S'),
+ 'c_dur':date('2020-01-02'),'c_ymdur':date('2020-01-02'),'c_dtdur':date('2020-01-02')
+ }
+]);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.3.query.sqlpp
new file mode 100644
index 0000000..e600abd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-4-typed-warn/create-view-4-typed-warn.3.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+use test1;
+
+select c_id,
+ c_i8, c_i16, c_i32, c_i64, c_f, c_d,
+ c_b, c_s,
+ c_datetime, c_date, c_time,
+ c_dur, c_ymdur, c_dtdur
+from v1
+order by c_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.1.ddl.sqlpp
new file mode 100644
index 0000000..d03801e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset t1(c_id int32 not unknown) open type primary key c_id;
+
+/* invalid custom date/time format (in with clause) */
+create view v1_invalid_datetime_format(
+ c_id int32,
+ c_datetime datetime, c_date date, c_time time
+) default null
+ datetime 'XX.ZZ.11'
+ date 'XX.ZZ.22'
+ time 'XX.ZZ.33'
+as t1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.2.update.sqlpp
new file mode 100644
index 0000000..3a6173f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 test1;
+
+insert into t1 ([
+ /* Invalid format (string values) */
+ {
+ 'c_id':0,
+ 'c_datetime':'a','c_date':'b','c_time':'c'
+ }
+]);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.3.query.sqlpp
new file mode 100644
index 0000000..dd71835
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-5-typed-warn/create-view-5-typed-warn.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+// param max-warnings:json=100
+
+use test1;
+
+select c_id,
+ c_datetime, c_date, c_time
+from v1_invalid_datetime_format
+order by c_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.1.ddl.sqlpp
new file mode 100644
index 0000000..859b498
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.1.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: unknown datatype's dataverse
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(unknown_dv.t1) default null as
+ select * from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.10.ddl.sqlpp
new file mode 100644
index 0000000..f3c2076
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.10.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+--- Negative: primary key declaration requires "not enforced" modifier
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint)
+ default null
+ primary key (r)
+ as select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.11.ddl.sqlpp
new file mode 100644
index 0000000..dd69e56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.11.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: view inline type has duplicate fields
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint, r bigint) default null as
+ select r, [r] a from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.12.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.12.ddl.sqlpp
new file mode 100644
index 0000000..53e94fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.12.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/*
+ * Negative: cannot replace typed view if the replacement declaration has a different primary key
+ * because there might be another typed view that refers to its primary key
+ * via foreign key declaration.
+ * (this limitation will be relaxed in the future)
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+create dataset test1.employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view test1.employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee;
+
+create or replace view test1.employee_v1(e_id int not unknown, e_mgr_id int not unknown)
+ default null
+ primary key (e_id, e_mgr_id) not enforced
+ as employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.13.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.13.ddl.sqlpp
new file mode 100644
index 0000000..101ed8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.13.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+--- Negative: primary key declaration refers to a non-existent nested field
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint not unknown)
+ default null
+ primary key (r.unknown_field) not enforced
+ as select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.14.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.14.ddl.sqlpp
new file mode 100644
index 0000000..26a9668
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.14.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+--- Negative: primary key declaration uses
+--- meta() reference
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint not unknown)
+ default null
+ primary key (meta().unknown_field) not enforced
+ as select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.15.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.15.ddl.sqlpp
new file mode 100644
index 0000000..00c186e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.15.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+/*
+ * Negative: foreign key definition requires 'not enforced' modifier
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id) references employee_v1
+ as employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.16.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.16.ddl.sqlpp
new file mode 100644
index 0000000..212f31d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.16.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * Negative: cannot create self-referenced foreign key if primary key is undefined
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ foreign key (e_mgr_id) references employee_v1 not enforced
+ as employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.17.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.17.ddl.sqlpp
new file mode 100644
index 0000000..2b33d73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.17.ddl.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.
+ */
+
+/*
+ * Negative: cannot create self-referenced foreign key if
+ * its definition doesn't match the primary key
+ * (foreign key declaration has more fields)
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int, e_hrr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id, e_hrr_id) references employee_v1 not enforced
+ as employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.18.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.18.ddl.sqlpp
new file mode 100644
index 0000000..493f31b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.18.ddl.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.
+ */
+
+/*
+ * Negative: cannot create self-referenced foreign key if
+ * its definition doesn't match the primary key
+ * (foreign key declaration has fewer fields)
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset employee(e_id1 int not unknown, e_id2 int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id1, e_id2;
+
+create view employee_v2(e_id1 int not unknown, e_id2 int not unknown, e_mgr_id int, e_hrr_id int)
+ default null
+ primary key (e_id1, e_id2) not enforced
+ foreign key (e_mgr_id) references employee_v2 not enforced
+ as employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.19.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.19.ddl.sqlpp
new file mode 100644
index 0000000..af69c77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.19.ddl.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.
+ */
+
+/*
+ * Negative: cannot create self-referenced foreign key if
+ * its definition doesn't match the primary key
+ * (foreign key declaration has same fields as primary key)
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset employee(e_id1 int not unknown, e_id2 int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id1, e_id2;
+
+create view employee_v2(e_id1 int not unknown, e_id2 int not unknown, e_mgr_id int, e_hrr_id int)
+ default null
+ primary key (e_id1, e_id2) not enforced
+ foreign key (e_id2, e_id1) references employee_v2 not enforced
+ as employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.2.ddl.sqlpp
new file mode 100644
index 0000000..571d3d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.2.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: unknown datatype
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(t1_unknown) default null as
+ select * from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.20.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.20.ddl.sqlpp
new file mode 100644
index 0000000..11c8f18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.20.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Negative: cannot create foreign key if referenced dataverse does not exist
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee;
+
+use test2;
+
+create view employee_v2(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id) references test3.employee_v1 not enforced
+ as test1.employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.21.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.21.ddl.sqlpp
new file mode 100644
index 0000000..9504a91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.21.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Negative: cannot create foreign key if referenced view does not exist
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee;
+
+use test2;
+
+create view employee_v2(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id) references test1.employee_v3 not enforced
+ as test1.employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.22.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.22.ddl.sqlpp
new file mode 100644
index 0000000..6a1dcc1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.22.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+/*
+ * Negative: cannot create foreign key if referenced object exist, but is not a view
+ * (this use-case will be allowed in the future)
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee;
+
+use test2;
+
+create view employee_v2(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id) references test1.employee not enforced
+ as test1.employee_v1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.23.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.23.ddl.sqlpp
new file mode 100644
index 0000000..d751f03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.23.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+/*
+ * Negative: invalid foreign key definition. nested field specified.
+ *
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee;
+
+use test2;
+
+create view employee_v2(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id.e_mgr_id2) references test1.employee_v1 not enforced
+ as test1.employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.24.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.24.ddl.sqlpp
new file mode 100644
index 0000000..d1d3be8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.24.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.
+ */
+
+/*
+ * Negative: invalid foreign key definition. meta() field specified.
+ *
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create dataset test1.employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view test1.employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee;
+
+create view test2.employee_v2(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (meta().e_mgr_id) references test1.employee_v1 not enforced
+ as test1.employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.25.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.25.ddl.sqlpp
new file mode 100644
index 0000000..d718fa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.25.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.
+ */
+
+/*
+ * Negative: invalid foreign key definition. number of fields
+ * doesn't match number of primary key fields of the referred view
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create dataset test1.employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view test1.employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee;
+
+create view test2.employee_v2(e_id int not unknown, e_mgr_id int, e_hrr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id, e_hrr_id) references test1.employee_v1 not enforced
+ as test1.employee;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.3.ddl.sqlpp
new file mode 100644
index 0000000..73de71e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.3.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+--- Negative: view type is not closed
+
+drop dataverse test if exists;
+create dataverse test;
+
+create type test.t1 as open {
+ r:int64?
+};
+
+create view test.v1(t1) default null as
+ select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.4.ddl.sqlpp
new file mode 100644
index 0000000..e900a32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.4.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+--- Negative: view type has non-primitive fields
+
+drop dataverse test if exists;
+create dataverse test;
+
+create type test.t1 as closed {
+ r:int64?,
+ a:[int64]?
+};
+
+create view test.v1(t1) default null as
+ select r, [r] a from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.5.ddl.sqlpp
new file mode 100644
index 0000000..4b814bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.5.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: view inline type has non-primitive fields
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint, a [bigint]) default null as
+ select r, [r] a from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.6.ddl.sqlpp
new file mode 100644
index 0000000..2e01c03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.6.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+--- Negative: invalid view parameter clause
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(cd date) default null
+ date 'YYYY-MM-DD'
+ date_illegal_property_name 'YYYY-MM-DD'
+as
+ select string(current_date()) cd from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.7.ddl.sqlpp
new file mode 100644
index 0000000..adb53a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.7.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+--- Negative: default null is required
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint) as
+ select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.8.ddl.sqlpp
new file mode 100644
index 0000000..dd7e995
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.8.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+--- Negative: primary key declaration refers to a non-existent field
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint)
+ default null
+ primary key (unknown_field) not enforced
+ as select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.9.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.9.ddl.sqlpp
new file mode 100644
index 0000000..8aba1c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-6-typed-negative/create-view-6-typed-negative.9.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+--- Negative: primary key declaration refers to a non-existent field
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1(r bigint not unknown)
+ default null
+ primary key (r, unknown_field_2) not enforced
+ as select r from range(1,2) r;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.1.ddl.sqlpp
new file mode 100644
index 0000000..2e6f347
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.1.ddl.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test 'self reference' foreign keys
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test1;
+
+create dataset employee(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view employee_v1(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id) references employee_v1 not enforced
+ as employee;
+
+create view employee_v2(e_id int not unknown, e_mgr_id int, e_hrr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id) references test1.employee_v2 not enforced
+ foreign key (e_hrr_id) references test1.employee_v2 not enforced
+ as employee;
+
+use test2;
+
+create dataset employee2(e_id1 int not unknown, e_id2 int not unknown, e_name string,
+ e_mgr_id1 int, e_mgr_id2 int, e_hrr_id1 int, e_hrr_id2 int)
+ primary key e_id1, e_id2;
+
+create view employee2_v1(e_id1 int not unknown, e_id2 int not unknown,
+ e_mgr_id1 int, e_mgr_id2 int, e_hrr_id1 int, e_hrr_id2 int)
+ default null
+ primary key (e_id1, e_id2) not enforced
+ foreign key (e_mgr_id1, e_mgr_id2) references employee2_v1 not enforced
+ foreign key (e_hrr_id1, e_hrr_id2) references employee2_v1 not enforced
+ as employee2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.2.query.sqlpp
new file mode 100644
index 0000000..2f44238
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+select d.DataverseName, d.DatasetName, d.ViewDetails
+from Metadata.`Dataset` d
+where d.DataverseName like "test%" and d.DatasetType = "VIEW"
+order by d.DataverseName, d.DatasetName;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.3.ddl.sqlpp
new file mode 100644
index 0000000..2f23876
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.3.ddl.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+/*
+ * Test multiple foreign keys, cross-dataverse references, composite keys
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test2;
+
+create dataset customers(c_id int not unknown, c_name string) primary key c_id;
+
+create dataset stores(s_id1 int not unknown, s_id2 int not unknown, s_name string) primary key s_id1, s_id2;
+
+create dataset orders(o_id int not unknown, o_cid int, o_sidX int, o_sidY int, o_amount int) primary key o_id;
+
+create view customers_v(c_id int not unknown, c_name string) default null
+ primary key (c_id) not enforced
+ as customers;
+
+use test1;
+
+create view stores_v(s_id1 int not unknown, s_id2 int not unknown, s_name string) default null
+ primary key (s_id1, s_id2) not enforced
+ as test2.stores;
+
+create view orders_v(o_id int not unknown, o_cid int, o_sidX int, o_sidY int, o_amount int) default null
+ primary key (o_id) not enforced
+ foreign key (o_cid) references test2.customers_v not enforced
+ foreign key (o_sidX, o_sidY) references test1.stores_v not enforced
+ as test2.orders;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.4.query.sqlpp
new file mode 100644
index 0000000..2f44238
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/create-view-7-foreign-key/create-view-7-foreign-key.4.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+select d.DataverseName, d.DatasetName, d.ViewDetails
+from Metadata.`Dataset` d
+where d.DataverseName like "test%" and d.DatasetType = "VIEW"
+order by d.DataverseName, d.DatasetName;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-1/drop-dataverse-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-1/drop-dataverse-1.1.ddl.sqlpp
new file mode 100644
index 0000000..87c26cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-1/drop-dataverse-1.1.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+--- test drop dataverse with views
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+create view test1.v1 as
+ select r from range(0,2) r;
+
+create view test1.v2 as
+ select v1.* from v1;
+
+create view test1.v3(r bigint) default null as v2;
+
+create type test1.t4 as closed { r:int64? };
+
+create view test1.v4(t4) default null as v2;
+
+create view test1.v5(r bigint not unknown)
+ default null
+ primary key (r) not enforced
+ as v3;
+
+create view test1.v6(r1 bigint not unknown, r2 bigint)
+ default null
+ primary key (r1) not enforced
+ foreign key (r2) references v5 not enforced
+ as select r r1, r+1 r2 from range(0,2) r;
+
+drop dataverse test1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-1/drop-dataverse-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-1/drop-dataverse-1.2.query.sqlpp
new file mode 100644
index 0000000..b707271
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-1/drop-dataverse-1.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+select count(*) cnt
+from Metadata.`Dataverse` d
+where d.DataverseName like "test%";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.1.ddl.sqlpp
new file mode 100644
index 0000000..f3f0334
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+--- Test that DROP DATAVERSE fails due to cross-dataverse dependencies
+
+--- View depends on view
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create view test2.v2 as select r from range(1,2) r;
+
+create view test1.v1 as select v2.* from test2.v2;
+
+drop dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.2.ddl.sqlpp
new file mode 100644
index 0000000..d8f90bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.2.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+--- Test that DROP DATAVERSE fails due to cross-dataverse dependencies
+
+--- Function depends on view
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create view test2.v2 as select r from range(1,2) r;
+
+create function test1.f1() { select v2.* from test2.v2 };
+
+drop dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.3.ddl.sqlpp
new file mode 100644
index 0000000..5089a6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.3.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+--- Test that DROP DATAVERSE fails due to cross-dataverse dependencies
+
+--- View depends on dataset
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create dataset test2.ds2(c1 bigint not unknown, c2 bigint) primary key c1;
+
+create view test1.v1 as select ds2.* from test2.ds2;
+
+drop dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.4.ddl.sqlpp
new file mode 100644
index 0000000..8c28081
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.4.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+--- Test that DROP DATAVERSE fails due to cross-dataverse dependencies
+
+--- View depends on function
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create function test2.f2() { select r from range(1,2) r };
+
+create view test1.v1 as select f2.* from test2.f2() f2;
+
+drop dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.5.ddl.sqlpp
new file mode 100644
index 0000000..cf055ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.5.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+--- Test that DROP DATAVERSE fails due to cross-dataverse dependencies
+
+--- View depends on synonym
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create dataset test2.ds2(c1 bigint not unknown, c2 bigint) primary key c1;
+
+create synonym test2.s3 for ds2;
+
+create view test1.v1 as select s3.* from test2.s3;
+
+drop dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.6.ddl.sqlpp
new file mode 100644
index 0000000..130e769
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.6.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+--- Test that DROP DATAVERSE fails due to cross-dataverse dependencies
+
+--- View depends on datatype
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create type test2.t1 as closed { r:int64? };
+
+create view test1.v1(test2.t1) default null as select r from range(1,2) r;
+
+drop dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.7.ddl.sqlpp
new file mode 100644
index 0000000..0456ab7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-dataverse-2-negative/drop-dataverse-2-negative.7.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+--- Test that DROP DATAVERSE fails due to cross-dataverse dependencies
+
+--- View refers to another view view foreign key declaration
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create dataset test2.employee_2(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view test2.employee_v2(e_id int not unknown, e_mgr_id int)
+ default null
+ primary key (e_id) not enforced
+ as employee_2;
+
+create dataset test1.employee_1(e_id int not unknown, e_name string, e_mgr_id int, e_hrr_id int)
+ primary key e_id;
+
+create view test1.employee_v1(e_id int not unknown, e_mgr_id int, e_hrr_id int)
+ default null
+ primary key (e_id) not enforced
+ foreign key (e_mgr_id) references test2.employee_v2 not enforced
+ as employee_1;
+
+drop dataverse test2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.1.ddl.sqlpp
new file mode 100644
index 0000000..16c1d6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+drop view test.v1 if exists;
+
+create view test.v2 as
+ select r from range(1,2) r;
+
+drop view test.v2 if exists;
+
+create view test.v3 as
+ select r from range(3,4) r;
+
+drop view test.v3;
+
+create view test.v4 as
+ select r from range(3,4) r;
+
+create synonym test.s4 for v4;
+
+-- ok to drop a view if a synonym depends on it
+
+drop view test.v4;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.2.query.sqlpp
new file mode 100644
index 0000000..4c8483e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+select count(*) cnt
+from Metadata.`Dataset`
+where DataverseName = "test" and DatasetType = "VIEW";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.3.ddl.sqlpp
new file mode 100644
index 0000000..c64efbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.3.ddl.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.
+ */
+
+/* Drop views and inline types, user-defined datatype must remain in the dataverse */
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as
+ select r from range(1,2) r;
+
+create view test.v2(r bigint) default null as v1;
+
+create type test.t3 as closed { r:int64? };
+
+create view test.v3(t3) default null as v1;
+
+drop view test.v3;
+
+drop view test.v2;
+
+drop view test.v1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.4.query.sqlpp
new file mode 100644
index 0000000..4c8483e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+select count(*) cnt
+from Metadata.`Dataset`
+where DataverseName = "test" and DatasetType = "VIEW";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.5.query.sqlpp
new file mode 100644
index 0000000..5bf0799
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-1/drop-view-1.5.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+select DatatypeName
+from Metadata.`Datatype`
+where DataverseName = "test"
+order by DatatypeName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.1.ddl.sqlpp
new file mode 100644
index 0000000..c92c33a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.1.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+--- Negative: unknown dataverse
+
+drop dataverse test if exists;
+
+drop view test.v1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.10.query.sqlpp
new file mode 100644
index 0000000..7aa6517
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.10.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- check that the function was not dropped
+
+select value f2 from test2.f2() f2 order by f2.r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.11.ddl.sqlpp
new file mode 100644
index 0000000..009dd83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.11.ddl.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.
+ */
+
+--- Negative: drop synonym fails if a view depends on it
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create dataset test2.ds2(
+ c_custkey integer not unknown,
+ c_name string not unknown
+) primary key c_custkey;
+
+create synonym test2.s2 for test2.ds2;
+
+create view test1.v1 as
+ select * from test2.s2;
+
+drop synonym test2.s2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.12.query.sqlpp
new file mode 100644
index 0000000..4a425b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.12.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- check that the synonym was not dropped
+
+select value count(*) from test2.s2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.13.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.13.ddl.sqlpp
new file mode 100644
index 0000000..27acbde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.13.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+--- Negative: drop view fails if a view depends on it
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create view test2.v2 as
+ select * from range(1,2) r;
+
+create view test1.v1 as
+ select * from test2.v2;
+
+drop view test2.v2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.14.query.sqlpp
new file mode 100644
index 0000000..f009fcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.14.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- check that the view was not dropped
+
+select value v2 from test2.v2 v2 order by v2.r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.15.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.15.ddl.sqlpp
new file mode 100644
index 0000000..c759a13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.15.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+--- Negative: drop view fails if a function depends on it
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create view test2.v2 as
+ select * from range(1,2) r;
+
+create function test1.f1() {
+ select * from test2.v2
+};
+
+drop view test2.v2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.16.query.sqlpp
new file mode 100644
index 0000000..f009fcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.16.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- check that the view was not dropped
+
+select value v2 from test2.v2 v2 order by v2.r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.17.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.17.ddl.sqlpp
new file mode 100644
index 0000000..0fbb626
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.17.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+--- Negative: drop type fails if a view depends on it
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create type test2.t1 as closed { r:int64? };
+
+create view test1.v1(test2.t1) default null as
+ select r from range(1,2) r;
+
+drop type test2.t1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.18.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.18.query.sqlpp
new file mode 100644
index 0000000..29f7a8d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.18.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+-- check that the type was not dropped
+
+select DatatypeName
+from Metadata.`Datatype`
+where DataverseName = "test2"
+order by DatatypeName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.2.ddl.sqlpp
new file mode 100644
index 0000000..4480882
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.2.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+--- Negative: unknown view
+
+drop dataverse test if exists;
+create dataverse test;
+
+drop view test.v1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.3.ddl.sqlpp
new file mode 100644
index 0000000..2034c22
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.3.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+--- Negative: drop dataset fails if target is a view
+
+drop dataverse test if exists;
+create dataverse test;
+
+create view test.v1 as select r from range(1,2) r;
+
+drop dataset test.v1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.4.query.sqlpp
new file mode 100644
index 0000000..1275dbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- check that the view was not dropped
+
+select value r from test.v1 r order by r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.5.ddl.sqlpp
new file mode 100644
index 0000000..3a1fd33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.5.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+--- Negative: drop view fails if target is a dataset
+
+drop dataverse test if exists;
+create dataverse test;
+
+create dataset test.ds1(
+ c_custkey integer not unknown,
+ c_name string not unknown
+) primary key c_custkey;
+
+drop view test.ds1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.6.query.sqlpp
new file mode 100644
index 0000000..12971b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.6.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- check that the dataset was not dropped
+
+select value count(*) from test.ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.7.ddl.sqlpp
new file mode 100644
index 0000000..92d2270
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.7.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+--- Negative: drop dataset fails if a view depends on it
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create dataset test2.ds2(
+ c_custkey integer not unknown,
+ c_name string not unknown
+) primary key c_custkey;
+
+create view test1.v1 as
+ select * from test2.ds2;
+
+drop dataset test2.ds2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.8.query.sqlpp
new file mode 100644
index 0000000..dd85dbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.8.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+-- check that the dataset was not dropped
+
+select value count(*) from test2.ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.9.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.9.ddl.sqlpp
new file mode 100644
index 0000000..83d4fab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/drop-view-2-negative/drop-view-2-negative.9.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+--- Negative: drop function fails if a view depends on it
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+create function test2.f2() {
+ select * from range(1,2) r
+};
+
+create view test1.v1 as
+ select * from test2.f2() f2;
+
+drop function test2.f2();
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.1.ddl.sqlpp
new file mode 100644
index 0000000..e5a22ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.1.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset ds1(
+ c1 bigint not unknown,
+ c2 bigint
+) primary key c1;
+
+create synonym s2 for ds1;
+
+create view v3 as
+ select c1, c2 from s2;
+
+create synonym s4 for v3;
+
+create function f5() {
+ select c1, c2 from s4
+};
+
+create view v6 as
+ select c1, c2 from f5() f5;
+
+create synonym s7 for v6;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.2.update.sqlpp
new file mode 100644
index 0000000..260b64f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 test1;
+
+insert into ds1 select r c1, 9 - r c2 from range(0, 9) r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.3.query.sqlpp
new file mode 100644
index 0000000..6c92c4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 test1;
+
+select s7.* from s7 order by c1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.4.query.sqlpp
new file mode 100644
index 0000000..57b5010
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+select "VIEW" Kind, DatasetName Name,
+ ViewDetails.`Definition` `Definition`,
+ ViewDetails.Dependencies Dependencies
+from Metadata.`Dataset`
+where DataverseName like "test1%" and DatasetType = "VIEW"
+
+union all
+
+select "FUNCTION" Kind, Name, `Definition`, Dependencies
+from Metadata.`Function`
+where DataverseName like "test1%"
+
+order by Name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.5.ddl.sqlpp
new file mode 100644
index 0000000..97f9ec4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.5.ddl.sqlpp
@@ -0,0 +1,74 @@
+/*
+ * 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.
+ */
+
+drop dataverse test2 if exists;
+create dataverse test2;
+
+use test2;
+
+create dataset ds1(
+ c1 bigint not unknown,
+ c2 bigint
+) primary key c1;
+
+create dataset ds2(
+ c1 bigint not unknown,
+ c2 bigint
+) primary key c1;
+
+create view v3 as
+ select c1, c2 from ds1
+ union all
+ select c1, c2 from ds2;
+
+create view v4 as
+ select c1, c2 from ds1
+ union all
+ select c1, c2 from ds2;
+
+create function f5() {
+ select c1, c2 from v3
+};
+
+create function f6() {
+ select c1, c2 from v4
+};
+
+create synonym s7 for ds1;
+
+create synonym s8 for ds2;
+
+create view v9 as
+ select c1, c2 from ds1
+ union all
+ select c1, c2 from ds2
+ union all
+ select c1, c2 from v3
+ union all
+ select c1, c2 from v4
+ union all
+ select c1, c2 from f5() f5
+ union all
+ select c1, c2 from f6() f6
+ union all
+ select c1, c2 from s7
+ union all
+ select c1, c2 from s8;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.6.update.sqlpp
new file mode 100644
index 0000000..902c718
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.6.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test2;
+
+insert into ds1 select r c1, 2 - r c2 from range(0, 2) r;
+
+insert into ds2 select r c1, 2 - r c2 from range(0, 2) r;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.7.query.sqlpp
new file mode 100644
index 0000000..762f234
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test2;
+
+select count(*) c,
+ array_sort(array_agg(distinct c1)) c1,
+ array_sort(array_agg(distinct c2)) c2
+from v9;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.8.query.sqlpp
new file mode 100644
index 0000000..d41ee20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-1/view-1.8.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+select "VIEW" Kind, DatasetName Name,
+ ViewDetails.Dependencies Dependencies
+from Metadata.`Dataset`
+where DataverseName like "test2%" and DatasetType = "VIEW"
+
+union all
+
+select "FUNCTION" Kind, Name, Dependencies
+from Metadata.`Function`
+where DataverseName like "test2%"
+
+order by Name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.1.ddl.sqlpp
new file mode 100644
index 0000000..90f0d31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+
+use test1;
+
+create dataset t1(id bigint not unknown, v string) primary key id;
+
+create view v1 as t1;
+
+create view v2 as t1;
+
+create view v3 as t1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.2.update.sqlpp
new file mode 100644
index 0000000..5e4b305
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.2.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test1;
+
+insert into t1 ([
+ {'id':1,'v':'hello1'},
+ {'id':2,'v':'hello2'}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.3.update.sqlpp
new file mode 100644
index 0000000..89507a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.3.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 test1;
+
+--- Failure: cannot insert into a view
+
+insert into v1 ([
+ {'id':3,'v':'hello1'}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.4.update.sqlpp
new file mode 100644
index 0000000..103f7f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.4.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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 test1;
+
+--- Failure: cannot upsert into a view
+
+upsert into v2 ([
+ {'id':2,'v':'hello22'}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.5.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.5.update.sqlpp
new file mode 100644
index 0000000..b3e5524
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/view/view-2-negative/view-2-negative.5.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test1;
+
+--- Failure: cannot delete from a view
+
+delete from v3
+where id = 2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp
new file mode 100644
index 0000000..57545a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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 : Test multiple window functions in the same statement
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+select c2,
+ (
+ select nth_value(c2, 3)
+ over (partition by one order by c2 range between unbounded preceding and unbounded following)
+ as nth
+ from t1 x
+ ) as q1
+from t1 y
+order by c2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_null_missing/win_null_missing.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_null_missing/win_null_missing.4.query.sqlpp
new file mode 100644
index 0000000..b1c215c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_null_missing/win_null_missing.4.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : ORDER BY MISSING/NULL/complex NULLS FIRST
+ * Expected Res : SUCCESS
+ */
+
+from [
+ { "y": "m" },
+ { "x": null, "y": "n" },
+ { "x": 1, "y": "i" },
+ { "x": "a", "y": "s" },
+ { "x": [ "b" ], "y": "a" },
+ { "x": { "c": 1 }, "y": "o" }
+] t
+select
+ nth_value(y, 0) over (order by x nulls first rows between unbounded preceding and unbounded following) w0,
+ nth_value(y, 1) over (order by x nulls first rows between unbounded preceding and unbounded following) w1,
+ nth_value(y, 2) over (order by x nulls first rows between unbounded preceding and unbounded following) w2,
+ nth_value(y, 3) over (order by x nulls first rows between unbounded preceding and unbounded following) w3,
+ nth_value(y, 4) over (order by x nulls first rows between unbounded preceding and unbounded following) w4,
+ nth_value(y, 5) over (order by x nulls first rows between unbounded preceding and unbounded following) w5,
+ nth_value(y, 6) over (order by x nulls first rows between unbounded preceding and unbounded following) w6,
+ x, y
+order by x, y
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_null_missing/win_null_missing.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_null_missing/win_null_missing.5.query.sqlpp
new file mode 100644
index 0000000..8d35f92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/win_null_missing/win_null_missing.5.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * 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 : ORDER BY MISSING/NULL/complex NULLS LAST
+ * Expected Res : SUCCESS
+ */
+
+from [
+ { "y": "m" },
+ { "x": null, "y": "n" },
+ { "x": 1, "y": "i" },
+ { "x": "a", "y": "s" },
+ { "x": [ "b" ], "y": "a" },
+ { "x": { "c": 1 }, "y": "o" }
+] t
+select
+ nth_value(y, 0) over (order by x nulls last rows between unbounded preceding and unbounded following) w0,
+ nth_value(y, 1) over (order by x nulls last rows between unbounded preceding and unbounded following) w1,
+ nth_value(y, 2) over (order by x nulls last rows between unbounded preceding and unbounded following) w2,
+ nth_value(y, 3) over (order by x nulls last rows between unbounded preceding and unbounded following) w3,
+ nth_value(y, 4) over (order by x nulls last rows between unbounded preceding and unbounded following) w4,
+ nth_value(y, 5) over (order by x nulls last rows between unbounded preceding and unbounded following) w5,
+ nth_value(y, 6) over (order by x nulls last rows between unbounded preceding and unbounded following) w6,
+ x, y
+order by x, y
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/kurtosis_distinct/kurtosis_distinct.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/kurtosis_distinct/kurtosis_distinct.1.adm
index 0f3644d..9ad4a6d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/kurtosis_distinct/kurtosis_distinct.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/kurtosis_distinct/kurtosis_distinct.1.adm
@@ -1 +1 @@
-{ "t1": -2, "t2": null, "t3": null, "t4": null }
+{ "t1": -1, "t2": null, "t3": null, "t4": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max/scalar_max.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max/scalar_max.1.adm
index 9c12d9b..00ce6ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max/scalar_max.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max/scalar_max.1.adm
@@ -5,4 +5,4 @@
3.0
3.0
"world"
-datetime("2012-03-01T00:00:00.000Z")
+datetime("2012-03-01T00:00:00.000")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max_null/scalar_max_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max_null/scalar_max_null.1.adm
index 9c12d9b..00ce6ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max_null/scalar_max_null.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_max_null/scalar_max_null.1.adm
@@ -5,4 +5,4 @@
3.0
3.0
"world"
-datetime("2012-03-01T00:00:00.000Z")
+datetime("2012-03-01T00:00:00.000")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min/scalar_min.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min/scalar_min.1.adm
index e052b95..e5b1130 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min/scalar_min.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min/scalar_min.1.adm
@@ -5,4 +5,4 @@
1.0
1.0
"bar"
-datetime("2012-01-01T00:00:00.000Z")
+datetime("2012-01-01T00:00:00.000")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min_null/scalar_min_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min_null/scalar_min_null.1.adm
index e052b95..e5b1130 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min_null/scalar_min_null.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/scalar_min_null/scalar_min_null.1.adm
@@ -5,4 +5,4 @@
1.0
1.0
"bar"
-datetime("2012-01-01T00:00:00.000Z")
+datetime("2012-01-01T00:00:00.000")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/kurtosis_distinct/kurtosis_distinct.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/kurtosis_distinct/kurtosis_distinct.1.adm
index 0f3644d..9ad4a6d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/kurtosis_distinct/kurtosis_distinct.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/kurtosis_distinct/kurtosis_distinct.1.adm
@@ -1 +1 @@
-{ "t1": -2, "t2": null, "t3": null, "t4": null }
+{ "t1": -1, "t2": null, "t3": null, "t4": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_max/scalar_max.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_max/scalar_max.1.adm
index 9c12d9b..00ce6ab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_max/scalar_max.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_max/scalar_max.1.adm
@@ -5,4 +5,4 @@
3.0
3.0
"world"
-datetime("2012-03-01T00:00:00.000Z")
+datetime("2012-03-01T00:00:00.000")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_min/scalar_min.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_min/scalar_min.1.adm
index e052b95..e5b1130 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_min/scalar_min.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_min/scalar_min.1.adm
@@ -5,4 +5,4 @@
1.0
1.0
"bar"
-datetime("2012-01-01T00:00:00.000Z")
+datetime("2012-01-01T00:00:00.000")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index 473319c..bf736b4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -8,8 +8,8 @@
"active\.memory\.global\.budget" : 67108864,
"active\.stop\.timeout" : 3600,
"active\.suspend\.timeout" : 3600,
- "compiler\.arrayindex" : false,
- "compiler\.external\.field\.pushdown" : false,
+ "compiler\.arrayindex" : true,
+ "compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 171ead0..d52cedd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -8,8 +8,8 @@
"active\.memory\.global\.budget" : 67108864,
"active\.stop\.timeout" : 3600,
"active\.suspend\.timeout" : 3600,
- "compiler\.arrayindex" : false,
- "compiler\.external\.field\.pushdown" : false,
+ "compiler\.arrayindex" : true,
+ "compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 4a6aff5..4f5267f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -8,8 +8,8 @@
"active\.memory\.global\.budget" : 67108864,
"active\.stop\.timeout" : 3600,
"active\.suspend\.timeout" : 3600,
- "compiler\.arrayindex" : false,
- "compiler\.external\.field\.pushdown" : false,
+ "compiler\.arrayindex" : true,
+ "compiler\.external\.field\.pushdown" : true,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
"compiler\.indexonly" : true,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson
new file mode 100644
index 0000000..40a764c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.1.regexjson
@@ -0,0 +1,9 @@
+{
+ "logicalPlan": {
+ "operator":"distribute-result",
+ "expressions":"R{.*}",
+ "operatorId":"R{.*}",
+ "execution-mode":"R{.*}",
+ "inputs":"R{.*}"
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
new file mode 100644
index 0000000..1ced420
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.2.adm
@@ -0,0 +1 @@
+{"statementCategory":"query","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
new file mode 100644
index 0000000..83b47f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.3.adm
@@ -0,0 +1 @@
+{"statementCategory":"query","statementParameters":[1,2],"explainOnly":true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
new file mode 100644
index 0000000..63c482a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/compileonly/compileonly.4.adm
@@ -0,0 +1 @@
+{"statementCategory":"update","statementParameters":[1,2]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.2.adm
new file mode 100644
index 0000000..7d37386
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.2.adm
@@ -0,0 +1 @@
+45
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.3.adm
new file mode 100644
index 0000000..aca544d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.3.adm
@@ -0,0 +1 @@
+145
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.4.adm
new file mode 100644
index 0000000..f11c82a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/request-dataverse/request-dataverse.4.adm
@@ -0,0 +1 @@
+9
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.1.adm
new file mode 100644
index 0000000..a10e23e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.1.adm
@@ -0,0 +1 @@
+{"*":"*"}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.2.adm
new file mode 100644
index 0000000..fae3584
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.2.adm
@@ -0,0 +1 @@
+{"name":["c_int8","c_int16","c_int32","c_int64","c_float","c_double","c_string","c_boolean","c_datetime","c_date","c_time","c_duration","c_year_month_duration","c_day_time_month_duration","c_null"],"type":["int8","int16","int32","int64","float","double","string","boolean","datetime","date","time","duration","year-month-duration","day-time-duration","null"]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.3.adm
new file mode 100644
index 0000000..86234be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.3.adm
@@ -0,0 +1 @@
+{"name":["c_number","c_record","c_array","c_multiset"],"type":["int64","object","array","multiset"]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.4.adm
new file mode 100644
index 0000000..5c5ffd0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.4.adm
@@ -0,0 +1 @@
+{"name":["c_number","c_case","c_missing"],"type":["int64","any","any"]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.5.adm
new file mode 100644
index 0000000..48710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/signature/signature.5.adm
@@ -0,0 +1 @@
+{"name":["DataverseName","DatasetName","ViewDetails"],"type":["string","string","any"]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.1.adm
new file mode 100644
index 0000000..32c3a65
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/composite-atomic/composite-atomic.1.adm
@@ -0,0 +1 @@
+{ "_id": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.1.adm
new file mode 100644
index 0000000..b811602
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.1.adm
@@ -0,0 +1,3 @@
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 300, "field4": 4, "field4_notindexed": 400 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.2.adm
new file mode 100644
index 0000000..27ffff6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.2.adm
@@ -0,0 +1,2 @@
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.3.adm
new file mode 100644
index 0000000..ee8a69b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.3.adm
@@ -0,0 +1,3 @@
+{ "field1": 1, "items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ], "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ], "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "items": [ { "field2": 2, "field3": 3, "field3_notindexed": 300 } ], "field4": 4, "field4_notindexed": 400 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.4.adm
new file mode 100644
index 0000000..327735d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.4.adm
@@ -0,0 +1,2 @@
+{ "field1": 1, "outer_items": [ { "field2_notindexed": 2, "inner_items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ] } ], "field4": 4 }
+{ "field1": 1, "outer_items": [ { "field2_notindexed": 2, "inner_items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ] } ], "field4": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.5.adm
new file mode 100644
index 0000000..e6bba45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.5.adm
@@ -0,0 +1,7 @@
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 100, "field2": 200, "field3": 300, "field4": 400 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.6.adm
new file mode 100644
index 0000000..34afde6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.6.adm
@@ -0,0 +1,4 @@
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field3_notindexed": 3, "field4": 4, "field4_notindexed": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.7.adm
new file mode 100644
index 0000000..e6bba45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.7.adm
@@ -0,0 +1,7 @@
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 1, "field2": 2, "field3": 3, "field4": 4 }
+{ "field1": 100, "field2": 200, "field3": 300, "field4": 400 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.8.adm
new file mode 100644
index 0000000..327735d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/composite-index-queries/composite-index-queries.8.adm
@@ -0,0 +1,2 @@
+{ "field1": 1, "outer_items": [ { "field2_notindexed": 2, "inner_items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ] } ], "field4": 4 }
+{ "field1": 1, "outer_items": [ { "field2_notindexed": 2, "inner_items": [ { "field2": 2, "field3": 3, "field3_notindexed": 3 } ] } ], "field4": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..e60c921
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.1.adm
@@ -0,0 +1,4 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "2" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "2" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..e60c921
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.2.adm
@@ -0,0 +1,4 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "2" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "2" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.3.adm
new file mode 100644
index 0000000..7533447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.3.adm
@@ -0,0 +1,2 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.4.adm
new file mode 100644
index 0000000..20b63fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.4.adm
@@ -0,0 +1,8 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id_a": "2", "business_id_b": "B" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id_a": "2", "business_id_b": "C" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id_a": "3", "business_id_b": "B" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id_a": "3", "business_id_b": "C" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id_a": "2", "business_id_b": "B" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id_a": "2", "business_id_b": "C" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id_a": "3", "business_id_b": "B" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id_a": "3", "business_id_b": "C" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.5.adm
new file mode 100644
index 0000000..7533447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-1/use-case-1.5.adm
@@ -0,0 +1,2 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..e60c921
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-2/use-case-2.1.adm
@@ -0,0 +1,4 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "2" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "2" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..7533447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-2/use-case-2.2.adm
@@ -0,0 +1,2 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..e60c921
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-3/use-case-3.1.adm
@@ -0,0 +1,4 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "2" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "2" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..7533447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-3/use-case-3.2.adm
@@ -0,0 +1,2 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..e60c921
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.1.adm
@@ -0,0 +1,4 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "2" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "2" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..e60c921
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.2.adm
@@ -0,0 +1,4 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "2" }
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "2" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..7533447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.3.adm
@@ -0,0 +1,2 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id": "3" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id": "3" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.4.adm
new file mode 100644
index 0000000..361e29b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-quantified-queries/use-case-4/use-case-4.4.adm
@@ -0,0 +1,2 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "business_id_a": "3", "business_id_b": "C" }
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "business_id_a": "3", "business_id_b": "C" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/loj-subquery/loj-subquery.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/loj-subquery/loj-subquery.1.adm
new file mode 100644
index 0000000..25abe47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/loj-subquery/loj-subquery.1.adm
@@ -0,0 +1,5 @@
+{ "val": null }
+{ "val": 1 }
+{ "val": 1 }
+{ "val": 2 }
+{ "val": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.1.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.1.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.3.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.4.adm
new file mode 100644
index 0000000..694c9d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/use-case-1.4.adm
@@ -0,0 +1 @@
+{ "$1": 107 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.4.adm
new file mode 100644
index 0000000..694c9d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.4.adm
@@ -0,0 +1 @@
+{ "$1": 107 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.5.adm
new file mode 100644
index 0000000..e593a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.5.adm
@@ -0,0 +1 @@
+{ "$1": 125 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.5.adm
new file mode 100644
index 0000000..e593a03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.5.adm
@@ -0,0 +1 @@
+{ "$1": 125 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.1.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.1.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.3.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.4.adm
new file mode 100644
index 0000000..694c9d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/with-open-index.4.adm
@@ -0,0 +1 @@
+{ "$1": 107 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/complex-structures/complex-structures-01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/complex-structures/complex-structures-01.1.adm
new file mode 100644
index 0000000..7618b68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/complex-structures/complex-structures-01.1.adm
@@ -0,0 +1,8 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "open_field_1", "open_field_2" ] ], "ProjectList": [ [ "open_field_3a" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "closed_field_1", "open_field_2" ] ], "ProjectList": [ [ "open_field_3a" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "open_field_1", "open_field_2" ] ], "ProjectList": [ [ "open_field_3b", "open_field_4" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "closed_field_1", "closed_field_2" ] ], "ProjectList": [ [ "open_field_3b", "open_field_4" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "open_field_1", "open_field_2" ], [ "open_field_3c", "open_field_4a" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "closed_field_1", "closed_field_2" ], [ "closed_field_3", "open_field_4a" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "open_field_1", "open_field_2" ], [ "open_field_3c", "open_field_4b" ] ], "ProjectList": [ [ "open_field_5" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "closed_field_1", "closed_field_2" ], [ "closed_field_3", "closed_field_4" ] ], "ProjectList": [ [ "open_field_5" ] ] } ], "SearchKeyType": [ [ "int64" ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm
index a2eea1e..9ba69cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm
@@ -1 +1,4 @@
-{ "SearchKey": [ ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "dates" ] ] } ], "SearchKeyType": [ [ "string" ], [ "string" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "outer_dates_1" ] ], "ProjectList": [ [ "date" ] ] }, [ [ "backup_business_id" ] ] ], "SearchKeyType": [ [ "string" ], [ "string" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "outer_dates_2" ] ], "ProjectList": [ [ "time" ] ] } ], "SearchKeyType": [ [ "string" ], [ "string" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "outer_dates" ], [ "inner_dates" ] ], "ProjectList": [ [ "date" ], [ "time" ] ] }, [ [ "backup_business_id" ] ] ], "SearchKeyType": [ [ "string" ], [ "string", "string" ], [ "string" ] ] }
+{ "SearchKey": [ ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "dates" ] ] } ], "SearchKeyType": [ [ "string" ], [ "string" ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.4.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.4.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/using-feed/using-feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/using-feed/using-feed.1.adm
new file mode 100644
index 0000000..fb7e3a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/using-feed/using-feed.1.adm
@@ -0,0 +1 @@
+{ "$1": 11 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_order/datetime_order.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_order/datetime_order.1.adm
index 21f8208..450872d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_order/datetime_order.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_order/datetime_order.1.adm
@@ -1,11 +1,11 @@
-datetime("-1937-07-07T15:00:00.000Z")
-datetime("-1600-02-29T18:00:00.384Z")
-datetime("-1600-02-29T23:59:59.999Z")
-datetime("-1600-03-01T06:00:00.384Z")
-datetime("2000-02-29T18:59:59.999Z")
-datetime("2000-02-29T23:59:59.999Z")
-datetime("2004-12-31T20:00:00.000Z")
-datetime("2012-01-01T00:00:00.000Z")
-datetime("2012-01-01T00:00:00.000Z")
-datetime("2012-01-01T00:00:00.000Z")
-datetime("2012-04-06T00:00:00.000Z")
+datetime("-1937-07-07T23:00:00.000")
+datetime("-1600-02-29T23:59:59.999")
+datetime("-1600-03-01T00:00:00.384")
+datetime("-1600-03-01T00:00:00.384")
+datetime("2000-02-29T23:59:59.999")
+datetime("2000-03-01T01:59:59.999")
+datetime("2005-01-01T00:00:00.000")
+datetime("2011-12-31T13:00:00.000")
+datetime("2011-12-31T14:00:00.000")
+datetime("2012-01-01T00:00:00.000")
+datetime("2012-04-06T00:00:00.000")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_tzeq/datetime_tzeq.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_tzeq/datetime_tzeq.1.adm
index 167bafb..45c75bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_tzeq/datetime_tzeq.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_tzeq/datetime_tzeq.1.adm
@@ -1 +1 @@
-{ "result1": true, "result2": true }
+{ "result1": false, "result2": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_tzeq/datetime_tzeq.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_tzeq/datetime_tzeq.2.adm
new file mode 100644
index 0000000..ab605ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/datetime_tzeq/datetime_tzeq.2.adm
@@ -0,0 +1 @@
+{ "result1": true, "result2": true, "result3": true, "result4": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/is_distinct_01/is_distinct_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/is_distinct_01/is_distinct_01.1.adm
new file mode 100644
index 0000000..6b0764d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/is_distinct_01/is_distinct_01.1.adm
@@ -0,0 +1,81 @@
+{ "!!==": false, "==": true, "arg1": "MISSING", "arg2": "MISSING" }
+{ "!!==": false, "==": true, "arg1": "NULL", "arg2": "NULL" }
+{ "!!==": false, "==": true, "arg1": 2, "arg2": 2 }
+{ "!!==": false, "==": true, "arg1": 2, "arg2": 2.0 }
+{ "!!==": false, "==": true, "arg1": 2.0, "arg2": 2 }
+{ "!!==": false, "==": true, "arg1": 2.0, "arg2": 2.0 }
+{ "!!==": false, "==": true, "arg1": 2.5, "arg2": 2.5 }
+{ "!!==": false, "==": true, "arg1": "str1", "arg2": "str1" }
+{ "!!==": false, "==": true, "arg1": "str2", "arg2": "str2" }
+{ "!!==": false, "==": true, "arg1": false, "arg2": false }
+{ "!!==": false, "==": true, "arg1": true, "arg2": true }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": 2 }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": 2.0 }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": false }
+{ "!!==": true, "==": false, "arg1": "MISSING", "arg2": true }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": 2 }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": 2.0 }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": false }
+{ "!!==": true, "==": false, "arg1": "NULL", "arg2": true }
+{ "!!==": true, "==": false, "arg1": 2, "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": 2.0, "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": 2, "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": 2.0, "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": 2, "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": 2.0, "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": 2, "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": 2.0, "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": 2, "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": 2.0, "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": 2, "arg2": false }
+{ "!!==": true, "==": false, "arg1": 2.0, "arg2": false }
+{ "!!==": true, "==": false, "arg1": 2, "arg2": true }
+{ "!!==": true, "==": false, "arg1": 2.0, "arg2": true }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": 2 }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": 2.0 }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": false }
+{ "!!==": true, "==": false, "arg1": 2.5, "arg2": true }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": 2 }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": 2.0 }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": false }
+{ "!!==": true, "==": false, "arg1": "str1", "arg2": true }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": 2 }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": 2.0 }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": false }
+{ "!!==": true, "==": false, "arg1": "str2", "arg2": true }
+{ "!!==": true, "==": false, "arg1": false, "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": false, "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": false, "arg2": 2 }
+{ "!!==": true, "==": false, "arg1": false, "arg2": 2.0 }
+{ "!!==": true, "==": false, "arg1": false, "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": false, "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": false, "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": false, "arg2": true }
+{ "!!==": true, "==": false, "arg1": true, "arg2": "MISSING" }
+{ "!!==": true, "==": false, "arg1": true, "arg2": "NULL" }
+{ "!!==": true, "==": false, "arg1": true, "arg2": 2 }
+{ "!!==": true, "==": false, "arg1": true, "arg2": 2.0 }
+{ "!!==": true, "==": false, "arg1": true, "arg2": 2.5 }
+{ "!!==": true, "==": false, "arg1": true, "arg2": "str1" }
+{ "!!==": true, "==": false, "arg1": true, "arg2": "str2" }
+{ "!!==": true, "==": false, "arg1": true, "arg2": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/is_distinct_01/is_distinct_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/is_distinct_01/is_distinct_01.2.adm
new file mode 100644
index 0000000..bd83c66
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/is_distinct_01/is_distinct_01.2.adm
@@ -0,0 +1 @@
+{ "t1": true, "t1_expected": true, "t1_unexpected": null, "t2": true, "t2_expected": true, "t2_unexpected": false, "t3": true, "t3_expected": true, "t3_unexpected": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_01/binary_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_01/binary_01.1.adm
index 7e08562..d5883d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_01/binary_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_01/binary_01.1.adm
@@ -1 +1,20 @@
-[ hex("ABCDEF0123456789"), hex("ABCDEF0123456789"), hex("0A0B0C0D0E0F"), hex("01020304050607080900"), hex(""), hex("ABCDEF0123456789"), hex("D35DB7E39EBBF3DAB07ABB72BA2A296AC75F8218E4973C5CBDB9E64161114D850838F2CA2471850D20195C254134CFBF"), hex(""), hex("41737465726978"), hex("4173746572697801"), hex("41737465726978013C"), hex("41737465726978013C") ]
+{ "g": 0, "i": 0, "actual": hex("ABCDEF0123456789") }
+{ "g": 0, "i": 1, "actual": hex("ABCDEF0123456789") }
+{ "g": 0, "i": 2, "actual": hex("0A0B0C0D0E0F") }
+{ "g": 0, "i": 3, "actual": hex("01020304050607080900") }
+{ "g": 0, "i": 4, "actual": hex("") }
+{ "g": 0, "i": 5, "actual": hex("ABCDEF0123456789") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_01/binary_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_01/binary_01.2.adm
new file mode 100644
index 0000000..4f206b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_01/binary_01.2.adm
@@ -0,0 +1,20 @@
+{ "g": 0, "i": 0, "actual": hex("D35DB7E39EBBF3DAB07ABB72BA2A296AC75F8218E4973C5CBDB9E64161114D850838F2CA2471850D20195C254134CFBF") }
+{ "g": 0, "i": 1, "actual": hex("") }
+{ "g": 0, "i": 2, "actual": hex("41737465726978") }
+{ "g": 0, "i": 3, "actual": hex("4173746572697801") }
+{ "g": 0, "i": 4, "actual": hex("41737465726978013C") }
+{ "g": 0, "i": 5, "actual": hex("41737465726978013C") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_02/binary_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_02/binary_02.1.adm
new file mode 100644
index 0000000..c01ae82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_02/binary_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_02/binary_02.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_02/binary_02.2.adm
new file mode 100644
index 0000000..c01ae82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/binary_02/binary_02.2.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/boolean_01/boolean_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/boolean_01/boolean_01.1.adm
index fd25362..65a40c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/boolean_01/boolean_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/boolean_01/boolean_01.1.adm
@@ -1 +1,42 @@
-{ "boolean1": true, "boolean2": false, "boolean3": false }
+{ "g": 0, "i": 0, "expected": false, "actual": false }
+{ "g": 0, "i": 1, "expected": false, "actual": false }
+{ "g": 0, "i": 2, "expected": false, "actual": false }
+{ "g": 0, "i": 3, "expected": false, "actual": false }
+{ "g": 0, "i": 4, "expected": false, "actual": false }
+{ "g": 0, "i": 5, "expected": false, "actual": false }
+{ "g": 0, "i": 6, "expected": false, "actual": false }
+{ "g": 0, "i": 7, "expected": false, "actual": false }
+{ "g": 0, "i": 8, "expected": false, "actual": false }
+{ "g": 0, "i": 9, "expected": false, "actual": false }
+{ "g": 0, "i": 10, "expected": false, "actual": false }
+{ "g": 1, "i": 0, "expected": true, "actual": true }
+{ "g": 1, "i": 1, "expected": true, "actual": true }
+{ "g": 1, "i": 2, "expected": true, "actual": true }
+{ "g": 1, "i": 3, "expected": true, "actual": true }
+{ "g": 1, "i": 4, "expected": true, "actual": true }
+{ "g": 1, "i": 5, "expected": true, "actual": true }
+{ "g": 1, "i": 6, "expected": true, "actual": true }
+{ "g": 1, "i": 7, "expected": true, "actual": true }
+{ "g": 1, "i": 8, "expected": true, "actual": true }
+{ "g": 1, "i": 9, "expected": true, "actual": true }
+{ "g": 1, "i": 10, "expected": true, "actual": true }
+{ "g": 1, "i": 11, "expected": true, "actual": true }
+{ "g": 1, "i": 12, "expected": true, "actual": true }
+{ "g": 1, "i": 13, "expected": true, "actual": true }
+{ "g": 1, "i": 14, "expected": true, "actual": true }
+{ "g": 1, "i": 15, "expected": true, "actual": true }
+{ "g": 2, "i": 0, "expected": null, "actual": null }
+{ "g": 2, "i": 1, "expected": null, "actual": null }
+{ "g": 2, "i": 2, "expected": null, "actual": null }
+{ "g": 2, "i": 3, "expected": null, "actual": null }
+{ "g": 2, "i": 4, "expected": null, "actual": null }
+{ "g": 2, "i": 5, "expected": null, "actual": null }
+{ "g": 2, "i": 6, "expected": null, "actual": null }
+{ "g": 2, "i": 7, "expected": null, "actual": null }
+{ "g": 2, "i": 8, "expected": null, "actual": null }
+{ "g": 2, "i": 9, "expected": null, "actual": null }
+{ "g": 2, "i": 10, "expected": null, "actual": null }
+{ "g": 2, "i": 11, "expected": null, "actual": null }
+{ "g": 2, "i": 12, "expected": null, "actual": null }
+{ "g": 2, "i": 13, "expected": null, "actual": null }
+{ "g": 3, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/boolean_02/boolean_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/boolean_02/boolean_02.1.adm
new file mode 100644
index 0000000..adb44f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/boolean_02/boolean_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.1.adm
index 527fc91..3b7f63c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.1.adm
@@ -1 +1,29 @@
-{ "date1": date("2010-10-30"), "date2": date("1987-11-19"), "date3": date("-1987-11-19"), "date4": date("0001-12-27"), "date5": date("-1951-12-27"), "date6": date("-2043-11-19"), "date7": date("-1928-03-29"), "date8": date("1928-03-29"), "date9": date("1900-02-28"), "date10": date("2000-02-29"), "date11": date("2000-02-29") }
+{ "g": 0, "i": 0, "actual": date("2010-10-30") }
+{ "g": 0, "i": 1, "actual": date("1987-11-19") }
+{ "g": 0, "i": 2, "actual": date("-1987-11-19") }
+{ "g": 0, "i": 3, "actual": date("0001-12-27") }
+{ "g": 0, "i": 4, "actual": date("-1951-12-27") }
+{ "g": 0, "i": 5, "actual": date("-2043-11-19") }
+{ "g": 0, "i": 6, "actual": date("-1928-03-29") }
+{ "g": 0, "i": 7, "actual": date("1928-03-29") }
+{ "g": 0, "i": 8, "actual": date("1900-02-28") }
+{ "g": 0, "i": 9, "actual": date("2000-02-29") }
+{ "g": 0, "i": 10, "actual": date("2000-02-29") }
+{ "g": 0, "i": 11, "actual": date("2010-10-30") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 1, "i": 13, "expected": null, "actual": null }
+{ "g": 1, "i": 14, "expected": null, "actual": null }
+{ "g": 1, "i": 15, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
new file mode 100644
index 0000000..d1d2f7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_01/date_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": date("2010-10-30") }
+{ "i": 1, "actual": date("2020-12-31") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
new file mode 100644
index 0000000..a7601ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/date_02/date_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.1.adm
index ad50e30..8d2b2d2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.1.adm
@@ -1 +1,31 @@
-{ "datetime1": datetime("2010-10-30T05:05:56.999Z"), "datetime2": datetime("2010-10-30T20:30:56.250Z"), "datetime3": datetime("1987-11-19T09:20:00.200Z"), "datetime4": datetime("1987-11-19T10:50:56.000Z"), "datetime5": datetime("-1987-11-19T16:20:56.099Z"), "datetime6": datetime("-0001-11-19T10:50:56.719Z"), "datetime7": datetime("1951-12-27T12:20:15.000Z"), "datetime8": datetime("2043-11-19T10:50:56.719Z"), "datetime9": datetime("-1928-03-30T00:19:37.374Z"), "datetime10": datetime("-1928-03-29T11:19:37.374Z"), "datetime11": datetime("-1928-03-29T17:49:37.374Z"), "datetime12": datetime("-1928-03-29T11:19:37.374Z"), "datetime13": datetime("-1928-03-29T11:19:37.370Z"), "datetime14": datetime("-1928-02-29T23:19:37.370Z"), "datetime15": datetime("-1928-02-29T23:19:37.370Z") }
+{ "g": 0, "i": 0, "actual": datetime("2010-10-30T10:50:56.999") }
+{ "g": 0, "i": 1, "actual": datetime("2010-10-30T10:30:56.250") }
+{ "g": 0, "i": 2, "actual": datetime("1987-11-19T09:20:00.200") }
+{ "g": 0, "i": 3, "actual": datetime("1987-11-19T10:50:56.000") }
+{ "g": 0, "i": 4, "actual": datetime("-1987-11-19T10:50:56.099") }
+{ "g": 0, "i": 5, "actual": datetime("-0001-11-19T10:50:56.719") }
+{ "g": 0, "i": 6, "actual": datetime("1951-12-27T12:20:15.000") }
+{ "g": 0, "i": 7, "actual": datetime("2043-11-19T10:50:56.719") }
+{ "g": 0, "i": 8, "actual": datetime("-1928-03-29T17:49:37.374") }
+{ "g": 0, "i": 9, "actual": datetime("-1928-03-29T17:49:37.374") }
+{ "g": 0, "i": 10, "actual": datetime("-1928-03-29T17:49:37.374") }
+{ "g": 0, "i": 11, "actual": datetime("-1928-03-29T17:49:37.374") }
+{ "g": 0, "i": 12, "actual": datetime("-1928-03-29T17:49:37.370") }
+{ "g": 0, "i": 13, "actual": datetime("-1928-03-01T05:49:37.370") }
+{ "g": 0, "i": 14, "actual": datetime("-1928-03-01T05:49:37.370") }
+{ "g": 0, "i": 15, "actual": datetime("2020-01-02T00:00:00.000") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 1, "i": 13, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
new file mode 100644
index 0000000..ec1db49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_01/datetime_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": datetime("2010-10-30T10:57:58.000") }
+{ "i": 1, "actual": datetime("2020-12-31T11:58:59.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
new file mode 100644
index 0000000..a7601ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/datetime_02/datetime_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/double_01/double_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/double_01/double_01.1.adm
index 0f1f5df..9d4a13a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/double_01/double_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/double_01/double_01.1.adm
@@ -1 +1,25 @@
-{ "double1": NaN, "double2": INF, "double3": -INF, "double4": -80.2, "double5": -2.056E-29, "double6": -2.056E-299, "double7": -2.056E-299 }
+{ "g": 0, "i": 0, "actual": NaN }
+{ "g": 0, "i": 1, "actual": INF }
+{ "g": 0, "i": 2, "actual": -INF }
+{ "g": 0, "i": 3, "actual": -80.2 }
+{ "g": 0, "i": 4, "actual": -2.056E-29 }
+{ "g": 0, "i": 5, "actual": -2.056E-299 }
+{ "g": 0, "i": 6, "actual": -2.056E-299 }
+{ "g": 0, "i": 7, "actual": 8.0 }
+{ "g": 0, "i": 8, "actual": 16.0 }
+{ "g": 0, "i": 9, "actual": 32.0 }
+{ "g": 0, "i": 10, "actual": 64.0 }
+{ "g": 0, "i": 11, "actual": 2.5 }
+{ "g": 0, "i": 12, "actual": 0.0 }
+{ "g": 0, "i": 13, "actual": 1.0 }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/double_02/double_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/double_02/double_02.1.adm
new file mode 100644
index 0000000..4d80977
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/double_02/double_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.1.adm
index bba1507..b47b786 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.1.adm
@@ -1 +1,31 @@
-{ "duration1": duration("P30Y10M25DT13H12M50S"), "duration2": duration("P25DT13H12M50S"), "duration3": duration("PT13H12M50S"), "duration4": duration("P30YT12M"), "duration5": duration("PT13H"), "duration6": duration("-P30Y10M25DT13H12M50S"), "duration7": duration("-P25DT13H12M50S"), "duration8": duration("-PT13H50S"), "duration9": duration("P120D"), "duration10": duration("-P2Y4M"), "duration11": duration("PT30M30.937S"), "duration12": duration("P301Y3M72DT13H46M2.435S"), "duration13": duration("P301Y3M72DT13H46M2.435S") }
+{ "g": 0, "i": 0, "actual": duration("P30Y10M25DT13H12M50S") }
+{ "g": 0, "i": 1, "actual": duration("P25DT13H12M50S") }
+{ "g": 0, "i": 2, "actual": duration("PT13H12M50S") }
+{ "g": 0, "i": 3, "actual": duration("P30YT12M") }
+{ "g": 0, "i": 4, "actual": duration("PT13H") }
+{ "g": 0, "i": 5, "actual": duration("-P30Y10M25DT13H12M50S") }
+{ "g": 0, "i": 6, "actual": duration("-P25DT13H12M50S") }
+{ "g": 0, "i": 7, "actual": duration("-PT13H50S") }
+{ "g": 0, "i": 8, "actual": duration("P120D") }
+{ "g": 0, "i": 9, "actual": duration("-P2Y4M") }
+{ "g": 0, "i": 10, "actual": duration("PT30M30.937S") }
+{ "g": 0, "i": 11, "actual": duration("P301Y3M72DT13H46M2.435S") }
+{ "g": 0, "i": 12, "actual": duration("P301Y3M72DT13H46M2.435S") }
+{ "g": 0, "i": 13, "actual": duration("P30Y10M") }
+{ "g": 0, "i": 14, "actual": duration("P25DT13H12M50S") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 1, "i": 13, "expected": null, "actual": null }
+{ "g": 1, "i": 14, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.2.adm
new file mode 100644
index 0000000..74c4b24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.2.adm
@@ -0,0 +1,24 @@
+{ "g": 0, "i": 0, "actual": year-month-duration("P30Y10M") }
+{ "g": 0, "i": 1, "actual": year-month-duration("P30Y") }
+{ "g": 0, "i": 2, "actual": year-month-duration("-P30Y10M") }
+{ "g": 0, "i": 3, "actual": year-month-duration("-P2Y4M") }
+{ "g": 0, "i": 4, "actual": year-month-duration("P301Y3M") }
+{ "g": 0, "i": 5, "actual": year-month-duration("P301Y3M") }
+{ "g": 0, "i": 6, "actual": year-month-duration("P301Y4M") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 1, "i": 13, "expected": null, "actual": null }
+{ "g": 1, "i": 14, "expected": null, "actual": null }
+{ "g": 1, "i": 15, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.3.adm
new file mode 100644
index 0000000..2f93b82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_01/duration_01.3.adm
@@ -0,0 +1,26 @@
+{ "g": 0, "i": 0, "actual": day-time-duration("P25DT13H12M50S") }
+{ "g": 0, "i": 1, "actual": day-time-duration("PT13H12M50S") }
+{ "g": 0, "i": 2, "actual": day-time-duration("PT13H") }
+{ "g": 0, "i": 3, "actual": day-time-duration("-P25DT13H12M50S") }
+{ "g": 0, "i": 4, "actual": day-time-duration("-PT13H50S") }
+{ "g": 0, "i": 5, "actual": day-time-duration("P120D") }
+{ "g": 0, "i": 6, "actual": day-time-duration("PT30M30.937S") }
+{ "g": 0, "i": 7, "actual": day-time-duration("PT14H") }
+{ "g": 0, "i": 8, "actual": day-time-duration("P3DT4H5M6S") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 1, "i": 13, "expected": null, "actual": null }
+{ "g": 1, "i": 14, "expected": null, "actual": null }
+{ "g": 1, "i": 15, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.1.adm
index f9e89d9..a7601ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.1.adm
@@ -1 +1 @@
-{ "duration1": year-month-duration("P30Y10M"), "duration2": day-time-duration("P25DT13H12M50S"), "duration3": day-time-duration("PT13H12M50S"), "duration4": year-month-duration("P30Y"), "duration5": day-time-duration("PT13H"), "duration6": year-month-duration("-P30Y10M"), "duration7": day-time-duration("-P25DT13H12M50S"), "duration8": day-time-duration("-PT13H50S"), "duration9": day-time-duration("P120D"), "duration10": year-month-duration("-P2Y4M"), "duration11": day-time-duration("PT30M30.937S"), "duration12": year-month-duration("P301Y3M"), "duration13": year-month-duration("P301Y3M") }
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.2.adm
new file mode 100644
index 0000000..a7601ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.2.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.3.adm
new file mode 100644
index 0000000..a7601ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.3.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/float_01/float_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/float_01/float_01.1.adm
index 722697d..25f84ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/float_01/float_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/float_01/float_01.1.adm
@@ -1 +1,24 @@
-{ "float1": NaN, "float2": INF, "float3": -INF, "float4": -80.2, "float5": -2.056E-29, "float6": -2.056E-29 }
+{ "g": 0, "i": 0, "actual": NaN }
+{ "g": 0, "i": 1, "actual": INF }
+{ "g": 0, "i": 2, "actual": -INF }
+{ "g": 0, "i": 3, "actual": -80.2 }
+{ "g": 0, "i": 4, "actual": -2.056E-29 }
+{ "g": 0, "i": 5, "actual": -2.056E-29 }
+{ "g": 0, "i": 6, "actual": 8.0 }
+{ "g": 0, "i": 7, "actual": 16.0 }
+{ "g": 0, "i": 8, "actual": 32.0 }
+{ "g": 0, "i": 9, "actual": 64.0 }
+{ "g": 0, "i": 10, "actual": 2.5 }
+{ "g": 0, "i": 11, "actual": 0.0 }
+{ "g": 0, "i": 12, "actual": 1.0 }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/float_02/float_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/float_02/float_02.1.adm
new file mode 100644
index 0000000..4d80977
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/float_02/float_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.1.adm
index 7945209..24ca59d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.1.adm
@@ -1 +1,37 @@
-{ "int8": 80, "int16": 160, "int32": 320, "int64": 640, "int8_2": -80, "int16_2": -160, "int32_2": -320, "int64_2": -640, "int64_min": -9223372036854775808, "int8_3": 80, "int16_3": 160, "int32_3": 320, "int64_3": 640 }
+{ "g": 0, "i": 0, "actual": 80 }
+{ "g": 0, "i": 1, "actual": -80 }
+{ "g": 0, "i": 2, "actual": 80 }
+{ "g": 0, "i": 3, "actual": 16 }
+{ "g": 0, "i": 4, "actual": 32 }
+{ "g": 0, "i": 5, "actual": 64 }
+{ "g": 0, "i": 6, "actual": 1 }
+{ "g": 0, "i": 7, "actual": 2 }
+{ "g": 0, "i": 8, "actual": 0 }
+{ "g": 0, "i": 9, "actual": 1 }
+{ "g": 0, "i": 10, "actual": 80 }
+{ "g": 0, "i": 11, "actual": 80 }
+{ "g": 0, "i": 12, "actual": 80 }
+{ "g": 0, "i": 13, "actual": -80 }
+{ "g": 0, "i": 14, "actual": -80 }
+{ "g": 0, "i": 15, "actual": -80 }
+{ "g": 0, "i": 16, "actual": 81 }
+{ "g": 0, "i": 17, "actual": -81 }
+{ "g": 0, "i": 18, "actual": 127 }
+{ "g": 0, "i": 19, "actual": 127 }
+{ "g": 0, "i": 20, "actual": 127 }
+{ "g": 0, "i": 21, "actual": 127 }
+{ "g": 0, "i": 22, "actual": 127 }
+{ "g": 0, "i": 23, "actual": 127 }
+{ "g": 0, "i": 24, "actual": -128 }
+{ "g": 0, "i": 25, "actual": 0 }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.2.adm
new file mode 100644
index 0000000..ec0cdff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.2.adm
@@ -0,0 +1,36 @@
+{ "g": 0, "i": 0, "actual": 160 }
+{ "g": 0, "i": 1, "actual": -160 }
+{ "g": 0, "i": 2, "actual": 160 }
+{ "g": 0, "i": 3, "actual": 8 }
+{ "g": 0, "i": 4, "actual": 32 }
+{ "g": 0, "i": 5, "actual": 64 }
+{ "g": 0, "i": 6, "actual": 1 }
+{ "g": 0, "i": 7, "actual": 2 }
+{ "g": 0, "i": 8, "actual": 0 }
+{ "g": 0, "i": 9, "actual": 1 }
+{ "g": 0, "i": 10, "actual": 160 }
+{ "g": 0, "i": 11, "actual": 160 }
+{ "g": 0, "i": 12, "actual": 160 }
+{ "g": 0, "i": 13, "actual": -160 }
+{ "g": 0, "i": 14, "actual": -160 }
+{ "g": 0, "i": 15, "actual": -160 }
+{ "g": 0, "i": 16, "actual": 161 }
+{ "g": 0, "i": 17, "actual": -161 }
+{ "g": 0, "i": 18, "actual": 32767 }
+{ "g": 0, "i": 19, "actual": 32767 }
+{ "g": 0, "i": 20, "actual": 32767 }
+{ "g": 0, "i": 21, "actual": 32767 }
+{ "g": 0, "i": 22, "actual": 32767 }
+{ "g": 0, "i": 23, "actual": -32768 }
+{ "g": 0, "i": 24, "actual": 0 }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.3.adm
new file mode 100644
index 0000000..46bf8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.3.adm
@@ -0,0 +1,35 @@
+{ "g": 0, "i": 0, "actual": 320 }
+{ "g": 0, "i": 1, "actual": -320 }
+{ "g": 0, "i": 2, "actual": 320 }
+{ "g": 0, "i": 3, "actual": 8 }
+{ "g": 0, "i": 4, "actual": 16 }
+{ "g": 0, "i": 5, "actual": 64 }
+{ "g": 0, "i": 6, "actual": 1 }
+{ "g": 0, "i": 7, "actual": 2 }
+{ "g": 0, "i": 8, "actual": 0 }
+{ "g": 0, "i": 9, "actual": 1 }
+{ "g": 0, "i": 10, "actual": 320 }
+{ "g": 0, "i": 11, "actual": 320 }
+{ "g": 0, "i": 12, "actual": 320 }
+{ "g": 0, "i": 13, "actual": -320 }
+{ "g": 0, "i": 14, "actual": -320 }
+{ "g": 0, "i": 15, "actual": -320 }
+{ "g": 0, "i": 16, "actual": 321 }
+{ "g": 0, "i": 17, "actual": -321 }
+{ "g": 0, "i": 18, "actual": 2147483647 }
+{ "g": 0, "i": 19, "actual": 2147483647 }
+{ "g": 0, "i": 20, "actual": 2147483647 }
+{ "g": 0, "i": 21, "actual": 2147483647 }
+{ "g": 0, "i": 22, "actual": -2147483648 }
+{ "g": 0, "i": 23, "actual": 0 }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.4.adm
new file mode 100644
index 0000000..b6262aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_01/int_01.4.adm
@@ -0,0 +1,35 @@
+{ "g": 0, "i": 0, "actual": 640 }
+{ "g": 0, "i": 1, "actual": -640 }
+{ "g": 0, "i": 2, "actual": -9223372036854775808 }
+{ "g": 0, "i": 3, "actual": 640 }
+{ "g": 0, "i": 4, "actual": 8 }
+{ "g": 0, "i": 5, "actual": 16 }
+{ "g": 0, "i": 6, "actual": 32 }
+{ "g": 0, "i": 7, "actual": 1 }
+{ "g": 0, "i": 8, "actual": 2 }
+{ "g": 0, "i": 9, "actual": 0 }
+{ "g": 0, "i": 10, "actual": 1 }
+{ "g": 0, "i": 11, "actual": 640 }
+{ "g": 0, "i": 12, "actual": 640 }
+{ "g": 0, "i": 13, "actual": 640 }
+{ "g": 0, "i": 14, "actual": -640 }
+{ "g": 0, "i": 15, "actual": -640 }
+{ "g": 0, "i": 16, "actual": -640 }
+{ "g": 0, "i": 17, "actual": 641 }
+{ "g": 0, "i": 18, "actual": -641 }
+{ "g": 0, "i": 19, "actual": 9223372036854775807 }
+{ "g": 0, "i": 20, "actual": 9223372036854775807 }
+{ "g": 0, "i": 21, "actual": 9223372036854775807 }
+{ "g": 0, "i": 22, "actual": -9223372036854775808 }
+{ "g": 0, "i": 23, "actual": 0 }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.1.adm
new file mode 100644
index 0000000..4d80977
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.2.adm
new file mode 100644
index 0000000..4d80977
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.2.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.3.adm
new file mode 100644
index 0000000..4d80977
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.3.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.4.adm
new file mode 100644
index 0000000..4d80977
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/int_02/int_02.4.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/interval/interval.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/interval/interval.1.adm
index bc00add..02a187b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/interval/interval.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/interval/interval.1.adm
@@ -1 +1 @@
-{ "interval41": interval(date("0001-12-27"), date("0006-01-27")), "interval42": interval(date("0001-12-27"), date("0006-01-27")), "interval43": interval(date("0001-12-27"), date("0006-01-27")), "interval44": interval(date("0001-12-27"), date("0006-01-27")), "interval45": null, "interval46": null, "interval51": interval(time("20:03:20.948Z"), time("20:57:50.886Z")), "interval52": interval(time("20:03:20.948Z"), time("20:57:50.886Z")), "interval53": interval(time("20:03:20.948Z"), time("20:57:50.886Z")), "interval54": interval(time("20:03:20.948Z"), time("20:57:50.886Z")), "interval55": null, "interval56": null, "interval61": interval(datetime("-2043-11-19T15:32:39.293Z"), datetime("-1603-03-12T12:12:38.242Z")), "interval62": interval(datetime("-2043-11-19T15:32:39.293Z"), datetime("-1603-03-12T12:12:38.242Z")), "interval63": interval(datetime("-2043-11-19T15:32:39.293Z"), datetime("-1603-03-12T12:12:38.242Z")), "interval64": interval(datetime("-2043-11-19T15:32:39.293Z"), datetime("-1603-03-12T12:12:38.242Z")), "interval65": null, "interval66": null, "interval71": interval(date("2010-10-30"), date("2012-10-21")), "interval72": null, "interval73": null, "interval74": interval(time("14:04:05.678Z"), time("21:24:25.267Z")), "interval75": null, "interval76": null, "interval77": interval(datetime("-1987-11-18T18:43:57.938Z"), datetime("1999-11-12T19:49:35.948Z")), "interval78": null, "interval79": null }
+{ "interval41": interval(date("0001-12-27"), date("0006-01-27")), "interval42": interval(date("0001-12-27"), date("0006-01-27")), "interval43": interval(date("0001-12-27"), date("0006-01-27")), "interval44": interval(date("0001-12-27"), date("0006-01-27")), "interval45": null, "interval46": null, "interval51": interval(time("20:03:20.948"), time("20:57:50.886")), "interval52": interval(time("20:03:20.948"), time("20:57:50.886")), "interval53": interval(time("20:03:20.948"), time("20:57:50.886")), "interval54": interval(time("20:03:20.948"), time("20:57:50.886")), "interval55": null, "interval56": null, "interval61": interval(datetime("-2043-11-19T15:32:39.293"), datetime("-1603-03-12T12:12:38.242")), "interval62": interval(datetime("-2043-11-19T15:32:39.293"), datetime("-1603-03-12T12:12:38.242")), "interval63": interval(datetime("-2043-11-19T15:32:39.293"), datetime("-1603-03-12T12:12:38.242")), "interval64": interval(datetime("-2043-11-19T15:32:39.293"), datetime("-1603-03-12T12:12:38.242")), "interval65": null, "interval66": null, "interval71": interval(date("2010-10-30"), date("2012-10-21")), "interval72": null, "interval73": null, "interval74": interval(time("03:04:05.678"), time("23:24:25.267")), "interval75": null, "interval76": null, "interval77": interval(datetime("-1987-11-19T02:43:57.938"), datetime("1999-11-12T12:49:35.948")), "interval78": null, "interval79": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/string_01/string_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/string_01/string_01.1.adm
index 068d061..1c04e34 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/string_01/string_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/string_01/string_01.1.adm
@@ -1 +1,22 @@
-{ "string1": "true", "string2": "false\"", "string3": "8", "string4": "16", "string5": "32", "string6": "64", "string7": "1.25", "string8": "2.5" }
+{ "g": 0, "i": 0, "actual": "true" }
+{ "g": 0, "i": 1, "actual": "false\"" }
+{ "g": 0, "i": 2, "actual": "8" }
+{ "g": 0, "i": 3, "actual": "16" }
+{ "g": 0, "i": 4, "actual": "32" }
+{ "g": 0, "i": 5, "actual": "64" }
+{ "g": 0, "i": 6, "actual": "1.25" }
+{ "g": 0, "i": 7, "actual": "2.5" }
+{ "g": 0, "i": 8, "actual": "true" }
+{ "g": 0, "i": 9, "actual": "false" }
+{ "g": 0, "i": 10, "actual": "2020-01-02T03:04:05.000" }
+{ "g": 0, "i": 11, "actual": "2021-01-02" }
+{ "g": 0, "i": 12, "actual": "01:02:03.000" }
+{ "g": 0, "i": 13, "actual": "P30Y10M25DT13H12M50S" }
+{ "g": 0, "i": 14, "actual": "P2Y" }
+{ "g": 0, "i": 15, "actual": "PT4S" }
+{ "g": 0, "i": 16, "actual": "02a199ca-bf58-412e-bd9f-60a0c975a8ac" }
+{ "g": 0, "i": 17, "actual": "0123456789qwertyuiopasdfghjklzxcvbnmQWERTYUIOPLKJHGFDSAZXCVBNM+/" }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/string_02/string_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/string_02/string_02.1.adm
new file mode 100644
index 0000000..746523d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/string_02/string_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.1.adm
index ade4e3b..c272370 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.1.adm
@@ -1 +1,30 @@
-{ "time1": time("05:50:56.200Z"), "time2": time("21:05:56.200Z"), "time3": time("10:50:56.000Z"), "time4": time("10:50:56.200Z"), "time5": time("13:29:59.999Z"), "time6": time("09:15:00.000Z"), "time7": time("13:59:00.019Z"), "time8": time("13:59:00.010Z"), "time9": time("13:59:00.019Z"), "time10": time("13:59:00.010Z"), "time11": time("11:59:00.019Z"), "time12": time("11:59:00.019Z") }
+{ "g": 0, "i": 0, "actual": time("10:50:56.200") }
+{ "g": 0, "i": 1, "actual": time("10:50:56.200") }
+{ "g": 0, "i": 2, "actual": time("10:50:56.000") }
+{ "g": 0, "i": 3, "actual": time("10:50:56.200") }
+{ "g": 0, "i": 4, "actual": time("23:59:59.999") }
+{ "g": 0, "i": 5, "actual": time("00:00:00.000") }
+{ "g": 0, "i": 6, "actual": time("12:59:00.019") }
+{ "g": 0, "i": 7, "actual": time("12:59:00.010") }
+{ "g": 0, "i": 8, "actual": time("12:59:00.019") }
+{ "g": 0, "i": 9, "actual": time("12:59:00.010") }
+{ "g": 0, "i": 10, "actual": time("12:59:00.019") }
+{ "g": 0, "i": 11, "actual": time("12:59:00.019") }
+{ "g": 0, "i": 12, "actual": time("01:02:03.000") }
+{ "g": 1, "i": 0, "expected": null, "actual": null }
+{ "g": 1, "i": 1, "expected": null, "actual": null }
+{ "g": 1, "i": 2, "expected": null, "actual": null }
+{ "g": 1, "i": 3, "expected": null, "actual": null }
+{ "g": 1, "i": 4, "expected": null, "actual": null }
+{ "g": 1, "i": 5, "expected": null, "actual": null }
+{ "g": 1, "i": 6, "expected": null, "actual": null }
+{ "g": 1, "i": 7, "expected": null, "actual": null }
+{ "g": 1, "i": 8, "expected": null, "actual": null }
+{ "g": 1, "i": 9, "expected": null, "actual": null }
+{ "g": 1, "i": 10, "expected": null, "actual": null }
+{ "g": 1, "i": 11, "expected": null, "actual": null }
+{ "g": 1, "i": 12, "expected": null, "actual": null }
+{ "g": 1, "i": 13, "expected": null, "actual": null }
+{ "g": 1, "i": 14, "expected": null, "actual": null }
+{ "g": 1, "i": 15, "expected": null, "actual": null }
+{ "g": 2, "i": 0, "expected": true, "actual": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
new file mode 100644
index 0000000..c415454
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_01/time_01.2.adm
@@ -0,0 +1,2 @@
+{ "i": 0, "actual": time("11:58:59.000") }
+{ "i": 1, "actual": time("10:57:58.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
new file mode 100644
index 0000000..a7601ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/time_02/time_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null, "null_2": null, "null_3": null, "null_4": null, "null_5": null, "null_6": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/uuid/uuid_02/uuid_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/uuid/uuid_02/uuid_02.1.adm
new file mode 100644
index 0000000..746523d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/constructor/uuid/uuid_02/uuid_02.1.adm
@@ -0,0 +1 @@
+{ "null_0": null, "null_1": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dapd/q2-7/q2-7.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dapd/q2-7/q2-7.1.adm
index 5adf5db..4247162 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/dapd/q2-7/q2-7.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dapd/q2-7/q2-7.1.adm
@@ -1,2 +1,2 @@
-{ "sig_id": 14, "total_count": 3, "chapter_breakdown": [ { "chapter_name": "Laguna Beach", "$1": [ { "e": { "Event": { "event_id": 1023, "name": "Art Opening: Southern Orange County Beaches", "organizers": {{ { "name": "Jane Smith" } }}, "sponsoring_sigs": [ { "sig_id": 14, "chapter_name": "San Clemente" }, { "sig_id": 14, "chapter_name": "Laguna Beach" } ], "interest_keywords": {{ "art", "landscape", "nature", "vernissage" }}, "start_time": datetime("2011-02-24T02:00:00.000Z"), "end_time": datetime("2011-02-24T05:00:00.000Z") }, "sponsor": { "sig_id": 14, "chapter_name": "Laguna Beach" } } }, { "e": { "Event": { "event_id": 1042, "name": "Orange County Landmarks", "organizers": {{ { "name": "John Smith" } }}, "sponsoring_sigs": [ { "sig_id": 14, "chapter_name": "Laguna Beach" } ], "interest_keywords": {{ "architecture", "photography" }}, "price": 10.0, "start_time": datetime("2011-02-24T01:00:00.000Z"), "end_time": datetime("2011-02-24T03:00:00.000Z") }, "sponsor": { "sig_id": 14, "chapter_name": "Laguna Beach" } } } ] }, { "chapter_name": "San Clemente", "$1": [ { "e": { "Event": { "event_id": 1023, "name": "Art Opening: Southern Orange County Beaches", "organizers": {{ { "name": "Jane Smith" } }}, "sponsoring_sigs": [ { "sig_id": 14, "chapter_name": "San Clemente" }, { "sig_id": 14, "chapter_name": "Laguna Beach" } ], "interest_keywords": {{ "art", "landscape", "nature", "vernissage" }}, "start_time": datetime("2011-02-24T02:00:00.000Z"), "end_time": datetime("2011-02-24T05:00:00.000Z") }, "sponsor": { "sig_id": 14, "chapter_name": "San Clemente" } } } ] } ] }
-{ "sig_id": 31, "total_count": 1, "chapter_breakdown": [ { "chapter_name": "Huntington Beach", "$1": [ { "e": { "Event": { "event_id": 941, "name": "Intro to Scuba Diving", "organizers": {{ { "name": "Joseph Surfer", "affiliation": "Huntington Beach Scuba Assoc." } }}, "sponsoring_sigs": [ { "sig_id": 31, "chapter_name": "Huntington Beach" } ], "interest_keywords": {{ "scuba", "diving", "aquatics" }}, "price": 40.0, "start_time": datetime("2010-10-16T17:00:00.000Z"), "end_time": datetime("2010-10-16T20:00:00.000Z") }, "sponsor": { "sig_id": 31, "chapter_name": "Huntington Beach" } } } ] } ] }
+{ "sig_id": 14, "total_count": 3, "chapter_breakdown": [ { "chapter_name": "Laguna Beach", "$1": [ { "e": { "Event": { "event_id": 1023, "name": "Art Opening: Southern Orange County Beaches", "organizers": {{ { "name": "Jane Smith" } }}, "sponsoring_sigs": [ { "sig_id": 14, "chapter_name": "San Clemente" }, { "sig_id": 14, "chapter_name": "Laguna Beach" } ], "interest_keywords": {{ "art", "landscape", "nature", "vernissage" }}, "start_time": datetime("2011-02-23T18:00:00.000"), "end_time": datetime("2011-02-23T21:00:00.000") }, "sponsor": { "sig_id": 14, "chapter_name": "Laguna Beach" } } }, { "e": { "Event": { "event_id": 1042, "name": "Orange County Landmarks", "organizers": {{ { "name": "John Smith" } }}, "sponsoring_sigs": [ { "sig_id": 14, "chapter_name": "Laguna Beach" } ], "interest_keywords": {{ "architecture", "photography" }}, "price": 10.0, "start_time": datetime("2011-02-23T17:00:00.000"), "end_time": datetime("2011-02-23T19:00:00.000") }, "sponsor": { "sig_id": 14, "chapter_name": "Laguna Beach" } } } ] }, { "chapter_name": "San Clemente", "$1": [ { "e": { "Event": { "event_id": 1023, "name": "Art Opening: Southern Orange County Beaches", "organizers": {{ { "name": "Jane Smith" } }}, "sponsoring_sigs": [ { "sig_id": 14, "chapter_name": "San Clemente" }, { "sig_id": 14, "chapter_name": "Laguna Beach" } ], "interest_keywords": {{ "art", "landscape", "nature", "vernissage" }}, "start_time": datetime("2011-02-23T18:00:00.000"), "end_time": datetime("2011-02-23T21:00:00.000") }, "sponsor": { "sig_id": 14, "chapter_name": "San Clemente" } } } ] } ] }
+{ "sig_id": 31, "total_count": 1, "chapter_breakdown": [ { "chapter_name": "Huntington Beach", "$1": [ { "e": { "Event": { "event_id": 941, "name": "Intro to Scuba Diving", "organizers": {{ { "name": "Joseph Surfer", "affiliation": "Huntington Beach Scuba Assoc." } }}, "sponsoring_sigs": [ { "sig_id": 31, "chapter_name": "Huntington Beach" } ], "interest_keywords": {{ "scuba", "diving", "aquatics" }}, "price": 40.0, "start_time": datetime("2010-10-16T09:00:00.000"), "end_time": datetime("2010-10-16T12:00:00.000") }, "sponsor": { "sig_id": 31, "chapter_name": "Huntington Beach" } } } ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.adm
index 992045b..8af06a6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.adm
@@ -1,12 +1,12 @@
-{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Closed", "DatasetName": "A_Customers_Closed", "DatatypeDataverseName": "test" }
-{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Default_Closed", "DatasetName": "A_Customers_Default_Closed", "DatatypeDataverseName": "test" }
-{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Open", "DatasetName": "A_Customers_Open", "DatatypeDataverseName": "test" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Closed", "DatasetName": "A_Customers_Closed", "DatatypeDataverseName": "test", "PrimaryKey": [ [ "c_custkey" ] ] }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Default_Closed", "DatasetName": "A_Customers_Default_Closed", "DatatypeDataverseName": "test", "PrimaryKey": [ [ "c_custkey" ] ] }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Open", "DatasetName": "A_Customers_Open", "DatatypeDataverseName": "test", "PrimaryKey": [ [ "c_custkey" ], [ "c_name" ] ] }
{ "en": "Dataset", "DatatypeName": "$d$t$i$B_Orders_Closed", "DatasetName": "B_Orders_Closed", "DatatypeDataverseName": "test" }
{ "en": "Dataset", "DatatypeName": "$d$t$i$B_Orders_Default_Closed", "DatasetName": "B_Orders_Default_Closed", "DatatypeDataverseName": "test" }
{ "en": "Dataset", "DatatypeName": "$d$t$i$B_Orders_Open", "DatasetName": "B_Orders_Open", "DatatypeDataverseName": "test" }
-{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Closed", "DatasetName": "C_Customers_Meta_Closed", "DatatypeDataverseName": "test", "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Closed" }
-{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Default_Closed", "DatasetName": "C_Customers_Meta_Default_Closed", "DatatypeDataverseName": "test", "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Default_Closed" }
-{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Open", "DatasetName": "C_Customers_Meta_Open", "DatatypeDataverseName": "test", "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Open" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Closed", "DatasetName": "C_Customers_Meta_Closed", "DatatypeDataverseName": "test", "PrimaryKey": [ [ "c_custkey" ], [ "c_x" ] ], "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Closed", "KeySourceIndicator": [ 0, 1 ] }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Default_Closed", "DatasetName": "C_Customers_Meta_Default_Closed", "DatatypeDataverseName": "test", "PrimaryKey": [ [ "c_custkey" ] ], "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Default_Closed" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Open", "DatasetName": "C_Customers_Meta_Open", "DatatypeDataverseName": "test", "PrimaryKey": [ [ "c_x" ], [ "c_y" ] ], "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Open", "KeySourceIndicator": [ 1, 1 ] }
{ "en": "Datatype", "DatatypeName": "$d$t$i$A_Customers_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
{ "en": "Datatype", "DatatypeName": "$d$t$i$A_Customers_Default_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
{ "en": "Datatype", "DatatypeName": "$d$t$i$A_Customers_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
@@ -18,4 +18,4 @@
{ "en": "Datatype", "DatatypeName": "$d$t$i$C_Customers_Meta_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
{ "en": "Datatype", "DatatypeName": "$d$t$m$C_Customers_Meta_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_x", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } } }
{ "en": "Datatype", "DatatypeName": "$d$t$m$C_Customers_Meta_Default_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_x", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } } }
-{ "en": "Datatype", "DatatypeName": "$d$t$m$C_Customers_Meta_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "c_x", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$m$C_Customers_Meta_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "c_x", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm
index fcd415b..e8c9dfa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create_dataset_with_filter_on_meta/create_dataset_with_filter_on_meta.1.adm
@@ -1,3 +1,3 @@
{ "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "KVStore", "IndexStructure": "BTREE", "SearchKey": [ [ "key" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 07 18:14:11 PDT 2020", "PendingOp": 0, "SearchKeySourceIndicator": [ 1 ] }
-{ "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_location", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 07 18:14:11 PDT 2020", "PendingOp": 0 }
-{ "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_rating", "IndexStructure": "BTREE", "SearchKey": [ [ "area_code" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 07 18:14:11 PDT 2020", "PendingOp": 0 }
+{ "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_location", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Tue Aug 03 21:39:47 AST 2021", "PendingOp": 0 }
+{ "DataverseName": "KeyVerse", "DatasetName": "KVStore", "IndexName": "s_rating", "IndexStructure": "BTREE", "SearchKey": [ [ "area_code" ] ], "IsPrimary": false, "Timestamp": "Tue Aug 03 21:39:47 AST 2021", "PendingOp": 0, "ExcludeUnknownKey": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.004.adm
index ac116183..272c3c8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.004.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.004.adm
@@ -1 +1,2 @@
+{ "values": [ 1 ] }
{ "values": [ 95, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.005.adm
index ac116183..272c3c8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.005.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-bad-fields/index-bad-fields.005.adm
@@ -1 +1,2 @@
+{ "values": [ 1 ] }
{ "values": [ 95, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.003.adm
new file mode 100644
index 0000000..e2aa8c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.003.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, null, 9 ] }
+{ "values": [ null, "1", 1 ] }
+{ "values": [ null, "4", 4 ] }
+{ "values": [ null, "5", 5 ] }
+{ "values": [ null, "8", 8 ] }
+{ "values": [ 2, "2", 2 ] }
+{ "values": [ 3, "3", 3 ] }
+{ "values": [ 6, "6", 6 ] }
+{ "values": [ 7, "7", 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.004.adm
new file mode 100644
index 0000000..cc2dc3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.004.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, null, 9 ] }
+{ "values": [ null, "1.5", 1 ] }
+{ "values": [ null, "4.5", 4 ] }
+{ "values": [ null, "5.5", 5 ] }
+{ "values": [ null, "8.5", 8 ] }
+{ "values": [ 2.0, "2.5", 2 ] }
+{ "values": [ 3.5, "3.5", 3 ] }
+{ "values": [ 6.0, "6.5", 6 ] }
+{ "values": [ 7.5, "7.5", 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.005.adm
new file mode 100644
index 0000000..799cdcc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.005.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 4 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 6, 6 ] }
+{ "values": [ 7, 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.006.adm
new file mode 100644
index 0000000..4ca6341
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.006.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ 0, 7 ] }
+{ "values": [ 1, 1 ] }
+{ "values": [ 1, 2 ] }
+{ "values": [ 1, 3 ] }
+{ "values": [ 1, 4 ] }
+{ "values": [ 1, 6 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.007.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.007.adm
new file mode 100644
index 0000000..b6cff8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.007.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ "1", 1 ] }
+{ "values": [ "1", 3 ] }
+{ "values": [ "1.5", 2 ] }
+{ "values": [ "1.5", 4 ] }
+{ "values": [ "false", 7 ] }
+{ "values": [ "str", 5 ] }
+{ "values": [ "true", 6 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.008.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.008.adm
new file mode 100644
index 0000000..ec983bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.008.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ 0.0, 7 ] }
+{ "values": [ 1.0, 1 ] }
+{ "values": [ 1.0, 3 ] }
+{ "values": [ 1.0, 6 ] }
+{ "values": [ 1.5, 2 ] }
+{ "values": [ 1.5, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.009.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.009.adm
new file mode 100644
index 0000000..e2aa8c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.009.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, null, 9 ] }
+{ "values": [ null, "1", 1 ] }
+{ "values": [ null, "4", 4 ] }
+{ "values": [ null, "5", 5 ] }
+{ "values": [ null, "8", 8 ] }
+{ "values": [ 2, "2", 2 ] }
+{ "values": [ 3, "3", 3 ] }
+{ "values": [ 6, "6", 6 ] }
+{ "values": [ 7, "7", 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.010.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.010.adm
new file mode 100644
index 0000000..cc2dc3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.010.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, null, 9 ] }
+{ "values": [ null, "1.5", 1 ] }
+{ "values": [ null, "4.5", 4 ] }
+{ "values": [ null, "5.5", 5 ] }
+{ "values": [ null, "8.5", 8 ] }
+{ "values": [ 2.0, "2.5", 2 ] }
+{ "values": [ 3.5, "3.5", 3 ] }
+{ "values": [ 6.0, "6.5", 6 ] }
+{ "values": [ 7.5, "7.5", 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.011.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.011.adm
new file mode 100644
index 0000000..799cdcc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.011.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 4 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 6, 6 ] }
+{ "values": [ 7, 7 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.012.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.012.adm
new file mode 100644
index 0000000..4ca6341
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.012.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ 0, 7 ] }
+{ "values": [ 1, 1 ] }
+{ "values": [ 1, 2 ] }
+{ "values": [ 1, 3 ] }
+{ "values": [ 1, 4 ] }
+{ "values": [ 1, 6 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.013.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.013.adm
new file mode 100644
index 0000000..b6cff8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.013.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ "1", 1 ] }
+{ "values": [ "1", 3 ] }
+{ "values": [ "1.5", 2 ] }
+{ "values": [ "1.5", 4 ] }
+{ "values": [ "false", 7 ] }
+{ "values": [ "str", 5 ] }
+{ "values": [ "true", 6 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.014.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.014.adm
new file mode 100644
index 0000000..ec983bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.014.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ 0.0, 7 ] }
+{ "values": [ 1.0, 1 ] }
+{ "values": [ 1.0, 3 ] }
+{ "values": [ 1.0, 6 ] }
+{ "values": [ 1.5, 2 ] }
+{ "values": [ 1.5, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.015.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.015.adm
new file mode 100644
index 0000000..6080d55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.015.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "ds2", "IndexName": "ds2_o_idx2", "IndexStructure": "BTREE", "SearchKey": [ [ "o_s_f" ], [ "o_d_f" ] ], "IsPrimary": false, "Timestamp": "Sun Oct 31 17:55:11 PDT 2021", "PendingOp": 0, "SearchKeyType": [ "double", "string" ], "ExcludeUnknownKey": false, "Cast": { "Default": null } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.016.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.016.adm
new file mode 100644
index 0000000..77ce9ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.016.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "date: { 2019-01-20 }", 4 ] }
+{ "values": [ "date: { 2020-02-20 }", 1 ] }
+{ "values": [ "date: { 2021-11-25 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.017.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.017.adm
new file mode 100644
index 0000000..38633f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.017.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "time: { 01:40:41.001 }", 1 ] }
+{ "values": [ "time: { 14:40:41.001 }", 4 ] }
+{ "values": [ "time: { 22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.018.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.018.adm
new file mode 100644
index 0000000..8b9d5df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.018.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "datetime: { 2019-01-20T14:40:41.001 }", 4 ] }
+{ "values": [ "datetime: { 2020-02-20T01:40:41.001 }", 1 ] }
+{ "values": [ "datetime: { 2021-11-25T22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.019.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.019.adm
new file mode 100644
index 0000000..77ce9ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.019.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "date: { 2019-01-20 }", 4 ] }
+{ "values": [ "date: { 2020-02-20 }", 1 ] }
+{ "values": [ "date: { 2021-11-25 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.020.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.020.adm
new file mode 100644
index 0000000..38633f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.020.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "time: { 01:40:41.001 }", 1 ] }
+{ "values": [ "time: { 14:40:41.001 }", 4 ] }
+{ "values": [ "time: { 22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.021.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.021.adm
new file mode 100644
index 0000000..8b9d5df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.021.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "datetime: { 2019-01-20T14:40:41.001 }", 4 ] }
+{ "values": [ "datetime: { 2020-02-20T01:40:41.001 }", 1 ] }
+{ "values": [ "datetime: { 2021-11-25T22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.022.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.022.adm
new file mode 100644
index 0000000..fe93409
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.022.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 2 ] }
+{ "values": [ null, 3 ] }
+{ "values": [ null, 4 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.023.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.023.adm
new file mode 100644
index 0000000..77ce9ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.023.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "date: { 2019-01-20 }", 4 ] }
+{ "values": [ "date: { 2020-02-20 }", 1 ] }
+{ "values": [ "date: { 2021-11-25 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.024.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.024.adm
new file mode 100644
index 0000000..38633f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.024.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "time: { 01:40:41.001 }", 1 ] }
+{ "values": [ "time: { 14:40:41.001 }", 4 ] }
+{ "values": [ "time: { 22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.025.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.025.adm
new file mode 100644
index 0000000..8b9d5df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.025.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "datetime: { 2019-01-20T14:40:41.001 }", 4 ] }
+{ "values": [ "datetime: { 2020-02-20T01:40:41.001 }", 1 ] }
+{ "values": [ "datetime: { 2021-11-25T22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.026.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.026.adm
new file mode 100644
index 0000000..77ce9ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.026.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "date: { 2019-01-20 }", 4 ] }
+{ "values": [ "date: { 2020-02-20 }", 1 ] }
+{ "values": [ "date: { 2021-11-25 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.027.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.027.adm
new file mode 100644
index 0000000..38633f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.027.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "time: { 01:40:41.001 }", 1 ] }
+{ "values": [ "time: { 14:40:41.001 }", 4 ] }
+{ "values": [ "time: { 22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.028.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.028.adm
new file mode 100644
index 0000000..8b9d5df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.028.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 3 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "datetime: { 2019-01-20T14:40:41.001 }", 4 ] }
+{ "values": [ "datetime: { 2020-02-20T01:40:41.001 }", 1 ] }
+{ "values": [ "datetime: { 2021-11-25T22:12:51.999 }", 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.029.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.029.adm
new file mode 100644
index 0000000..fe93409
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.029.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 2 ] }
+{ "values": [ null, 3 ] }
+{ "values": [ null, 4 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.030.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.030.adm
new file mode 100644
index 0000000..0101b1d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.030.adm
@@ -0,0 +1,7 @@
+{ "IndexName": "ds3_o_idx_f_d", "Cast": { "Default": null } }
+{ "IndexName": "ds3_o_idx_f_d_fmt", "Cast": { "Default": null, "DataFormat": [ null, "MM/DD/YYYY", null ] } }
+{ "IndexName": "ds3_o_idx_f_dt", "Cast": { "Default": null } }
+{ "IndexName": "ds3_o_idx_f_dt_fmt", "Cast": { "Default": null, "DataFormat": [ "MM/DD/YYYY hh:mm:ss.nnna", null, null ] } }
+{ "IndexName": "ds3_o_idx_f_t", "Cast": { "Default": null } }
+{ "IndexName": "ds3_o_idx_f_t_fmt", "Cast": { "Default": null, "DataFormat": [ null, null, "hh:mm:ss.nnna" ] } }
+{ "IndexName": "ds3_o_idx_invalid_fmt", "Cast": { "Default": null, "DataFormat": [ null, "invalid_format", null ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.031.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.031.adm
new file mode 100644
index 0000000..a8f3fda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.031.adm
@@ -0,0 +1,7 @@
+{ "IndexName": "ds4_o_idx_f_d", "Cast": { "Default": null } }
+{ "IndexName": "ds4_o_idx_f_d_fmt", "Cast": { "Default": null, "DataFormat": [ null, "MM/DD/YYYY", null ] } }
+{ "IndexName": "ds4_o_idx_f_dt", "Cast": { "Default": null } }
+{ "IndexName": "ds4_o_idx_f_dt_fmt", "Cast": { "Default": null, "DataFormat": [ "MM/DD/YYYY hh:mm:ss.nnna", null, null ] } }
+{ "IndexName": "ds4_o_idx_f_t", "Cast": { "Default": null } }
+{ "IndexName": "ds4_o_idx_f_t_fmt", "Cast": { "Default": null, "DataFormat": [ null, null, "hh:mm:ss.nnna" ] } }
+{ "IndexName": "ds4_o_idx_invalid_fmt", "Cast": { "Default": null, "DataFormat": [ null, "invalid_format", null ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.032.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.032.adm
new file mode 100644
index 0000000..64cd5cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.032.adm
@@ -0,0 +1,6 @@
+{ "values": [ "", 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "5", 5 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.033.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.033.adm
new file mode 100644
index 0000000..64cd5cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.033.adm
@@ -0,0 +1,6 @@
+{ "values": [ "", 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "5", 5 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.034.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.034.adm
new file mode 100644
index 0000000..1674d82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.034.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 1 ] }
+{ "values": [ 0, 6 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
+{ "values": [ 5, 5 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.035.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.035.adm
new file mode 100644
index 0000000..97e4e83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.035.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.036.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.036.adm
new file mode 100644
index 0000000..feca349
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.036.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.037.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.037.adm
new file mode 100644
index 0000000..8d8fe2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.037.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.038.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.038.adm
new file mode 100644
index 0000000..f03683f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.038.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ 1, 1 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.039.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.039.adm
new file mode 100644
index 0000000..12f73b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.039.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "1", 1 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3", 3 ] }
+{ "values": [ "4", 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.040.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.040.adm
new file mode 100644
index 0000000..64cd5cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.040.adm
@@ -0,0 +1,6 @@
+{ "values": [ "", 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "5", 5 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.041.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.041.adm
new file mode 100644
index 0000000..64cd5cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.041.adm
@@ -0,0 +1,6 @@
+{ "values": [ "", 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "5", 5 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.042.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.042.adm
new file mode 100644
index 0000000..1674d82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.042.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 1 ] }
+{ "values": [ 0, 6 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
+{ "values": [ 5, 5 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.043.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.043.adm
new file mode 100644
index 0000000..97e4e83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.043.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.044.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.044.adm
new file mode 100644
index 0000000..feca349
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.044.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3.5", 3 ] }
+{ "values": [ "4", 4 ] }
+{ "values": [ "s", 1 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.045.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.045.adm
new file mode 100644
index 0000000..8d8fe2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.045.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.046.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.046.adm
new file mode 100644
index 0000000..f03683f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.046.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ 1, 1 ] }
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.047.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.047.adm
new file mode 100644
index 0000000..12f73b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.047.adm
@@ -0,0 +1,6 @@
+{ "values": [ null, 5 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ "1", 1 ] }
+{ "values": [ "2", 2 ] }
+{ "values": [ "3", 3 ] }
+{ "values": [ "4", 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.048.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.048.adm
new file mode 100644
index 0000000..086d8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.048.adm
@@ -0,0 +1,3 @@
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.049.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.049.adm
new file mode 100644
index 0000000..086d8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.049.adm
@@ -0,0 +1,3 @@
+{ "values": [ 2, 2 ] }
+{ "values": [ 3, 3 ] }
+{ "values": [ 4, 4 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.050.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.050.adm
new file mode 100644
index 0000000..99eb793
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/index-cast-null/index-cast-null.050.adm
@@ -0,0 +1,18 @@
+{ "IndexName": "idx1", "SearchKey": [ [ "s_f1" ] ] }
+{ "IndexName": "idx10", "SearchKey": [ [ "s_f1" ] ], "SearchKeyType": [ "string" ], "Cast": { "Default": null } }
+{ "IndexName": "idx11", "SearchKey": [ [ "s_f1" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
+{ "IndexName": "idx12", "SearchKey": [ [ "s_f2" ] ] }
+{ "IndexName": "idx13", "SearchKey": [ [ "s_f2" ] ], "SearchKeyType": [ "string" ], "Cast": { "Default": null } }
+{ "IndexName": "idx14", "SearchKey": [ [ "s_f2" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
+{ "IndexName": "idx15", "SearchKey": [ [ "i_f" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
+{ "IndexName": "idx16", "SearchKey": [ [ "i_f" ] ], "SearchKeyType": [ "string" ], "Cast": { "Default": null } }
+{ "IndexName": "idx2", "SearchKey": [ [ "s_f1" ] ], "SearchKeyType": [ "string" ], "Cast": { "Default": null } }
+{ "IndexName": "idx3", "SearchKey": [ [ "s_f1" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
+{ "IndexName": "idx4", "SearchKey": [ [ "s_f2" ] ] }
+{ "IndexName": "idx5", "SearchKey": [ [ "s_f2" ] ], "SearchKeyType": [ "string" ], "Cast": { "Default": null } }
+{ "IndexName": "idx6", "SearchKey": [ [ "s_f2" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
+{ "IndexName": "idx7", "SearchKey": [ [ "i_f" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
+{ "IndexName": "idx8", "SearchKey": [ [ "i_f" ] ], "SearchKeyType": [ "string" ], "Cast": { "Default": null } }
+{ "IndexName": "idx9", "SearchKey": [ [ "s_f1" ] ] }
+{ "IndexName": "idx_exc1", "SearchKey": [ [ "s_f2" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
+{ "IndexName": "idx_exc2", "SearchKey": [ [ "s_f2" ] ], "SearchKeyType": [ "int64" ], "Cast": { "Default": null } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.04.adm
new file mode 100644
index 0000000..ff5b048
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.04.adm
@@ -0,0 +1,3 @@
+{ "values": [ 1, 1 ] }
+{ "values": [ 4, 4 ] }
+{ "values": [ 5, 5 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.05.adm
new file mode 100644
index 0000000..06d2398
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.05.adm
@@ -0,0 +1,5 @@
+{ "values": [ null, "2", 2 ] }
+{ "values": [ 1, "1", 1 ] }
+{ "values": [ 4, null, 4 ] }
+{ "values": [ 5, 5 ] }
+{ "values": [ "3", 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.06.adm
new file mode 100644
index 0000000..27e033c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.06.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 2 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ 1, 1 ] }
+{ "values": [ 3 ] }
+{ "values": [ 4, 4 ] }
+{ "values": [ 5, 5 ] }
+{ "values": [ 7 ] }
+{ "values": [ 9 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.07.adm
new file mode 100644
index 0000000..bdee019
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.07.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, null, 8 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ null, 7 ] }
+{ "values": [ null, "2", 2 ] }
+{ "values": [ 1, "1", 1 ] }
+{ "values": [ 4, null, 4 ] }
+{ "values": [ 5, 5 ] }
+{ "values": [ 9 ] }
+{ "values": [ "3", 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.08.adm
new file mode 100644
index 0000000..27e033c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.08.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, 2 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ null, 8 ] }
+{ "values": [ 1, 1 ] }
+{ "values": [ 3 ] }
+{ "values": [ 4, 4 ] }
+{ "values": [ 5, 5 ] }
+{ "values": [ 7 ] }
+{ "values": [ 9 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.09.adm
new file mode 100644
index 0000000..bdee019
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.09.adm
@@ -0,0 +1,9 @@
+{ "values": [ null, null, 8 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ null, 7 ] }
+{ "values": [ null, "2", 2 ] }
+{ "values": [ 1, "1", 1 ] }
+{ "values": [ 4, null, 4 ] }
+{ "values": [ 5, 5 ] }
+{ "values": [ 9 ] }
+{ "values": [ "3", 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.10.adm
new file mode 100644
index 0000000..e839d15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.10.adm
@@ -0,0 +1 @@
+{ "cnt": 9 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.11.adm
new file mode 100644
index 0000000..69fc309
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/index-unknown-key/index-unknown-key.11.adm
@@ -0,0 +1,7 @@
+{ "DataverseName": "test", "IndexName": "ds1" }
+{ "DataverseName": "test", "IndexName": "exclude_unknown_idx1", "ExcludeUnknownKey": true }
+{ "DataverseName": "test", "IndexName": "exclude_unknown_idx2", "ExcludeUnknownKey": true }
+{ "DataverseName": "test", "IndexName": "idx1", "ExcludeUnknownKey": false }
+{ "DataverseName": "test", "IndexName": "idx2", "ExcludeUnknownKey": false }
+{ "DataverseName": "test", "IndexName": "include_unknown_idx1", "ExcludeUnknownKey": false }
+{ "DataverseName": "test", "IndexName": "include_unknown_idx2", "ExcludeUnknownKey": false }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/result.001.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/azure_blob_storage/auth-methods/anonymous-no-auth-public-access-allowed/result.001.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/csv/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/csv/result.001.adm
new file mode 100644
index 0000000..19d10f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/csv/result.001.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "age": 1 }
+{ "id": 2, "age": 2 }
+{ "id": 3, "age": 3 }
+{ "id": 4, "age": 4 }
+{ "id": 5, "age": 5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/json/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/json/result.001.adm
new file mode 100644
index 0000000..19d10f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/json/result.001.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "age": 1 }
+{ "id": 2, "age": 2 }
+{ "id": 3, "age": 3 }
+{ "id": 4, "age": 4 }
+{ "id": 5, "age": 5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/tsv/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/tsv/result.001.adm
new file mode 100644
index 0000000..19d10f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/byte_order_mark/tsv/result.001.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "age": 1 }
+{ "id": 2, "age": 2 }
+{ "id": 3, "age": 3 }
+{ "id": 4, "age": 4 }
+{ "id": 5, "age": 5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.02.adm
new file mode 100644
index 0000000..18f3275
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.02.adm
@@ -0,0 +1,2 @@
+{ "display_url": "string" }
+{ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm
new file mode 100644
index 0000000..f0fad1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.03.adm
@@ -0,0 +1,28 @@
+distribute result [$$17]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"display_url": $$20}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ order (ASC, $$19)
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20, $$19])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.04.adm
new file mode 100644
index 0000000..18f3275
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.04.adm
@@ -0,0 +1,2 @@
+{ "display_url": "string" }
+{ }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm
new file mode 100644
index 0000000..2a24b09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.05.adm
@@ -0,0 +1,28 @@
+distribute result [$$17]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"display_url": $$20}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$19(ASC) ] |PARTITIONED|
+ order (ASC, $$19)
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20, $$19])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20, $$19] <- [get-item($$p.getField("entities").getField("urls"), 0).getField("display_url"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.06.adm
new file mode 100644
index 0000000..695240b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.06.adm
@@ -0,0 +1 @@
+{ "display_url": [ "string" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm
new file mode 100644
index 0000000..1ed4eed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.07.adm
@@ -0,0 +1,34 @@
+distribute result [$$21]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": $$25}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$25])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ order (ASC, $$24)
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$24])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [array-star($$23).getField("display_url")]
+ -- ASSIGN |PARTITIONED|
+ select (not(is-missing($$23)))
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$23, $$24])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.08.adm
new file mode 100644
index 0000000..695240b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.08.adm
@@ -0,0 +1 @@
+{ "display_url": [ "string" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm
new file mode 100644
index 0000000..2302d35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.09.adm
@@ -0,0 +1,34 @@
+distribute result [$$21]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": $$25}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$25])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$24(ASC) ] |PARTITIONED|
+ order (ASC, $$24)
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$25, $$24])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$25] <- [array-star($$23).getField("display_url")]
+ -- ASSIGN |PARTITIONED|
+ select (not(is-missing($$23)))
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$23, $$24])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$23, $$24] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.10.adm
new file mode 100644
index 0000000..41c14f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.10.adm
@@ -0,0 +1 @@
+{ "display_url": "string" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm
new file mode 100644
index 0000000..a9ee166
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.11.adm
@@ -0,0 +1,32 @@
+distribute result [$$26]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$26])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$26] <- [{"display_url": $$urls.getField("display_url")}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$urls])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ order (ASC, $$29)
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$urls, $$29])
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$urls <- scan-collection($$28)
+ -- UNNEST |PARTITIONED|
+ project ([$$28, $$29])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.12.adm
new file mode 100644
index 0000000..41c14f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.12.adm
@@ -0,0 +1 @@
+{ "display_url": "string" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm
new file mode 100644
index 0000000..a29ef6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.13.adm
@@ -0,0 +1,32 @@
+distribute result [$$26]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$26])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$26] <- [{"display_url": $$urls.getField("display_url")}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$urls])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ order (ASC, $$29)
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$urls, $$29])
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$urls <- scan-collection($$28)
+ -- UNNEST |PARTITIONED|
+ project ([$$28, $$29])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28, $$29] <- [$$p.getField("entities").getField("urls"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]},id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.14.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.14.adm
@@ -0,0 +1 @@
+2
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
new file mode 100644
index 0000000..de214d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.15.adm
@@ -0,0 +1,39 @@
+distribute result [$$46]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$46] <- [agg-sql-sum($$52)]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$52] <- [agg-sql-count(1)]
+ -- AGGREGATE |PARTITIONED|
+ select ($$39)
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$39])
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$39] <- [empty-stream()]
+ -- AGGREGATE |LOCAL|
+ select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+ -- STREAM_SELECT |LOCAL|
+ assign [$$48] <- [$$ht.getField("display_url")]
+ -- ASSIGN |LOCAL|
+ unnest $$ht <- scan-collection($$47)
+ -- UNNEST |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SUBPLAN |PARTITIONED|
+ project ([$$47])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [$$p.getField("entities").getField("urls")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.16.adm
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.16.adm
@@ -0,0 +1 @@
+2
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
new file mode 100644
index 0000000..9f122c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.17.adm
@@ -0,0 +1,39 @@
+distribute result [$$46]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$46] <- [agg-sql-sum($$52)]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$52] <- [agg-sql-count(1)]
+ -- AGGREGATE |PARTITIONED|
+ select ($$39)
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$39])
+ -- STREAM_PROJECT |PARTITIONED|
+ subplan {
+ aggregate [$$39] <- [empty-stream()]
+ -- AGGREGATE |LOCAL|
+ select (not(if-missing-or-null(eq($$48, "string"), FALSE)))
+ -- STREAM_SELECT |LOCAL|
+ assign [$$48] <- [$$ht.getField("display_url")]
+ -- ASSIGN |LOCAL|
+ unnest $$ht <- scan-collection($$47)
+ -- UNNEST |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SUBPLAN |PARTITIONED|
+ project ([$$47])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$47] <- [$$p.getField("entities").getField("urls")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({entities:{urls:[{display_url:any}]}})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.18.adm
new file mode 100644
index 0000000..15eecd2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.18.adm
@@ -0,0 +1,2 @@
+1.1
+1.1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm
new file mode 100644
index 0000000..37849ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.19.adm
@@ -0,0 +1,28 @@
+distribute result [$$18]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$18])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20)
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.20.adm
new file mode 100644
index 0000000..15eecd2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.20.adm
@@ -0,0 +1,2 @@
+1.1
+1.1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm
new file mode 100644
index 0000000..ee3a128
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/array-access-pushdown/array-access-pushdown.21.adm
@@ -0,0 +1,28 @@
+distribute result [$$18]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$18])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$18] <- [get-item(get-item(get-item($$21, 0), 0), 0)]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20)
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21, $$20] <- [$$p.getField("place").getField("bounding_box").getField("coordinates"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({place:{bounding_box:{coordinates:[[[any]]]}},id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.02.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.02.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.3.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.03.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.04.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.04.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.3.adm
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.05.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.4.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.06.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.4.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.06.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm
new file mode 100644
index 0000000..48d64c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.07.adm
@@ -0,0 +1,46 @@
+distribute result [$$28]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$28])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28] <- [{"p1": $$p1, "id": $$30}]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ order (ASC, $$30)
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$p1, $$30])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$30))
+ -- HYBRID_HASH_JOIN [$$31][$$30] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ assign [$$31] <- [$$p1.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
+ project ([$$30])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$30] <- [$$p2.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.4.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.08.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.4.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.08.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm
new file mode 100644
index 0000000..2e7bd2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.09.adm
@@ -0,0 +1,46 @@
+distribute result [$$28]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$28])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$28] <- [{"p1": $$p1, "id": $$30}]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
+ order (ASC, $$30)
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$p1, $$30])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$30))
+ -- HYBRID_HASH_JOIN [$$31][$$30] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ assign [$$31] <- [$$p1.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
+ project ([$$30])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$30] <- [$$p2.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.7.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.10.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.7.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.10.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm
new file mode 100644
index 0000000..8eb02fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.11.adm
@@ -0,0 +1,50 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$29])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$29] <- [{"age": $$34, "name": $$35}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$34, $$35])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
+ order (ASC, $$32)
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$34, $$35, $$32])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$32))
+ -- HYBRID_HASH_JOIN [$$31][$$32] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ project ([$$34, $$31])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ project ([$$35, $$32])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset3
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.7.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.12.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.7.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.12.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm
new file mode 100644
index 0000000..12e1b6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.13.adm
@@ -0,0 +1,50 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$29])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$29] <- [{"age": $$34, "name": $$35}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$34, $$35])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
+ order (ASC, $$32)
+ -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$34, $$35, $$32])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$31, $$32))
+ -- HYBRID_HASH_JOIN [$$31][$$32] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
+ project ([$$34, $$31])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset project ({id:any,age:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
+ project ([$$35, $$32])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset3 project ({name:any,id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.9.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.14.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.9.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.14.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm
new file mode 100644
index 0000000..dc8c103
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.15.adm
@@ -0,0 +1,32 @@
+distribute result [$$17]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21, $$22])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20)
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.9.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.16.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.9.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.16.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm
new file mode 100644
index 0000000..1ffc9a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.17.adm
@@ -0,0 +1,32 @@
+distribute result [$$17]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21, $$22])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20)
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.18.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.18.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm
new file mode 100644
index 0000000..229e53a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.19.adm
@@ -0,0 +1,18 @@
+distribute result [$$31]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$31] <- [agg-sql-sum($$32)]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$32] <- [agg-sql-count(1)]
+ -- AGGREGATE |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.20.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.20.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm
new file mode 100644
index 0000000..6937d90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.21.adm
@@ -0,0 +1,18 @@
+distribute result [$$31]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ aggregate [$$31] <- [agg-sql-sum($$32)]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$32] <- [agg-sql-count(1)]
+ -- AGGREGATE |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 project ({})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm
new file mode 100644
index 0000000..1ffc9a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/field-access-pushdown/field-access-pushdown.22.adm
@@ -0,0 +1,32 @@
+distribute result [$$17]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$17] <- [{"id": $$21, "name": $$22}]
+ -- ASSIGN |PARTITIONED|
+ project ([$$21, $$22])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
+ order (ASC, $$20)
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$19, $$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset4 project ({id:any,user:{name:any,id:any}})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.02.adm
new file mode 100644
index 0000000..1ebe993
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.02.adm
@@ -0,0 +1,6 @@
+[ "1", "2" ]
+[ "3", "4" ]
+[ "5", "6" ]
+"7"
+"8"
+"9"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
new file mode 100644
index 0000000..4a602ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.03.adm
@@ -0,0 +1,30 @@
+distribute result [$$20]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$24, $$25])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ order (ASC, $$23)
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$23])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.04.adm
new file mode 100644
index 0000000..1ebe993
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.04.adm
@@ -0,0 +1,6 @@
+[ "1", "2" ]
+[ "3", "4" ]
+[ "5", "6" ]
+"7"
+"8"
+"9"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
new file mode 100644
index 0000000..56dd2ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/heterogeneous-access-pushdown/heterogeneous-access-pushdown.05.adm
@@ -0,0 +1,30 @@
+distribute result [$$20]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$20])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$20] <- [switch-case(TRUE, is-array($$22), $$24, $$25)]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$24, $$25])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
+ order (ASC, $$23)
+ -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$24, $$25] <- [array-star($$22).getField("text"), $$22.getField("text")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$22, $$23])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22, $$23] <- [$$p.getField("arrayOrObject"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({arrayOrObject:<[{text:any}],{text:any}>,id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/missing-fields/missing-fields.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.1.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/missing-fields/missing-fields.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/missing-fields/missing-fields.3.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/missing-fields/missing-fields.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/multi-file-multi-schema/multi-file-multi-schema.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/multi-file-multi-schema/multi-file-multi-schema.1.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.3.adm
new file mode 100644
index 0000000..8df138d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.3.adm
@@ -0,0 +1,14 @@
+{ "id": 8, "age": 10 }
+{ "id": 9, "age": 20 }
+{ "id": 10, "age": 30 }
+{ "id": 11, "age": 40 }
+{ "id": 12, "age": 50 }
+{ "id": 13, "age": 60 }
+{ "id": 14, "age": 70 }
+{ "id": 15, "age": "10" }
+{ "id": 16, "age": "20" }
+{ "id": 17, "age": "30" }
+{ "id": 18, "age": "40" }
+{ "id": 19, "age": "50" }
+{ "id": 20, "age": "60" }
+{ "id": 21, "age": "70" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.4.adm
new file mode 100644
index 0000000..9256ef5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/multi-file-multi-schema/multi-file-multi-schema.4.adm
@@ -0,0 +1,14 @@
+10
+20
+30
+40
+50
+60
+70
+"10"
+"20"
+"30"
+"40"
+"50"
+"60"
+"70"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/no-files/no-files.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/no-files/no-files.02.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/no-files/no-files.02.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.1.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm
new file mode 100644
index 0000000..365218f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.3.adm
@@ -0,0 +1,24 @@
+distribute result [$$16]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$16])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
+ order (ASC, $$18)
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$16, $$18])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.3.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.4.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.3.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.4.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm
new file mode 100644
index 0000000..4533f63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/object-concat/object-concat.5.adm
@@ -0,0 +1,28 @@
+distribute result [$$15]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$15])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$15] <- [object-concat($$18, $$19)]
+ -- ASSIGN |PARTITIONED|
+ project ([$$18, $$19])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
+ order (ASC, $$17)
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$18, $$19, $$17])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset project ({coordinates:any,id:any,user:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm
new file mode 100644
index 0000000..b020ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.02.adm
@@ -0,0 +1,50 @@
+distribute result [$$48]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$48])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$48] <- [{"$1": $$51}]
+ -- ASSIGN |UNPARTITIONED|
+ aggregate [$$51] <- [agg-global-sql-sum($$53)]
+ -- AGGREGATE |UNPARTITIONED|
+ exchange
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ aggregate [$$53] <- [agg-local-sql-sum($$46)]
+ -- AGGREGATE |PARTITIONED|
+ project ([$$46])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$46] <- [object-length($$p1)]
+ -- ASSIGN |PARTITIONED|
+ project ([$$p1])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ join (eq($$49, $$50))
+ -- HYBRID_HASH_JOIN [$$50][$$49] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$50] |PARTITIONED|
+ assign [$$50] <- [$$p1.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$49] |PARTITIONED|
+ project ([$$49])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$49] <- [$$p2.getField("id")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2 project ({id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.adm
new file mode 100644
index 0000000..47348c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.03.adm
@@ -0,0 +1,22 @@
+distribute result [$$p1]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ distinct ([$$p1])
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$p1)
+ -- STABLE_SORT [$$p1(ASC)] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$p1] |PARTITIONED|
+ select (gt($$p1.getField("id"), 10))
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm
new file mode 100644
index 0000000..d22cd15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.04.adm
@@ -0,0 +1,50 @@
+distribute result [$$65]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$65])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$65] <- [{"text": $$text, "$1": $$68}]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$text := $$76]) decor ([]) {
+ aggregate [$$68] <- [agg-global-sql-sum($$75)]
+ -- AGGREGATE |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SORT_GROUP_BY[$$76] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$76] |PARTITIONED|
+ group by ([$$76 := $$67]) decor ([]) {
+ aggregate [$$75] <- [agg-local-sql-sum(array-distinct($$70))]
+ -- AGGREGATE |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SORT_GROUP_BY[$$67] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ select (eq(lowercase($$67), "string"))
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$70, $$67])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$67] <- [$$ht.getField("text")]
+ -- ASSIGN |PARTITIONED|
+ unnest $$ht <- scan-collection($$70)
+ -- UNNEST |PARTITIONED|
+ project ([$$70])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$70] <- [$$p1.getField("entities").getField("hashtags")]
+ -- ASSIGN |PARTITIONED|
+ select (gt($$p1.getField("id"), 10))
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:any},id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm
new file mode 100644
index 0000000..8ad73f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.05.adm
@@ -0,0 +1,52 @@
+distribute result [$$64]
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$64])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$64] <- [{"text": $$text, "$1": $$67}]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$text := $$74]) decor ([]) {
+ aggregate [$$67] <- [agg-global-sql-sum($$73)]
+ -- AGGREGATE |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SORT_GROUP_BY[$$74] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$74] |PARTITIONED|
+ group by ([$$74 := $$66]) decor ([]) {
+ aggregate [$$73] <- [agg-local-sql-sum(sql-sum($$70))]
+ -- AGGREGATE |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SORT_GROUP_BY[$$66] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ select (eq(lowercase($$66), "string"))
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$70, $$66])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$70, $$66] <- [$$ht.getField("indices"), $$ht.getField("text")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$ht])
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$ht <- scan-collection($$69)
+ -- UNNEST |PARTITIONED|
+ project ([$$69])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$69] <- [$$p1.getField("entities").getField("hashtags")]
+ -- ASSIGN |PARTITIONED|
+ select (gt($$p1.getField("id"), 10))
+ -- STREAM_SELECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{indices:any,text:any}]},id:any})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm
new file mode 100644
index 0000000..cac9db7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.06.adm
@@ -0,0 +1,26 @@
+distribute result [$$21]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |UNPARTITIONED|
+ exchange
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ project ([$$21])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$21] <- [{"display_url": get-item($$22, 0).getField("display_url")}]
+ -- ASSIGN |PARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$22])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$22] <- [$$p.getField("entities").getField("urls")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p] <- test.ParquetDataset1 condition (gt(sql-count($$p.getField("entities").getField("urls")), 10)) limit 10 project ({entities:{urls:any}})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
new file mode 100644
index 0000000..30d5ac3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/pushdown-plans/pushdown-plans.07.adm
@@ -0,0 +1,113 @@
+distribute result [$$94]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$94])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$94] <- [{"uname": $$uname, "cnt": $$96}]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$96(DESC) ] |PARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (topK: 10) (DESC, $$96)
+ -- STABLE_SORT [topK: 10] [$$96(DESC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$uname := $$107]) decor ([]) {
+ aggregate [$$96] <- [agg-sql-sum($$106)]
+ -- AGGREGATE |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SORT_GROUP_BY[$$107] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$107] |PARTITIONED|
+ group by ([$$107 := $$95]) decor ([]) {
+ aggregate [$$106] <- [agg-sql-count(1)]
+ -- AGGREGATE |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- SORT_GROUP_BY[$$95] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$95])
+ -- STREAM_PROJECT |PARTITIONED|
+ select ($$85)
+ -- STREAM_SELECT |PARTITIONED|
+ project ([$$85, $$95])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ group by ([$$105 := $$103]) decor ([$$95]) {
+ aggregate [$$85] <- [non-empty-stream()]
+ -- AGGREGATE |LOCAL|
+ select (not(is-missing($$104)))
+ -- STREAM_SELECT |LOCAL|
+ nested tuple source
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- PRE_CLUSTERED_GROUP_BY[$$103] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$103)
+ -- STABLE_SORT [$$103(ASC)] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$103] |PARTITIONED|
+ project ([$$95, $$104, $$103])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ left outer join (eq($$97, $$81))
+ -- HYBRID_HASH_JOIN [$$97][$$81] |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$97] |PARTITIONED|
+ running-aggregate [$$103] <- [create-query-uid()]
+ -- RUNNING_AGGREGATE |PARTITIONED|
+ project ([$$95, $$97])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$97] <- [$$ht1.getField("text")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$95, $$ht1])
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$ht1 <- scan-collection($$98)
+ -- UNNEST |PARTITIONED|
+ project ([$$98, $$95])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$98, $$95] <- [$$p1.getField("entities").getField("hashtags"), $$p1.getField("user").getField("name")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p1] <- test.ParquetDataset1 project ({entities:{hashtags:[{text:any}]},user:{name:any}})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
+ project ([$$104, $$81])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$104, $$81] <- [TRUE, $$ht2.getField("text")]
+ -- ASSIGN |PARTITIONED|
+ project ([$$ht2])
+ -- STREAM_PROJECT |PARTITIONED|
+ unnest $$ht2 <- scan-collection($$99)
+ -- UNNEST |PARTITIONED|
+ project ([$$99])
+ -- STREAM_PROJECT |PARTITIONED|
+ assign [$$99] <- [$$p2.getField("entities").getField("hashtags")]
+ -- ASSIGN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$p2] <- test.ParquetDataset2 project ({entities:{hashtags:[{text:any}]}})
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-all-fields/select-all-fields.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.1.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-all-fields/select-all-fields.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-all-fields/select-all-fields.3.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-all-fields/select-all-fields.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.3.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.4.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.4.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.5.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/select-count-one-field/select-count-one-field.5.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.1.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.3.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/string-standard-utf8/string-standard-utf8.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.02.adm
new file mode 100644
index 0000000..0be5d98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.02.adm
@@ -0,0 +1,2 @@
+true
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.03.adm
new file mode 100644
index 0000000..0be5d98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.03.adm
@@ -0,0 +1,2 @@
+true
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.04.adm
new file mode 100644
index 0000000..0be5d98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/parquet/type-mismatch/type-mismatch.04.adm
@@ -0,0 +1,2 @@
+true
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.01.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/s3/parquet-anonymous-access/parquet-anonymous-access.01.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.01.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/s3/parquet-temporary-access/parquet-temporary-access.01.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/big_object_pyudf/join.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/big_object_pyudf/join.1.adm
new file mode 100644
index 0000000..dfe2582
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/big_object_pyudf/join.1.adm
@@ -0,0 +1 @@
+{ "len_c_comment": 2972.995, "len_o_comment": 375.386 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.1.adm.template b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.1.adm.template
new file mode 100644
index 0000000..a0cc7aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/big_object_roundtrip_20M/big_object_roundtrip_20M.1.adm.template
@@ -0,0 +1 @@
+{ "id": 1, "name": "Person One", "hobbies": [ "%lorembytes:20971520%" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mixedudfs/getCapital.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mixedudfs/getCapital.1.adm
new file mode 100644
index 0000000..dd2dfbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mixedudfs/getCapital.1.adm
@@ -0,0 +1,6 @@
+{ "natCap": { "country": "China", "capital": "Beijing" }, "sum": 3 }
+{ "natCap": { "country": "England", "capital": "London" }, "sum": 3 }
+{ "natCap": { "country": "India", "capital": "New Delhi" }, "sum": 3 }
+{ "natCap": { "country": "Italy", "capital": "Rome" }, "sum": 3 }
+{ "natCap": { "country": "Jupiter", "capital": "NOT_FOUND" }, "sum": 3 }
+{ "natCap": { "country": "United States", "capital": "Washington D.C." }, "sum": 3 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.13.adm
index 65a7c81..c960344 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.13.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.13.adm
@@ -1,9 +1,9 @@
{ "id": 670301227662491648, "len": 20, "sent": 1 }
{ "id": 670301227553566720, "len": 139, "sent": 0 }
{ "id": 670301227041857536, "len": 112, "sent": 0 }
-{ "id": 670301227037519876, "len": 33, "sent": 0 }
+{ "id": 670301227037519876, "len": 34, "sent": 0 }
{ "id": 670301226987159552, "len": 57, "sent": 0 }
-{ "id": 670301226513391616, "len": 28, "sent": 1 }
+{ "id": 670301226513391616, "len": 29, "sent": 1 }
{ "id": 670301226337202180, "len": 77, "sent": 1 }
{ "id": 670301226190278656, "len": 25, "sent": 0 }
{ "id": 670301225959579648, "len": 112, "sent": 1 }
@@ -15,25 +15,25 @@
{ "id": 670301225162661889, "len": 28, "sent": 1 }
{ "id": 670301224885837824, "len": 63, "sent": 0 }
{ "id": 670301224814698496, "len": 59, "sent": 0 }
-{ "id": 670301224709849090, "len": 33, "sent": 1 }
+{ "id": 670301224709849090, "len": 37, "sent": 1 }
{ "id": 670301224684556288, "len": 21, "sent": 0 }
{ "id": 670301224680480768, "len": 39, "sent": 0 }
{ "id": 670301224348946433, "len": 64, "sent": 1 }
{ "id": 670301224261058560, "len": 61, "sent": 1 }
-{ "id": 670301224231690240, "len": 33, "sent": 0 }
-{ "id": 670301224214794240, "len": 33, "sent": 0 }
+{ "id": 670301224231690240, "len": 34, "sent": 0 }
+{ "id": 670301224214794240, "len": 41, "sent": 0 }
{ "id": 670301223753351168, "len": 105, "sent": 1 }
{ "id": 670301223426367488, "len": 23, "sent": 0 }
-{ "id": 670301223216545792, "len": 31, "sent": 0 }
+{ "id": 670301223216545792, "len": 34, "sent": 0 }
{ "id": 670301223182974976, "len": 34, "sent": 1 }
{ "id": 670301223128535041, "len": 21, "sent": 0 }
{ "id": 670301222759301121, "len": 132, "sent": 0 }
{ "id": 670301222734307329, "len": 110, "sent": 1 }
{ "id": 670301222717419520, "len": 81, "sent": 0 }
{ "id": 670301222318936064, "len": 110, "sent": 1 }
-{ "id": 670301222302150657, "len": 131, "sent": 0 }
+{ "id": 670301222302150657, "len": 135, "sent": 0 }
{ "id": 670301222222602240, "len": 43, "sent": 1 }
-{ "id": 670301222113517568, "len": 27, "sent": 0 }
+{ "id": 670301222113517568, "len": 29, "sent": 0 }
{ "id": 670301221836615680, "len": 44, "sent": 1 }
{ "id": 670301221719310336, "len": 28, "sent": 0 }
{ "id": 670301221442486272, "len": 34, "sent": 0 }
@@ -44,13 +44,13 @@
{ "id": 670301220305821696, "len": 140, "sent": 0 }
{ "id": 670301220247072770, "len": 83, "sent": 1 }
{ "id": 670301220196626432, "len": 36, "sent": 0 }
-{ "id": 670301220079312901, "len": 31, "sent": 1 }
-{ "id": 670301219949305857, "len": 70, "sent": 1 }
+{ "id": 670301220079312901, "len": 32, "sent": 1 }
+{ "id": 670301219949305857, "len": 94, "sent": 1 }
{ "id": 670301219739574273, "len": 131, "sent": 1 }
{ "id": 670301219206877184, "len": 27, "sent": 0 }
{ "id": 670301219139620864, "len": 124, "sent": 0 }
-{ "id": 670301218737123328, "len": 124, "sent": 0 }
-{ "id": 670301218640531458, "len": 31, "sent": 1 }
+{ "id": 670301218737123328, "len": 126, "sent": 0 }
+{ "id": 670301218640531458, "len": 33, "sent": 1 }
{ "id": 670301218598756352, "len": 47, "sent": 0 }
{ "id": 670301218565156865, "len": 44, "sent": 0 }
{ "id": 670301218414206976, "len": 71, "sent": 1 }
@@ -58,14 +58,14 @@
{ "id": 670301218078629888, "len": 9, "sent": 0 }
{ "id": 670301217851990017, "len": 111, "sent": 0 }
{ "id": 670301217793269760, "len": 113, "sent": 0 }
-{ "id": 670301217508036608, "len": 47, "sent": 0 }
+{ "id": 670301217508036608, "len": 55, "sent": 0 }
{ "id": 670301217369657344, "len": 137, "sent": 0 }
-{ "id": 670301217311088641, "len": 28, "sent": 0 }
+{ "id": 670301217311088641, "len": 29, "sent": 0 }
{ "id": 670301217231347712, "len": 123, "sent": 0 }
{ "id": 670301216891473920, "len": 44, "sent": 0 }
{ "id": 670301216874721280, "len": 68, "sent": 0 }
{ "id": 670301216799232000, "len": 50, "sent": 1 }
-{ "id": 670301216669171713, "len": 54, "sent": 0 }
+{ "id": 670301216669171713, "len": 55, "sent": 0 }
{ "id": 670301216493060097, "len": 113, "sent": 1 }
{ "id": 670301216400924676, "len": 35, "sent": 1 }
{ "id": 670301216371552258, "len": 58, "sent": 0 }
@@ -78,21 +78,21 @@
{ "id": 670301214958055424, "len": 58, "sent": 1 }
{ "id": 670301214605733888, "len": 139, "sent": 1 }
{ "id": 670301214509129728, "len": 114, "sent": 1 }
-{ "id": 670301214442041344, "len": 18, "sent": 1 }
+{ "id": 670301214442041344, "len": 19, "sent": 1 }
{ "id": 670301214295392256, "len": 47, "sent": 0 }
-{ "id": 670301213737529344, "len": 9, "sent": 0 }
+{ "id": 670301213737529344, "len": 10, "sent": 0 }
{ "id": 670301213544595457, "len": 63, "sent": 1 }
{ "id": 670301213515235333, "len": 107, "sent": 0 }
{ "id": 670301213464899584, "len": 105, "sent": 1 }
{ "id": 670301213120942080, "len": 39, "sent": 0 }
{ "id": 670301212961603585, "len": 63, "sent": 0 }
-{ "id": 670301212961603584, "len": 20, "sent": 0 }
-{ "id": 670301212856737792, "len": 51, "sent": 0 }
+{ "id": 670301212961603584, "len": 25, "sent": 0 }
+{ "id": 670301212856737792, "len": 55, "sent": 0 }
{ "id": 670301212760117248, "len": 133, "sent": 1 }
{ "id": 670301211808010240, "len": 103, "sent": 0 }
-{ "id": 670301211774468096, "len": 40, "sent": 0 }
+{ "id": 670301211774468096, "len": 41, "sent": 0 }
{ "id": 670301211703144450, "len": 138, "sent": 1 }
-{ "id": 670301211581685761, "len": 25, "sent": 1 }
+{ "id": 670301211581685761, "len": 26, "sent": 1 }
{ "id": 670301211560685568, "len": 12, "sent": 1 }
{ "id": 670301211090751490, "len": 140, "sent": 0 }
{ "id": 670301210654699520, "len": 13, "sent": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
index 8f4dc82..f164f97 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/type_validation/type_validation.1.adm
@@ -1 +1 @@
-"907 9.07 \"907\" 9.07 TRUE \"date\": { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039Z } "
+"907 9.07 \"907\" 9.07 TRUE date: { 2013-01-01 } datetime: { 1989-09-07T12:13:14.039 } "
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-push-socket/feed-push-socket.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-push-socket/feed-push-socket.1.adm
index 6466feb..d43ff11 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-push-socket/feed-push-socket.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-push-socket/feed-push-socket.1.adm
@@ -1,10 +1,10 @@
-{ "tweetid": "1", "tweetid-copy": "1", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstein", "followers_count": 3311368 }, "sender-location": point("42.13,80.43"), "send-time": datetime("2005-12-05T21:06:41.000Z"), "send-time-copy": datetime("2005-12-05T21:06:41.000Z"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " love samsung the plan is amazing" }
-{ "tweetid": "10", "tweetid-copy": "10", "user": { "screen-name": "Rolldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstful", "followers_count": 3311368 }, "sender-location": point("46.94,93.98"), "send-time": datetime("2011-04-07T14:08:46.000Z"), "send-time-copy": datetime("2011-04-07T14:08:46.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is good" }
-{ "tweetid": "2", "tweetid-copy": "2", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "David Eckhardstein", "followers_count": 3311368 }, "sender-location": point("28.86,70.44"), "send-time": datetime("2007-08-15T06:44:17.000Z"), "send-time-copy": datetime("2007-08-15T06:44:17.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint its voice-clarity is mind-blowing" }
-{ "tweetid": "3", "tweetid-copy": "3", "user": { "screen-name": "RollandEckhard#500", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Hetfield", "followers_count": 3311368 }, "sender-location": point("39.84,86.48"), "send-time": datetime("2008-12-24T00:07:04.000Z"), "send-time-copy": datetime("2008-12-24T00:07:04.000Z"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is terrible:(" }
-{ "tweetid": "4", "tweetid-copy": "4", "user": { "screen-name": "RollandEckhardstein#221", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstinz", "followers_count": 3311368 }, "sender-location": point("27.67,87.32"), "send-time": datetime("2007-02-05T16:39:13.000Z"), "send-time-copy": datetime("2007-02-05T16:39:13.000Z"), "referred-topics": {{ "t-mobile", "customer-service" }}, "message-text": " love t-mobile its customer-service is mind-blowing" }
-{ "tweetid": "5", "tweetid-copy": "5", "user": { "screen-name": "RollandEcstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardst", "followers_count": 3311368 }, "sender-location": point("27.3,92.77"), "send-time": datetime("2010-09-12T06:15:28.000Z"), "send-time-copy": datetime("2010-09-12T06:15:28.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile the customization is amazing:)" }
-{ "tweetid": "6", "tweetid-copy": "6", "user": { "screen-name": "Rollkhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Kirk Hammette ", "followers_count": 3311368 }, "sender-location": point("45.62,84.78"), "send-time": datetime("2012-01-23T06:23:13.000Z"), "send-time-copy": datetime("2012-01-23T06:23:13.000Z"), "referred-topics": {{ "iphone", "network" }}, "message-text": " like iphone its network is awesome:)" }
-{ "tweetid": "7", "tweetid-copy": "7", "user": { "screen-name": "andEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland khardstein", "followers_count": 3311368 }, "sender-location": point("44.12,81.46"), "send-time": datetime("2012-02-17T17:30:26.000Z"), "send-time-copy": datetime("2012-02-17T17:30:26.000Z"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " hate t-mobile the network is bad" }
-{ "tweetid": "8", "tweetid-copy": "8", "user": { "screen-name": "Rolltein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Ron Eckhardstein", "followers_count": 3311368 }, "sender-location": point("36.86,90.71"), "send-time": datetime("2009-03-12T13:18:04.000Z"), "send-time-copy": datetime("2009-03-12T13:18:04.000Z"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " dislike at&t its touch-screen is OMG" }
-{ "tweetid": "9", "tweetid-copy": "9", "user": { "screen-name": "Roldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckdstein", "followers_count": 3311368 }, "sender-location": point("29.07,97.05"), "send-time": datetime("2012-08-15T20:19:46.000Z"), "send-time-copy": datetime("2012-08-15T20:19:46.000Z"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " hate verizon its speed is bad" }
+{ "tweetid": "1", "tweetid-copy": "1", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstein", "followers_count": 3311368 }, "sender-location": point("42.13,80.43"), "send-time": datetime("2005-12-05T21:06:41.000"), "send-time-copy": datetime("2005-12-05T21:06:41.000"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " love samsung the plan is amazing" }
+{ "tweetid": "10", "tweetid-copy": "10", "user": { "screen-name": "Rolldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstful", "followers_count": 3311368 }, "sender-location": point("46.94,93.98"), "send-time": datetime("2011-04-07T14:08:46.000"), "send-time-copy": datetime("2011-04-07T14:08:46.000"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is good" }
+{ "tweetid": "2", "tweetid-copy": "2", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "David Eckhardstein", "followers_count": 3311368 }, "sender-location": point("28.86,70.44"), "send-time": datetime("2007-08-15T06:44:17.000"), "send-time-copy": datetime("2007-08-15T06:44:17.000"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint its voice-clarity is mind-blowing" }
+{ "tweetid": "3", "tweetid-copy": "3", "user": { "screen-name": "RollandEckhard#500", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Hetfield", "followers_count": 3311368 }, "sender-location": point("39.84,86.48"), "send-time": datetime("2008-12-24T00:07:04.000"), "send-time-copy": datetime("2008-12-24T00:07:04.000"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is terrible:(" }
+{ "tweetid": "4", "tweetid-copy": "4", "user": { "screen-name": "RollandEckhardstein#221", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstinz", "followers_count": 3311368 }, "sender-location": point("27.67,87.32"), "send-time": datetime("2007-02-05T16:39:13.000"), "send-time-copy": datetime("2007-02-05T16:39:13.000"), "referred-topics": {{ "t-mobile", "customer-service" }}, "message-text": " love t-mobile its customer-service is mind-blowing" }
+{ "tweetid": "5", "tweetid-copy": "5", "user": { "screen-name": "RollandEcstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardst", "followers_count": 3311368 }, "sender-location": point("27.3,92.77"), "send-time": datetime("2010-09-12T06:15:28.000"), "send-time-copy": datetime("2010-09-12T06:15:28.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile the customization is amazing:)" }
+{ "tweetid": "6", "tweetid-copy": "6", "user": { "screen-name": "Rollkhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Kirk Hammette ", "followers_count": 3311368 }, "sender-location": point("45.62,84.78"), "send-time": datetime("2012-01-23T06:23:13.000"), "send-time-copy": datetime("2012-01-23T06:23:13.000"), "referred-topics": {{ "iphone", "network" }}, "message-text": " like iphone its network is awesome:)" }
+{ "tweetid": "7", "tweetid-copy": "7", "user": { "screen-name": "andEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland khardstein", "followers_count": 3311368 }, "sender-location": point("44.12,81.46"), "send-time": datetime("2012-02-17T17:30:26.000"), "send-time-copy": datetime("2012-02-17T17:30:26.000"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " hate t-mobile the network is bad" }
+{ "tweetid": "8", "tweetid-copy": "8", "user": { "screen-name": "Rolltein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Ron Eckhardstein", "followers_count": 3311368 }, "sender-location": point("36.86,90.71"), "send-time": datetime("2009-03-12T13:18:04.000"), "send-time-copy": datetime("2009-03-12T13:18:04.000"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " dislike at&t its touch-screen is OMG" }
+{ "tweetid": "9", "tweetid-copy": "9", "user": { "screen-name": "Roldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckdstein", "followers_count": 3311368 }, "sender-location": point("29.07,97.05"), "send-time": datetime("2012-08-15T20:19:46.000"), "send-time-copy": datetime("2012-08-15T20:19:46.000"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " hate verizon its speed is bad" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.1.adm
index 7a4df99..09abbcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feed-with-filtered-dataset/feed-with-filtered-dataset.1.adm
@@ -1,7 +1,7 @@
-{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000Z") }
-{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
-{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000Z") }
-{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000Z") }
-{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000") }
+{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000") }
+{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000") }
+{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/equality-predicate/equality-predicate.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/equality-predicate/equality-predicate.1.adm
index f79ea81..326d8e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/equality-predicate/equality-predicate.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/equality-predicate/equality-predicate.1.adm
@@ -1 +1 @@
-{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.adm
index ba324ff..bad52d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.adm
@@ -1 +1 @@
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.adm
index b89f147..904fbfd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.adm
@@ -1,3 +1,3 @@
-{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
-{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.adm
index b89f147..904fbfd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.adm
@@ -1,3 +1,3 @@
-{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
-{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.adm
index 1c872bf..a4c7d3e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.adm
@@ -1,3 +1,3 @@
-{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
-{ "message-id": 6, "author-id": 2, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing", "send-time": datetime("2012-06-20T10:10:00.000Z") }
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000") }
+{ "message-id": 6, "author-id": 2, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing", "send-time": datetime("2012-06-20T10:10:00.000") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert/insert.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert/insert.1.adm
index 7a4df99..09abbcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert/insert.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/insert/insert.1.adm
@@ -1,7 +1,7 @@
-{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000Z") }
-{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
-{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000Z") }
-{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000Z") }
-{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000") }
+{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000") }
+{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000") }
+{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree/load-with-secondary-btree.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree/load-with-secondary-btree.1.adm
index ba324ff..bad52d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree/load-with-secondary-btree.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree/load-with-secondary-btree.1.adm
@@ -1 +1 @@
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.adm
index b89f147..904fbfd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.adm
@@ -1,3 +1,3 @@
-{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
-{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.adm
index b89f147..904fbfd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.adm
@@ -1,3 +1,3 @@
-{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
-{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.adm
index 1c872bf..a4c7d3e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.adm
@@ -1,3 +1,3 @@
-{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
-{ "message-id": 6, "author-id": 2, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing", "send-time": datetime("2012-06-20T10:10:00.000Z") }
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000") }
+{ "message-id": 6, "author-id": 2, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing", "send-time": datetime("2012-06-20T10:10:00.000") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.2.adm
index f87754d..8d71652 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.2.adm
@@ -1 +1 @@
-{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
\ No newline at end of file
+{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load/load.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load/load.1.adm
index 7a4df99..09abbcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load/load.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/load/load.1.adm
@@ -1,7 +1,7 @@
-{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000Z") }
-{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
-{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000Z") }
-{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000Z") }
-{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000") }
+{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000") }
+{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000") }
+{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/nested-filter-equality-predicate/equality-predicate.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/nested-filter-equality-predicate/equality-predicate.1.adm
index f79ea81..326d8e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/nested-filter-equality-predicate/equality-predicate.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/nested-filter-equality-predicate/equality-predicate.1.adm
@@ -1 +1 @@
-{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at01/at01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at01/at01.1.adm
index b4e8ba3..65f3b66 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at01/at01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at01/at01.1.adm
@@ -1,10 +1,10 @@
-{ "num": 1, "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z") }
-{ "num": 2, "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z") }
-{ "num": 3, "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z") }
-{ "num": 4, "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000Z") }
-{ "num": 5, "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z") }
-{ "num": 6, "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z") }
-{ "num": 7, "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000Z") }
-{ "num": 8, "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000Z") }
-{ "num": 9, "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000Z") }
-{ "num": 10, "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000Z") }
+{ "num": 1, "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000") }
+{ "num": 2, "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000") }
+{ "num": 3, "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000") }
+{ "num": 4, "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000") }
+{ "num": 5, "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000") }
+{ "num": 6, "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000") }
+{ "num": 7, "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000") }
+{ "num": 8, "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000") }
+{ "num": 9, "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000") }
+{ "num": 10, "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-issue550/query-issue550.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-issue550/query-issue550.1.adm
index f068c4d..338a854 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-issue550/query-issue550.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-issue550/query-issue550.1.adm
@@ -1,2 +1,2 @@
-{ "u": "1a2b", "recs": [ { "r": 1, "uid": "1a2b", "t": datetime("2000-01-01T01:00:00.000Z"), "event": "e1" }, { "r": 2, "uid": "1a2b", "t": datetime("2000-01-01T01:01:00.000Z"), "event": "e2" }, { "r": 5, "uid": "1a2b", "t": datetime("2000-01-01T01:04:00.000Z"), "event": "e1" }, { "r": 6, "uid": "1a2b", "t": datetime("2000-01-01T01:05:00.000Z"), "event": "e4" } ] }
-{ "u": "3c4d", "recs": [ { "r": 3, "uid": "3c4d", "t": datetime("2000-01-01T01:02:00.000Z"), "event": "e1" }, { "r": 4, "uid": "3c4d", "t": datetime("2000-01-01T01:03:00.000Z"), "event": "e3" } ] }
+{ "u": "1a2b", "recs": [ { "r": 1, "uid": "1a2b", "t": datetime("2000-01-01T01:00:00.000"), "event": "e1" }, { "r": 2, "uid": "1a2b", "t": datetime("2000-01-01T01:01:00.000"), "event": "e2" }, { "r": 5, "uid": "1a2b", "t": datetime("2000-01-01T01:04:00.000"), "event": "e1" }, { "r": 6, "uid": "1a2b", "t": datetime("2000-01-01T01:05:00.000"), "event": "e4" } ] }
+{ "u": "3c4d", "recs": [ { "r": 3, "uid": "3c4d", "t": datetime("2000-01-01T01:02:00.000"), "event": "e1" }, { "r": 4, "uid": "3c4d", "t": datetime("2000-01-01T01:03:00.000"), "event": "e3" } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/select-let/select-let.2.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/select-let/select-let.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm
new file mode 100644
index 0000000..e1e3cf1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm
@@ -0,0 +1,2 @@
+{ "cnt": 6, "x": "a" }
+{ "cnt": 6, "x": "b" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/hdfs/parquet/parquet.2.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/hdfs/parquet/parquet.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.1.adm
new file mode 100644
index 0000000..683f403
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.1.adm
@@ -0,0 +1,2 @@
+{ "c": 1 }
+{ "c": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.2.adm
new file mode 100644
index 0000000..683f403
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-join/btree-multiple-join/btree-multiple-join.2.adm
@@ -0,0 +1,2 @@
+{ "c": 1 }
+{ "c": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.03.adm
new file mode 100644
index 0000000..6b56041
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.03.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "x": 1, "y": 2 }
+{ "id": 3, "x": 1, "y": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.04.adm
new file mode 100644
index 0000000..6b56041
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.04.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "x": 1, "y": 2 }
+{ "id": 3, "x": 1, "y": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.05.adm
new file mode 100644
index 0000000..140eea9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.05.adm
@@ -0,0 +1 @@
+{ "id": 1, "x": "1", "y": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.06.adm
new file mode 100644
index 0000000..38e6954
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.06.adm
@@ -0,0 +1 @@
+{ "id": 1, "x": 1, "y": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.07.adm
new file mode 100644
index 0000000..a975f87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.07.adm
@@ -0,0 +1,8 @@
+{ "v4x": 1, "v2x": 1 }
+{ "v4x": 1, "v2x": 1 }
+{ "v4x": 1, "v2x": 1 }
+{ "v4x": 1, "v2x": 1 }
+{ "v4x": 2, "v2x": 2 }
+{ "v4x": 2, "v2x": 2 }
+{ "v4x": 2, "v2x": 2 }
+{ "v4x": 2, "v2x": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.08.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.1.ast
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.08.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.09.adm
new file mode 100644
index 0000000..e2015b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.09.adm
@@ -0,0 +1,8 @@
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 2, "v1x": 2 }
+{ "v4x": 2, "v1x": 2 }
+{ "v4x": 2, "v1x": 2 }
+{ "v4x": 2, "v1x": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.10.adm
new file mode 100644
index 0000000..e2015b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.10.adm
@@ -0,0 +1,8 @@
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 1, "v1x": 1 }
+{ "v4x": 2, "v1x": 2 }
+{ "v4x": 2, "v1x": 2 }
+{ "v4x": 2, "v1x": 2 }
+{ "v4x": 2, "v1x": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.11.adm
new file mode 100644
index 0000000..8d727bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.11.adm
@@ -0,0 +1,4 @@
+{ "v4x": 1, "ds2x": 1 }
+{ "v4x": 1, "ds2x": 1 }
+{ "v4x": 2, "ds2x": 2 }
+{ "v4x": 2, "ds2x": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.12.adm
new file mode 100644
index 0000000..107ba48
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.12.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "f_dt_fmt": datetime("2020-02-20T01:40:41.001") }
+{ "id": 4, "f_dt_fmt": datetime("2019-01-20T14:40:41.001") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.13.adm
new file mode 100644
index 0000000..bac020d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.13.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "f_d_fmt": date("2020-02-20") }
+{ "id": 4, "f_d_fmt": date("2019-01-20") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.14.adm
new file mode 100644
index 0000000..66960b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.14.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "f_t_fmt": time("01:40:41.001") }
+{ "id": 4, "f_t_fmt": time("14:40:41.001") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.15.adm
new file mode 100644
index 0000000..efc9e6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.15.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "f_dt": datetime("2020-02-20T01:40:41.001") }
+{ "id": 4, "f_dt": datetime("2019-01-20T14:40:41.001") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.16.adm
new file mode 100644
index 0000000..c0843a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.16.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "f_d": date("2020-02-20") }
+{ "id": 4, "f_d": date("2019-01-20") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.17.adm
new file mode 100644
index 0000000..bae3de6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.17.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "f_t": time("01:40:41.001") }
+{ "id": 4, "f_t": time("14:40:41.001") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.18.adm
new file mode 100644
index 0000000..448a834
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.18.adm
@@ -0,0 +1,3 @@
+{ "v6f_dt": datetime("2019-01-20T14:40:41.001"), "v5f_dt_fmt": datetime("2019-01-20T14:40:41.001") }
+{ "v6f_dt": datetime("2020-02-20T01:40:41.001"), "v5f_dt_fmt": datetime("2020-02-20T01:40:41.001") }
+{ "v6f_dt": datetime("2021-11-25T22:12:51.999"), "v5f_dt_fmt": datetime("2021-11-25T22:12:51.999") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.19.adm
new file mode 100644
index 0000000..49938d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.19.adm
@@ -0,0 +1,2 @@
+{ "id": 2, "s_f2": "2" }
+{ "id": 3, "s_f2": "3.5" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.20.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.20.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.21.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.21.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.22.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.22.adm
new file mode 100644
index 0000000..78954d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.22.adm
@@ -0,0 +1,2 @@
+{ "id": 2, "s_f2": 2 }
+{ "id": 3, "s_f2": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.23.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.23.adm
new file mode 100644
index 0000000..49938d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/cast-default-null/cast-default-null.23.adm
@@ -0,0 +1,2 @@
+{ "id": 2, "s_f2": "2" }
+{ "id": 3, "s_f2": "3.5" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-filter/intersection.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-filter/intersection.1.adm
index 44cc08a..5e2e080 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-filter/intersection.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-filter/intersection.1.adm
@@ -1 +1 @@
-{ "tweetid": 9, "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": 9, "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection/intersection.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection/intersection.1.adm
index 44cc08a..5e2e080 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection/intersection.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection/intersection.1.adm
@@ -1 +1 @@
-{ "tweetid": 9, "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": 9, "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.3.adm
new file mode 100644
index 0000000..4f947fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.3.adm
@@ -0,0 +1 @@
+{ "cnt": 10000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.4.adm
new file mode 100644
index 0000000..d6aa101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.4.adm
@@ -0,0 +1,25 @@
+{ "x": 0, "y": 1 }
+{ "x": 0, "y": 21 }
+{ "x": 0, "y": 41 }
+{ "x": 0, "y": 61 }
+{ "x": 0, "y": 81 }
+{ "x": 20, "y": 1 }
+{ "x": 20, "y": 21 }
+{ "x": 20, "y": 41 }
+{ "x": 20, "y": 61 }
+{ "x": 20, "y": 81 }
+{ "x": 40, "y": 1 }
+{ "x": 40, "y": 21 }
+{ "x": 40, "y": 41 }
+{ "x": 40, "y": 61 }
+{ "x": 40, "y": 81 }
+{ "x": 60, "y": 1 }
+{ "x": 60, "y": 21 }
+{ "x": 60, "y": 41 }
+{ "x": 60, "y": 61 }
+{ "x": 60, "y": 81 }
+{ "x": 80, "y": 1 }
+{ "x": 80, "y": 21 }
+{ "x": 80, "y": 41 }
+{ "x": 80, "y": 61 }
+{ "x": 80, "y": 81 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.5.adm
new file mode 100644
index 0000000..ffd1e6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/cross-join-01/cross-join-01.5.adm
@@ -0,0 +1,4 @@
+{ "x": 1, "y": 3 }
+{ "x": 1, "y": 4 }
+{ "x": 2, "y": 3 }
+{ "x": 2, "y": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash_join_missing/hash_join_missing.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash_join_missing/hash_join_missing.1.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash_join_missing/hash_join_missing.1.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash_join_missing/hash_join_missing.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash_join_missing/hash_join_missing.2.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/join/hash_join_missing/hash_join_missing.2.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.1.adm
new file mode 100644
index 0000000..a59c66d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.1.adm
@@ -0,0 +1 @@
+{ "nullVal": null, "intVal": 1, "stringVal": "stringVal", "doubleVal": 3.75, "objectVal": { "id": 7, "name": "Jerry", "comment": "😢😢💉💉 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا" }, "arrayVal": [ 1, 2, 3 ], "nullArgVal": null, "unexpectedType": null, "emptyString": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.2.adm
new file mode 100644
index 0000000..533d942
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.2.adm
@@ -0,0 +1,7 @@
+{ "multipleValues": null }
+{ "multipleValues": { "a": 1 } }
+{ "multipleValues": [ 1, 2 ] }
+{ "multipleValues": { "c": "test" } }
+{ "multipleValues": { "a": 3.57 } }
+{ "multipleValues": [ "a", "b" ] }
+{ "multipleValues": { "c": { "d": 1 } } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.3.adm
new file mode 100644
index 0000000..41bc026
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/parse-json-function/parse-json-function.3.adm
@@ -0,0 +1,6 @@
+{ "malformedVal": null }
+{ "malformedVal": [ 1, 2 ] }
+{ "malformedVal": { "c": "test" } }
+{ "malformedVal": null }
+{ "malformedVal": [ "a", "b" ] }
+{ "malformedVal": { "c": { "d": 1 } } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/temporal/temporal.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/temporal/temporal.1.json
index cdfc51d..0fc89b3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/temporal/temporal.1.json
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/json-parser/temporal/temporal.1.json
@@ -1 +1 @@
-{ "datefield": date("-2012-12-12"), "dateFieldPlus": date("0990-01-01"), "timeField": time("23:49:12.390Z"), "timeFieldPlus": time("03:23:12.200Z"), "datetimeField": datetime("2012-12-12T00:00:00.001Z"), "datetimeFieldPlus": datetime("-0013-08-10T22:10:15.398Z") }
\ No newline at end of file
+{ "datefield": date("-2012-12-12"), "dateFieldPlus": date("0990-01-01"), "timeField": time("23:49:12.390"), "timeFieldPlus": time("03:23:12.200"), "datetimeField": datetime("2012-12-12T00:00:00.001"), "datetimeFieldPlus": datetime("-0013-08-10T22:10:15.398") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/ordered-list-constructor_05/ordered-list-constructor_05.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/ordered-list-constructor_05/ordered-list-constructor_05.1.adm
new file mode 100644
index 0000000..466187e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/ordered-list-constructor_05/ordered-list-constructor_05.1.adm
@@ -0,0 +1,2 @@
+{ "r": 1, "a1": [ "foo", "bar", "foobar" ], "a2": [ 1, [ 1 ], null ], "a3": [ [ 2 ], [ 3 ] ], "eq": true }
+{ "r": 2, "a1": [ "foo", "bar", "foobar" ], "a2": [ 2, [ 2 ], null ], "a3": [ [ 2 ], [ 5 ] ], "eq": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm
index b5cba5a..94b5256 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_01/csv_01.1.adm
@@ -1,8 +1,8 @@
-{ "id": 1, "float": 0.89968276, "double": 5.6256, "date-before": "2013-08-07", "date-after": date("2013-08-07"), "time-before": "07:22:35", "time-after": time("07:22:35.000Z"), "datetime-before": "1979-02-25T23:48:27.034", "datetime-after": datetime("1979-02-25T23:48:27.034Z") }
-{ "id": 2, "float": 0.6690524, "double": null, "date-before": "-1923-03-29", "date-after": date("-1923-03-29"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.002", "datetime-after": datetime("-1979-02-25T23:48:27.002Z") }
-{ "id": 3, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-28", "date-after": date("-1923-03-28"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 4, "float": null, "double": 192674.0, "date-before": "-1923-03-27", "date-after": date("-1923-03-27"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 5, "float": 0.57273304, "double": 192674.0, "date-before": null, "date-after": null, "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 6, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-25", "date-after": date("-1923-03-25"), "time-before": null, "time-after": null, "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 7, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-24", "date-after": date("-1923-03-24"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": null, "datetime-after": null }
+{ "id": 1, "float": 0.89968276, "double": 5.6256, "date-before": "2013-08-07", "date-after": date("2013-08-07"), "time-before": "07:22:35", "time-after": time("07:22:35.000"), "datetime-before": "1979-02-25T23:48:27.034", "datetime-after": datetime("1979-02-25T23:48:27.034") }
+{ "id": 2, "float": 0.6690524, "double": null, "date-before": "-1923-03-29", "date-after": date("-1923-03-29"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.002", "datetime-after": datetime("-1979-02-25T23:48:27.002") }
+{ "id": 3, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-28", "date-after": date("-1923-03-28"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 4, "float": null, "double": 192674.0, "date-before": "-1923-03-27", "date-after": date("-1923-03-27"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 5, "float": 0.57273304, "double": 192674.0, "date-before": null, "date-after": null, "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 6, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-25", "date-after": date("-1923-03-25"), "time-before": null, "time-after": null, "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 7, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-24", "date-after": date("-1923-03-24"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": null, "datetime-after": null }
{ "id": 8, "float": null, "double": null, "date-before": null, "date-after": null, "time-before": null, "time-after": null, "datetime-before": null, "datetime-after": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_08/csv_08.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_08/csv_08.1.adm
index b5cba5a..94b5256 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_08/csv_08.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_08/csv_08.1.adm
@@ -1,8 +1,8 @@
-{ "id": 1, "float": 0.89968276, "double": 5.6256, "date-before": "2013-08-07", "date-after": date("2013-08-07"), "time-before": "07:22:35", "time-after": time("07:22:35.000Z"), "datetime-before": "1979-02-25T23:48:27.034", "datetime-after": datetime("1979-02-25T23:48:27.034Z") }
-{ "id": 2, "float": 0.6690524, "double": null, "date-before": "-1923-03-29", "date-after": date("-1923-03-29"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.002", "datetime-after": datetime("-1979-02-25T23:48:27.002Z") }
-{ "id": 3, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-28", "date-after": date("-1923-03-28"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 4, "float": null, "double": 192674.0, "date-before": "-1923-03-27", "date-after": date("-1923-03-27"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 5, "float": 0.57273304, "double": 192674.0, "date-before": null, "date-after": null, "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 6, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-25", "date-after": date("-1923-03-25"), "time-before": null, "time-after": null, "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001Z") }
-{ "id": 7, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-24", "date-after": date("-1923-03-24"), "time-before": "19:33:34", "time-after": time("19:33:34.000Z"), "datetime-before": null, "datetime-after": null }
+{ "id": 1, "float": 0.89968276, "double": 5.6256, "date-before": "2013-08-07", "date-after": date("2013-08-07"), "time-before": "07:22:35", "time-after": time("07:22:35.000"), "datetime-before": "1979-02-25T23:48:27.034", "datetime-after": datetime("1979-02-25T23:48:27.034") }
+{ "id": 2, "float": 0.6690524, "double": null, "date-before": "-1923-03-29", "date-after": date("-1923-03-29"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.002", "datetime-after": datetime("-1979-02-25T23:48:27.002") }
+{ "id": 3, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-28", "date-after": date("-1923-03-28"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 4, "float": null, "double": 192674.0, "date-before": "-1923-03-27", "date-after": date("-1923-03-27"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 5, "float": 0.57273304, "double": 192674.0, "date-before": null, "date-after": null, "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 6, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-25", "date-after": date("-1923-03-25"), "time-before": null, "time-after": null, "datetime-before": "-1979-02-25T23:48:27.001", "datetime-after": datetime("-1979-02-25T23:48:27.001") }
+{ "id": 7, "float": 0.57273304, "double": 192674.0, "date-before": "-1923-03-24", "date-after": date("-1923-03-24"), "time-before": "19:33:34", "time-after": time("19:33:34.000"), "datetime-before": null, "datetime-after": null }
{ "id": 8, "float": null, "double": null, "date-before": null, "date-after": null, "time-before": null, "time-after": null, "datetime-before": null, "datetime-after": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.adm
index bfbb1bd..79666ef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.adm
@@ -1,14 +1,14 @@
-{ "name": "BramHatch", "id": 10, "alias": "Bram", "userSince": datetime("2010-10-16T10:10:00.000Z"), "friendIds": {{ 1, 5, 9 }}, "employment": [ { "organizationName": "physcane", "startDate": date("2007-06-05"), "endDate": date("2011-11-05") } ] }
+{ "name": "BramHatch", "id": 10, "alias": "Bram", "userSince": datetime("2010-10-16T10:10:00.000"), "friendIds": {{ 1, 5, 9 }}, "employment": [ { "organizationName": "physcane", "startDate": date("2007-06-05"), "endDate": date("2011-11-05") } ] }
{ "name": "Chang Ewing", "screenName": "ChangEwing_573", "lang": "en", "friendsCount": 182, "statusesCount": 394, "followersCount": 32136 }
{ "name": "Coline Geyer", "screenName": "ColineGeyer@63", "lang": "en", "friendsCount": 121, "statusesCount": 362, "followersCount": 17159 }
-{ "name": "EmoryUnk", "id": 3, "alias": "Emory", "userSince": datetime("2012-07-10T10:10:00.000Z"), "friendIds": {{ 1, 5, 8, 9 }}, "employment": [ { "organizationName": "geomedia", "startDate": date("2010-06-17"), "endDate": date("2010-01-26") } ] }
-{ "name": "IsbelDull", "id": 2, "alias": "Isbel", "userSince": datetime("2011-01-22T10:10:00.000Z"), "friendIds": {{ 1, 4 }}, "employment": [ { "organizationName": "Hexviafind", "startDate": date("2010-04-27") } ], "nickname": "Izzy" }
-{ "name": "MargaritaStoddard", "id": 1, "alias": "Margarita", "userSince": datetime("2012-08-20T10:10:00.000Z"), "friendIds": {{ 2, 3, 6, 10 }}, "employment": [ { "organizationName": "Codetechno", "startDate": date("2006-08-06") }, { "organizationName": "geomedia", "startDate": date("2010-06-17"), "endDate": date("2010-01-26") } ], "nickname": "Mags", "gender": "F" }
+{ "name": "EmoryUnk", "id": 3, "alias": "Emory", "userSince": datetime("2012-07-10T10:10:00.000"), "friendIds": {{ 1, 5, 8, 9 }}, "employment": [ { "organizationName": "geomedia", "startDate": date("2010-06-17"), "endDate": date("2010-01-26") } ] }
+{ "name": "IsbelDull", "id": 2, "alias": "Isbel", "userSince": datetime("2011-01-22T10:10:00.000"), "friendIds": {{ 1, 4 }}, "employment": [ { "organizationName": "Hexviafind", "startDate": date("2010-04-27") } ], "nickname": "Izzy" }
+{ "name": "MargaritaStoddard", "id": 1, "alias": "Margarita", "userSince": datetime("2012-08-20T10:10:00.000"), "friendIds": {{ 2, 3, 6, 10 }}, "employment": [ { "organizationName": "Codetechno", "startDate": date("2006-08-06") }, { "organizationName": "geomedia", "startDate": date("2010-06-17"), "endDate": date("2010-01-26") } ], "nickname": "Mags", "gender": "F" }
{ "name": "Nathan Giesen", "screenName": "NathanGiesen@211", "lang": "en", "friendsCount": 18, "statusesCount": 473, "followersCount": 49416 }
-{ "name": "NicholasStroh", "id": 4, "alias": "Nicholas", "userSince": datetime("2010-12-27T10:10:00.000Z"), "friendIds": {{ 2 }}, "employment": [ { "organizationName": "Zamcorporation", "startDate": date("2010-06-08") } ] }
+{ "name": "NicholasStroh", "id": 4, "alias": "Nicholas", "userSince": datetime("2010-12-27T10:10:00.000"), "friendIds": {{ 2 }}, "employment": [ { "organizationName": "Zamcorporation", "startDate": date("2010-06-08") } ] }
{ "name": "Nila Milliron", "screenName": "NilaMilliron_tw", "lang": "en", "friendsCount": 445, "statusesCount": 164, "followersCount": 22649 }
-{ "name": "NilaMilliron", "id": 8, "alias": "Nila", "userSince": datetime("2008-01-01T10:10:00.000Z"), "friendIds": {{ 3 }}, "employment": [ { "organizationName": "Plexlane", "startDate": date("2010-02-28") } ] }
-{ "name": "SuzannaTillson", "id": 7, "alias": "Suzanna", "userSince": datetime("2012-08-07T10:10:00.000Z"), "friendIds": {{ 6 }}, "employment": [ { "organizationName": "Labzatron", "startDate": date("2011-04-19") } ] }
-{ "name": "VonKemble", "id": 5, "alias": "Von", "userSince": datetime("2010-01-05T10:10:00.000Z"), "friendIds": {{ 3, 6, 10 }}, "employment": [ { "organizationName": "Kongreen", "startDate": date("2010-11-27") } ] }
-{ "name": "WillisWynne", "id": 6, "alias": "Willis", "userSince": datetime("2005-01-17T10:10:00.000Z"), "friendIds": {{ 1, 3, 7 }}, "employment": [ { "organizationName": "jaydax", "startDate": date("2009-05-15") } ] }
-{ "name": "WoodrowNehling", "id": 9, "alias": "Woodrow", "userSince": datetime("2005-09-20T10:10:00.000Z"), "friendIds": {{ 3, 10 }}, "employment": [ { "organizationName": "Zuncan", "startDate": date("2003-04-22"), "endDate": date("2009-12-13") } ], "nickname": "Woody" }
\ No newline at end of file
+{ "name": "NilaMilliron", "id": 8, "alias": "Nila", "userSince": datetime("2008-01-01T10:10:00.000"), "friendIds": {{ 3 }}, "employment": [ { "organizationName": "Plexlane", "startDate": date("2010-02-28") } ] }
+{ "name": "SuzannaTillson", "id": 7, "alias": "Suzanna", "userSince": datetime("2012-08-07T10:10:00.000"), "friendIds": {{ 6 }}, "employment": [ { "organizationName": "Labzatron", "startDate": date("2011-04-19") } ] }
+{ "name": "VonKemble", "id": 5, "alias": "Von", "userSince": datetime("2010-01-05T10:10:00.000"), "friendIds": {{ 3, 6, 10 }}, "employment": [ { "organizationName": "Kongreen", "startDate": date("2010-11-27") } ] }
+{ "name": "WillisWynne", "id": 6, "alias": "Willis", "userSince": datetime("2005-01-17T10:10:00.000"), "friendIds": {{ 1, 3, 7 }}, "employment": [ { "organizationName": "jaydax", "startDate": date("2009-05-15") } ] }
+{ "name": "WoodrowNehling", "id": 9, "alias": "Woodrow", "userSince": datetime("2005-09-20T10:10:00.000"), "friendIds": {{ 3, 10 }}, "employment": [ { "organizationName": "Zuncan", "startDate": date("2003-04-22"), "endDate": date("2009-12-13") } ], "nickname": "Woody" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/flushtest/flushtest.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/flushtest/flushtest.1.adm
index 23372b5..efaeb60 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/flushtest/flushtest.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/flushtest/flushtest.1.adm
@@ -1,1000 +1,1000 @@
-{ "id": 9005038, "id-copy": 9005038, "alias": "Anabel", "name": "AnabelWheeler", "user-since": datetime("2006-12-12T13:40:23.000Z"), "user-since-copy": datetime("2006-12-12T13:40:23.000Z"), "friend-ids": {{ 18713256, 35193719, 42245821, 37249622, 12210708, 15557948, 467039, 43997520, 45171035, 43682410, 47884198, 43102086, 39620955, 36438278, 42976932, 11158113, 21543594, 9861181, 36944403, 47928849, 29593861, 37897057, 42360015, 27956902 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2000-03-21") } ] }
-{ "id": 9008185, "id-copy": 9008185, "alias": "Francene", "name": "FranceneZoucks", "user-since": datetime("2009-10-18T08:37:00.000Z"), "user-since-copy": datetime("2009-10-18T08:37:00.000Z"), "friend-ids": {{ 47321113, 34578577, 25011033, 19259482, 6221464, 4912987, 20361608, 27957639, 33209653, 46928253, 37111867, 11534180, 31643335, 39967918, 8490889, 23713207, 28827713, 22143989, 21710696, 3545622, 13887489, 41557233, 26554092 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2004-02-01"), "end-date": date("2011-10-10") } ] }
-{ "id": 9012382, "id-copy": 9012382, "alias": "Laureen", "name": "LaureenOneal", "user-since": datetime("2009-12-10T22:17:58.000Z"), "user-since-copy": datetime("2009-12-10T22:17:58.000Z"), "friend-ids": {{ 25012654, 4572832, 38401260, 3015853, 42975956, 16328675, 39626774, 26936410, 15112607, 3302431 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2007-11-10") } ] }
-{ "id": 9012778, "id-copy": 9012778, "alias": "Godfrey", "name": "GodfreyBraun", "user-since": datetime("2010-03-18T19:15:53.000Z"), "user-since-copy": datetime("2010-03-18T19:15:53.000Z"), "friend-ids": {{ 3867712, 22078166 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2005-10-02") } ] }
-{ "id": 9042022, "id-copy": 9042022, "alias": "Fran", "name": "FranIronmonger", "user-since": datetime("2006-05-22T03:51:10.000Z"), "user-since-copy": datetime("2006-05-22T03:51:10.000Z"), "friend-ids": {{ 38546356, 31805246 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2002-06-06") } ] }
-{ "id": 9102208, "id-copy": 9102208, "alias": "Lottie", "name": "LottieReddish", "user-since": datetime("2007-05-22T00:42:45.000Z"), "user-since-copy": datetime("2007-05-22T00:42:45.000Z"), "friend-ids": {{ 45227463, 22488433, 39033954, 40377121, 17357169, 8890953, 1623690, 11657739, 489001, 26227491, 29459012, 39985553, 3584598, 6381312, 22457740, 43317482, 40035088, 29397671, 18293877, 6788834, 44860241 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2009-04-08") } ] }
-{ "id": 9129220, "id-copy": 9129220, "alias": "Lessie", "name": "LessieGoodman", "user-since": datetime("2008-09-01T06:07:35.000Z"), "user-since-copy": datetime("2008-09-01T06:07:35.000Z"), "friend-ids": {{ 16418186, 35990435, 22056439, 36479650, 36405609, 12039460, 33551878, 10736746, 41967761, 20046069, 8949956, 26571267 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2004-10-23"), "end-date": date("2011-05-08") } ] }
-{ "id": 9142198, "id-copy": 9142198, "alias": "Sherry", "name": "SherryFea", "user-since": datetime("2011-03-28T23:09:22.000Z"), "user-since-copy": datetime("2011-03-28T23:09:22.000Z"), "friend-ids": {{ 6835080, 34471872, 30942941, 34858577, 5996593, 47293442, 43097072, 44809621, 33969893, 26410931, 6628186, 29944391, 35957320, 20326929, 40284077, 11681583, 43878314, 40265961, 16871274, 28406169, 1349311 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2004-07-28") } ] }
-{ "id": 9185848, "id-copy": 9185848, "alias": "Brendon", "name": "BrendonJelliman", "user-since": datetime("2008-10-13T17:36:00.000Z"), "user-since-copy": datetime("2008-10-13T17:36:00.000Z"), "friend-ids": {{ 12675636, 6787931, 19218962, 12655930 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2008-06-09"), "end-date": date("2009-10-16") } ] }
-{ "id": 9199078, "id-copy": 9199078, "alias": "Erwin", "name": "ErwinErrett", "user-since": datetime("2011-04-20T12:44:31.000Z"), "user-since-copy": datetime("2011-04-20T12:44:31.000Z"), "friend-ids": {{ 31928109, 8101864, 44247743, 21370948 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2000-03-06") } ] }
-{ "id": 9207832, "id-copy": 9207832, "alias": "Tammy", "name": "TammyHozier", "user-since": datetime("2005-08-24T14:34:19.000Z"), "user-since-copy": datetime("2005-08-24T14:34:19.000Z"), "friend-ids": {{ 26919119, 35729176, 28949827 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2006-09-14") } ] }
-{ "id": 9211711, "id-copy": 9211711, "alias": "Seraphina", "name": "SeraphinaFlanders", "user-since": datetime("2009-05-19T18:39:15.000Z"), "user-since-copy": datetime("2009-05-19T18:39:15.000Z"), "friend-ids": {{ 34432294, 10796959, 46386746, 32318131, 10393677, 12832313, 34490791, 6187782, 46595448, 30591963, 35530646, 22485004, 18950892, 19762388, 19181134, 13928403, 22513246, 24969298 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2008-05-14"), "end-date": date("2009-06-17") } ] }
-{ "id": 9212815, "id-copy": 9212815, "alias": "Erica", "name": "EricaBraun", "user-since": datetime("2009-01-11T07:32:03.000Z"), "user-since-copy": datetime("2009-01-11T07:32:03.000Z"), "friend-ids": {{ 1314906, 6581233, 35117578, 11133528, 19606776, 37833518, 40040803, 44107209, 38804989, 35779440, 41138709 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2008-04-03") } ] }
-{ "id": 9243769, "id-copy": 9243769, "alias": "Florentino", "name": "FlorentinoRiggle", "user-since": datetime("2012-04-04T17:10:31.000Z"), "user-since-copy": datetime("2012-04-04T17:10:31.000Z"), "friend-ids": {{ 41929020, 22354873 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2007-07-23") } ] }
-{ "id": 9265747, "id-copy": 9265747, "alias": "Nicolas", "name": "NicolasPirl", "user-since": datetime("2011-11-07T13:52:49.000Z"), "user-since-copy": datetime("2011-11-07T13:52:49.000Z"), "friend-ids": {{ 5832017, 30839617, 27328653, 9766355, 35973149, 21029594, 18840511, 43035135, 44902336, 11576374, 21756219, 23374243, 42201568, 12860309 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2003-11-10"), "end-date": date("2010-03-27") } ] }
-{ "id": 9311659, "id-copy": 9311659, "alias": "Kate", "name": "KateBender", "user-since": datetime("2007-06-10T05:55:50.000Z"), "user-since-copy": datetime("2007-06-10T05:55:50.000Z"), "friend-ids": {{ 27875958, 10379355, 4286877, 26410945, 10609943, 15960135 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-01-04") } ] }
-{ "id": 9318094, "id-copy": 9318094, "alias": "Carlo", "name": "CarloKelley", "user-since": datetime("2012-07-19T09:18:41.000Z"), "user-since-copy": datetime("2012-07-19T09:18:41.000Z"), "friend-ids": {{ 39873731, 29304807, 519851, 16423529, 10838418, 9915172, 3040071, 39730361, 23320290, 20572900, 7293676, 35037765, 1744053, 38875858 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2008-08-15") } ] }
-{ "id": 9326218, "id-copy": 9326218, "alias": "Lindsay", "name": "LindsayPaynter", "user-since": datetime("2011-08-27T00:03:13.000Z"), "user-since-copy": datetime("2011-08-27T00:03:13.000Z"), "friend-ids": {{ 3006430, 25941368, 46866627, 21404266, 35141764, 14931901 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-04-06"), "end-date": date("2008-03-02") } ] }
-{ "id": 9343705, "id-copy": 9343705, "alias": "Ramsey", "name": "RamseyWarner", "user-since": datetime("2006-04-24T09:52:39.000Z"), "user-since-copy": datetime("2006-04-24T09:52:39.000Z"), "friend-ids": {{ 36909861, 36881715, 40993685, 18669519, 42428458, 2780280, 6070725, 10466662, 26215221, 16329040, 38464211, 14024902, 8083000, 27857433, 14282674, 1976238, 6345526, 35452338, 21503723, 34910137, 26860195, 426384, 27759959 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2003-10-28") } ] }
-{ "id": 9379330, "id-copy": 9379330, "alias": "Esther", "name": "EstherReichard", "user-since": datetime("2006-09-23T09:53:43.000Z"), "user-since-copy": datetime("2006-09-23T09:53:43.000Z"), "friend-ids": {{ 29035495, 33601969, 32342695, 28995226, 34638799, 38330225, 38512256 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2006-05-27") } ] }
-{ "id": 9379975, "id-copy": 9379975, "alias": "Kyra", "name": "KyraLangston", "user-since": datetime("2012-01-18T06:06:56.000Z"), "user-since-copy": datetime("2012-01-18T06:06:56.000Z"), "friend-ids": {{ 46662872, 1388016, 21715152, 3266023, 18080709, 25857347, 29710885, 22300787, 25086634, 25220921, 17189604, 21754574, 27820275, 7441940, 10911235, 46304871, 6518794 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2008-04-03"), "end-date": date("2008-04-07") } ] }
-{ "id": 9417499, "id-copy": 9417499, "alias": "Wendell", "name": "WendellJoyce", "user-since": datetime("2011-07-25T14:30:30.000Z"), "user-since-copy": datetime("2011-07-25T14:30:30.000Z"), "friend-ids": {{ 10079972, 29246113, 40533159, 31279768, 31969044, 46120195, 35004468, 24465042, 2702879, 44166678, 20176481, 32056309, 38254930, 20950061, 4687108 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2006-03-18") } ] }
-{ "id": 9430849, "id-copy": 9430849, "alias": "Emil", "name": "EmilGarland", "user-since": datetime("2008-07-03T15:56:07.000Z"), "user-since-copy": datetime("2008-07-03T15:56:07.000Z"), "friend-ids": {{ 40429008, 45432330, 22293451, 2129366, 19514477, 20108162, 28656704, 35403173, 33855801, 14660181 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2010-02-10") } ] }
-{ "id": 9442978, "id-copy": 9442978, "alias": "Osborne", "name": "OsborneHiles", "user-since": datetime("2012-07-28T10:59:39.000Z"), "user-since-copy": datetime("2012-07-28T10:59:39.000Z"), "friend-ids": {{ 40833026, 39533118, 6206868, 27383373, 3010465, 14776443, 43239645, 21956253, 4112089, 27667721, 34336067, 38377619, 32701403, 20907262, 32732275, 30488150, 12349697, 47468946, 20956164, 16141416 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2011-08-21") } ] }
-{ "id": 9461098, "id-copy": 9461098, "alias": "Teodoro", "name": "TeodoroBullard", "user-since": datetime("2010-07-24T07:40:44.000Z"), "user-since-copy": datetime("2010-07-24T07:40:44.000Z"), "friend-ids": {{ 8278091, 1756629, 9893864, 11184021, 2292251, 20614604, 48014557, 23491569, 11328678, 11572435, 45790306, 44930978, 34910222, 16655255, 29338869, 27169036, 19669405, 20512510, 33598988, 38104427 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2003-01-17"), "end-date": date("2007-05-28") } ] }
-{ "id": 9471385, "id-copy": 9471385, "alias": "Weldon", "name": "WeldonMaclagan", "user-since": datetime("2010-01-24T22:21:59.000Z"), "user-since-copy": datetime("2010-01-24T22:21:59.000Z"), "friend-ids": {{ 42864267, 16710494, 27436346, 7324905, 3901396, 11812437, 31490561, 3906397 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2002-09-07"), "end-date": date("2006-07-08") } ] }
-{ "id": 9477040, "id-copy": 9477040, "alias": "Chery", "name": "CheryWatson", "user-since": datetime("2012-05-02T14:27:00.000Z"), "user-since-copy": datetime("2012-05-02T14:27:00.000Z"), "friend-ids": {{ 36360097, 36835617, 25761112, 30806900, 22340413, 16802957 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2008-12-26"), "end-date": date("2009-03-17") } ] }
-{ "id": 9477994, "id-copy": 9477994, "alias": "Cory", "name": "CoryKeener", "user-since": datetime("2012-02-27T22:03:31.000Z"), "user-since-copy": datetime("2012-02-27T22:03:31.000Z"), "friend-ids": {{ 22204843, 35394804, 22795967, 16575437, 31764908, 27359073, 50023, 26383393, 36534917, 23478654, 31022293, 43803666, 24764841, 19469389, 6401330, 10543085, 5159571 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2012-02-09"), "end-date": date("2012-02-19") } ] }
-{ "id": 9481756, "id-copy": 9481756, "alias": "Esmaralda", "name": "EsmaraldaAgg", "user-since": datetime("2012-06-26T19:57:38.000Z"), "user-since-copy": datetime("2012-06-26T19:57:38.000Z"), "friend-ids": {{ 40976868 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2008-11-26"), "end-date": date("2008-01-13") } ] }
-{ "id": 9490342, "id-copy": 9490342, "alias": "Gisela", "name": "GiselaTomlinson", "user-since": datetime("2011-10-21T20:36:09.000Z"), "user-since-copy": datetime("2011-10-21T20:36:09.000Z"), "friend-ids": {{ 27609144, 42495049, 21250269, 22561106, 29149509, 16776721, 16980559, 19600765 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2003-02-23") } ] }
-{ "id": 9512971, "id-copy": 9512971, "alias": "Algar", "name": "AlgarKepplinger", "user-since": datetime("2011-10-11T02:54:01.000Z"), "user-since-copy": datetime("2011-10-11T02:54:01.000Z"), "friend-ids": {{ 1076656, 1837449, 43428033, 21710004, 41167492, 17526252 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2003-04-26"), "end-date": date("2006-02-24") } ] }
-{ "id": 9516652, "id-copy": 9516652, "alias": "Emmanuel", "name": "EmmanuelStrickland", "user-since": datetime("2006-01-14T03:08:13.000Z"), "user-since-copy": datetime("2006-01-14T03:08:13.000Z"), "friend-ids": {{ 21213113, 8011145, 9382308, 14949454, 114459, 30046906, 40091327, 22275481, 14642211, 5602065, 15265189, 22736575, 12746303, 46033445, 17273286, 39395247, 6653955, 14664612, 35055957 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-10-15") } ] }
-{ "id": 9525361, "id-copy": 9525361, "alias": "Leonardo", "name": "LeonardoSurrency", "user-since": datetime("2008-12-21T10:09:26.000Z"), "user-since-copy": datetime("2008-12-21T10:09:26.000Z"), "friend-ids": {{ 12471014, 47714763, 18071069, 32545366, 46041462, 35261185, 20826834, 29002678, 47207065, 7370034, 38283272, 47090645, 33425043, 16014552, 15633873, 24101778, 26168621, 21955493, 17856723, 18158610 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2011-12-06"), "end-date": date("2011-04-04") } ] }
-{ "id": 9532474, "id-copy": 9532474, "alias": "Chester", "name": "ChesterAshmore", "user-since": datetime("2012-02-03T20:36:34.000Z"), "user-since-copy": datetime("2012-02-03T20:36:34.000Z"), "friend-ids": {{ 11340481, 15957237, 47048138, 41603112, 6953329, 6926093, 20866295, 329274, 16187993, 13406075, 34601684, 46151089, 26165473, 2882718, 20731108 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2009-03-14") } ] }
-{ "id": 9546133, "id-copy": 9546133, "alias": "Renae", "name": "RenaeWhitehead", "user-since": datetime("2012-04-21T14:38:30.000Z"), "user-since-copy": datetime("2012-04-21T14:38:30.000Z"), "friend-ids": {{ 31261211, 19892104, 35568606, 12050300, 42512152, 37032282, 27185051 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2012-02-20"), "end-date": date("2012-07-04") } ] }
-{ "id": 9552016, "id-copy": 9552016, "alias": "Shantelle", "name": "ShantelleDealtry", "user-since": datetime("2006-05-03T06:49:13.000Z"), "user-since-copy": datetime("2006-05-03T06:49:13.000Z"), "friend-ids": {{ 35758396, 16562240, 23596680, 16342769, 19892813, 46485447, 25711418, 23765073, 11303996, 36451291, 17586370, 38010455, 29457199, 25847013, 12604123, 46533018, 26999208, 24740610, 35225441, 33613663 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2003-08-07"), "end-date": date("2003-07-17") } ] }
-{ "id": 9560251, "id-copy": 9560251, "alias": "Nivek", "name": "NivekJowers", "user-since": datetime("2007-02-04T08:02:07.000Z"), "user-since-copy": datetime("2007-02-04T08:02:07.000Z"), "friend-ids": {{ 15730417, 36745553, 26133088, 38675683, 14617495, 39244216, 4651791, 639869, 8377526, 15158817, 13368295, 15386494, 5649384, 8449938, 34497809, 6644713, 45481442, 27678941, 14214532, 5753112, 9991855, 25975202, 9530884, 19069924 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2003-08-15") } ] }
-{ "id": 9562348, "id-copy": 9562348, "alias": "Jefferson", "name": "JeffersonKeister", "user-since": datetime("2005-06-11T01:42:58.000Z"), "user-since-copy": datetime("2005-06-11T01:42:58.000Z"), "friend-ids": {{ 43801762 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2005-07-26"), "end-date": date("2011-12-02") } ] }
-{ "id": 9577729, "id-copy": 9577729, "alias": "Jann", "name": "JannPorter", "user-since": datetime("2006-05-03T08:57:08.000Z"), "user-since-copy": datetime("2006-05-03T08:57:08.000Z"), "friend-ids": {{ 7711959, 4131696, 10146353, 46418552, 37999454, 38333059, 16381326, 45028736, 16829150 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2006-10-19") } ] }
-{ "id": 9591646, "id-copy": 9591646, "alias": "Hoyt", "name": "HoytGilman", "user-since": datetime("2011-05-13T07:22:20.000Z"), "user-since-copy": datetime("2011-05-13T07:22:20.000Z"), "friend-ids": {{ 11207445 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2004-04-27") } ] }
-{ "id": 9595279, "id-copy": 9595279, "alias": "Emmaline", "name": "EmmalineSchuth", "user-since": datetime("2008-09-12T22:25:17.000Z"), "user-since-copy": datetime("2008-09-12T22:25:17.000Z"), "friend-ids": {{ 26784778, 6200196, 37440596, 12250319, 21921557, 19278082, 583040, 12012653, 21578028, 16395818, 29088493, 29578064, 37745574, 41998781, 22594273, 38002130, 2166585, 7823908, 18253304, 6162341, 40270219, 41832701, 36455204 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2009-02-13") } ] }
-{ "id": 9606691, "id-copy": 9606691, "alias": "Reva", "name": "RevaChristman", "user-since": datetime("2010-03-04T11:53:00.000Z"), "user-since-copy": datetime("2010-03-04T11:53:00.000Z"), "friend-ids": {{ 21390421 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2010-12-13") } ] }
-{ "id": 9621157, "id-copy": 9621157, "alias": "Trixie", "name": "TrixieFair", "user-since": datetime("2010-12-25T23:36:49.000Z"), "user-since-copy": datetime("2010-12-25T23:36:49.000Z"), "friend-ids": {{ 17519006, 17545060, 27836293, 11477603, 37895380, 23251592, 12010503, 25406806 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2003-09-23") } ] }
-{ "id": 9629395, "id-copy": 9629395, "alias": "Julius", "name": "JuliusWire", "user-since": datetime("2008-03-22T13:36:24.000Z"), "user-since-copy": datetime("2008-03-22T13:36:24.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2006-11-19") } ] }
-{ "id": 9634393, "id-copy": 9634393, "alias": "Burt", "name": "BurtPearson", "user-since": datetime("2007-11-01T14:25:29.000Z"), "user-since-copy": datetime("2007-11-01T14:25:29.000Z"), "friend-ids": {{ 26065414, 8710639, 22639162, 23787625, 24443211, 42598742, 45171006, 38246985, 25125478, 23071168, 22455706, 24720860, 34917747, 24262081, 2259812, 14262605, 37533604 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-05-07") } ] }
-{ "id": 9640915, "id-copy": 9640915, "alias": "Harrison", "name": "HarrisonHildyard", "user-since": datetime("2009-05-25T11:56:05.000Z"), "user-since-copy": datetime("2009-05-25T11:56:05.000Z"), "friend-ids": {{ 41488832, 16139664, 18327029, 38811764, 38271538, 13106137, 26450611, 11574808, 33108523, 31639017, 9208159, 18456510, 47955463, 2606160, 29293146, 13981743, 39967993, 23629640, 32666499, 35046044, 2402842, 1117025, 17741007, 14997808 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2010-03-06") } ] }
-{ "id": 9643768, "id-copy": 9643768, "alias": "Gil", "name": "GilVeith", "user-since": datetime("2006-04-26T11:42:30.000Z"), "user-since-copy": datetime("2006-04-26T11:42:30.000Z"), "friend-ids": {{ 22270431, 9614818, 9080111, 6500797, 37876717, 28122656, 13971193, 20936637, 19883735, 37455193, 32129291, 40710966, 17779823, 41523128, 41276564, 34424817, 19326867, 26058281 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2002-02-04") } ] }
-{ "id": 9669178, "id-copy": 9669178, "alias": "Gerard", "name": "GerardBeck", "user-since": datetime("2011-04-24T15:49:24.000Z"), "user-since-copy": datetime("2011-04-24T15:49:24.000Z"), "friend-ids": {{ 30087138, 44736614, 1531569 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2003-09-25"), "end-date": date("2005-06-28") } ] }
-{ "id": 9680644, "id-copy": 9680644, "alias": "Mirtha", "name": "MirthaRahl", "user-since": datetime("2008-02-09T04:05:03.000Z"), "user-since-copy": datetime("2008-02-09T04:05:03.000Z"), "friend-ids": {{ 25328638, 9009324, 16627989, 46602908, 32685062, 10538437, 22403363, 4205292, 27910567, 28430833, 8519372, 39774027, 12120028, 1211979 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2006-12-19") } ] }
-{ "id": 9683656, "id-copy": 9683656, "alias": "Antone", "name": "AntoneMays", "user-since": datetime("2006-07-24T22:48:29.000Z"), "user-since-copy": datetime("2006-07-24T22:48:29.000Z"), "friend-ids": {{ 11275116, 40325672, 41154035, 8987353, 31187312, 11505721, 11584703, 42743337, 23225356, 8653923 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2011-06-12") } ] }
-{ "id": 9695773, "id-copy": 9695773, "alias": "Daron", "name": "DaronFiddler", "user-since": datetime("2006-12-25T17:08:50.000Z"), "user-since-copy": datetime("2006-12-25T17:08:50.000Z"), "friend-ids": {{ 14397778, 33469556, 41690231, 7827360, 42196316 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2007-12-24") } ] }
-{ "id": 9707074, "id-copy": 9707074, "alias": "Melvyn", "name": "MelvynSybilla", "user-since": datetime("2012-06-07T16:06:49.000Z"), "user-since-copy": datetime("2012-06-07T16:06:49.000Z"), "friend-ids": {{ 4487400, 488933, 15650706, 44692005, 25068052, 16975927 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2010-12-13") } ] }
-{ "id": 9740008, "id-copy": 9740008, "alias": "Woodrow", "name": "WoodrowBlois", "user-since": datetime("2011-12-18T11:34:56.000Z"), "user-since-copy": datetime("2011-12-18T11:34:56.000Z"), "friend-ids": {{ 1753941, 17603348, 44569557, 6816408, 17403631, 29707555, 21215516, 9837919, 35887854, 35236051, 7897485, 9880491, 16145458, 33128036, 41471362, 44171952, 23542112, 36155237, 2596261, 36702766 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2007-08-02") } ] }
-{ "id": 9747652, "id-copy": 9747652, "alias": "Graham", "name": "GrahamGarratt", "user-since": datetime("2006-04-16T19:35:33.000Z"), "user-since-copy": datetime("2006-04-16T19:35:33.000Z"), "friend-ids": {{ 9995821, 7082678, 29813051, 33625501, 32785793, 23170533, 26581328, 35564866, 9147486, 17626916, 12721534, 22070579, 25749282, 27771492, 35217137, 6426437, 4217778, 6819045, 6410966, 43080321, 32112201, 20323505 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2005-09-26") } ] }
-{ "id": 9761152, "id-copy": 9761152, "alias": "Royle", "name": "RoyleStewart", "user-since": datetime("2010-05-15T17:14:18.000Z"), "user-since-copy": datetime("2010-05-15T17:14:18.000Z"), "friend-ids": {{ 21868661, 15545005, 11285872, 45768523, 12486235 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2002-11-20") } ] }
-{ "id": 9779623, "id-copy": 9779623, "alias": "Alberto", "name": "AlbertoCraig", "user-since": datetime("2009-11-25T14:48:04.000Z"), "user-since-copy": datetime("2009-11-25T14:48:04.000Z"), "friend-ids": {{ 6737836, 26882597, 30254391, 4861442, 18105612 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2012-01-25") } ] }
-{ "id": 9799264, "id-copy": 9799264, "alias": "Bradley", "name": "BradleyTodd", "user-since": datetime("2011-05-18T23:42:33.000Z"), "user-since-copy": datetime("2011-05-18T23:42:33.000Z"), "friend-ids": {{ 8836368, 35488923, 26777243, 46550104, 9866525, 965209 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2007-12-22") } ] }
-{ "id": 9799591, "id-copy": 9799591, "alias": "Royston", "name": "RoystonChurchill", "user-since": datetime("2011-01-21T13:57:31.000Z"), "user-since-copy": datetime("2011-01-21T13:57:31.000Z"), "friend-ids": {{ 22757950, 4629721, 19522595, 27737642, 39393176, 9321441, 13496995, 43301849, 3869585, 34993450, 24876688 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2000-07-15") } ] }
-{ "id": 9804973, "id-copy": 9804973, "alias": "Harriette", "name": "HarrietteHoopengarner", "user-since": datetime("2011-08-14T20:51:52.000Z"), "user-since-copy": datetime("2011-08-14T20:51:52.000Z"), "friend-ids": {{ 18754696, 27799194, 36904141, 29647419, 8521621, 35146470, 45194388, 43397176, 12596887, 33315, 39826335, 31228413, 123596, 35927645, 11445687, 33208186, 21941268 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2003-03-24") } ] }
-{ "id": 9819796, "id-copy": 9819796, "alias": "Emerson", "name": "EmersonWardle", "user-since": datetime("2006-08-20T20:22:11.000Z"), "user-since-copy": datetime("2006-08-20T20:22:11.000Z"), "friend-ids": {{ 5697147, 42936553, 12624322, 45309083, 10785774, 4176618 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-05-16") } ] }
-{ "id": 9820681, "id-copy": 9820681, "alias": "Caitlin", "name": "CaitlinWolfe", "user-since": datetime("2012-05-23T07:59:39.000Z"), "user-since-copy": datetime("2012-05-23T07:59:39.000Z"), "friend-ids": {{ 22005473, 7664709, 22913945, 16078115, 11724028, 45958589, 33357270, 6935384, 2696233, 28938665, 37992833, 11929142, 16203505, 20365802 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2010-06-18") } ] }
-{ "id": 9826402, "id-copy": 9826402, "alias": "Rachyl", "name": "RachylRumbaugh", "user-since": datetime("2006-01-05T03:38:59.000Z"), "user-since-copy": datetime("2006-01-05T03:38:59.000Z"), "friend-ids": {{ 11891915, 15900581, 38420311, 21084667, 24569500, 9181299, 32167823, 9967774, 18138704, 10742133, 29173609, 1113683, 21048344, 33794587, 42308958, 9303744 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2005-04-24"), "end-date": date("2008-08-17") } ] }
-{ "id": 9829834, "id-copy": 9829834, "alias": "Darryl", "name": "DarrylSullivan", "user-since": datetime("2011-07-24T00:12:33.000Z"), "user-since-copy": datetime("2011-07-24T00:12:33.000Z"), "friend-ids": {{ 8297654, 6071837, 27236382, 4657522, 9035310, 40427605, 2360931, 19796421, 7301200, 1264845, 12653555, 27518516 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2005-01-18"), "end-date": date("2010-05-20") } ] }
-{ "id": 9840013, "id-copy": 9840013, "alias": "Inger", "name": "IngerRuhl", "user-since": datetime("2009-05-27T20:14:42.000Z"), "user-since-copy": datetime("2009-05-27T20:14:42.000Z"), "friend-ids": {{ 36044692 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2010-09-15") } ] }
-{ "id": 9845113, "id-copy": 9845113, "alias": "Chia", "name": "ChiaGeddinge", "user-since": datetime("2008-12-12T16:50:57.000Z"), "user-since-copy": datetime("2008-12-12T16:50:57.000Z"), "friend-ids": {{ 16725476, 120161, 762756, 40795640, 34195102, 27938737 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2001-01-03"), "end-date": date("2001-11-03") } ] }
-{ "id": 9859726, "id-copy": 9859726, "alias": "Taryn", "name": "TarynGisiko", "user-since": datetime("2010-12-28T21:42:56.000Z"), "user-since-copy": datetime("2010-12-28T21:42:56.000Z"), "friend-ids": {{ 45036313, 47860435, 40658528, 4106429, 25411752, 7216290, 20549107, 28317961, 43600081, 6359672, 36131464, 19078372, 4379305, 884797, 11605059, 6467240, 23316141 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2007-06-26"), "end-date": date("2010-08-04") } ] }
-{ "id": 9872791, "id-copy": 9872791, "alias": "Yasmine", "name": "YasmineCanham", "user-since": datetime("2005-06-08T14:45:42.000Z"), "user-since-copy": datetime("2005-06-08T14:45:42.000Z"), "friend-ids": {{ 7340569, 16137560, 43341029, 31700386, 24881875, 17852264, 42730676, 32655012 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2004-05-09"), "end-date": date("2011-02-28") } ] }
-{ "id": 9877837, "id-copy": 9877837, "alias": "Marilee", "name": "MarileeDowning", "user-since": datetime("2007-09-06T15:02:25.000Z"), "user-since-copy": datetime("2007-09-06T15:02:25.000Z"), "friend-ids": {{ 3032720, 7000379, 16658012, 33487490, 624779, 13480315, 8308906, 6949934, 9472007, 36568244, 41737195, 1310478, 42870077, 46663613 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2005-10-09") } ] }
-{ "id": 9919033, "id-copy": 9919033, "alias": "Bailey", "name": "BaileyHay", "user-since": datetime("2005-01-06T07:43:18.000Z"), "user-since-copy": datetime("2005-01-06T07:43:18.000Z"), "friend-ids": {{ 28198532 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2009-02-08"), "end-date": date("2010-06-08") } ] }
-{ "id": 9922381, "id-copy": 9922381, "alias": "Cecilia", "name": "CeciliaOsteen", "user-since": datetime("2009-06-03T03:58:36.000Z"), "user-since-copy": datetime("2009-06-03T03:58:36.000Z"), "friend-ids": {{ 22246989, 9095240, 8953245, 16326669, 38845534, 13608449, 35076758, 42004583 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2010-08-02") } ] }
-{ "id": 9929866, "id-copy": 9929866, "alias": "Emilie", "name": "EmilieJohns", "user-since": datetime("2009-10-01T00:51:03.000Z"), "user-since-copy": datetime("2009-10-01T00:51:03.000Z"), "friend-ids": {{ 45496950, 38109555, 46259676, 14141368, 31720484, 35564907, 23226721, 36026226, 34003258, 47176035, 46593035, 5050811, 27858647, 3784968 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2012-08-10"), "end-date": date("2012-08-24") } ] }
-{ "id": 9931588, "id-copy": 9931588, "alias": "Sheri", "name": "SheriHindman", "user-since": datetime("2011-02-19T03:55:37.000Z"), "user-since-copy": datetime("2011-02-19T03:55:37.000Z"), "friend-ids": {{ 10993709, 28005344, 31884585, 1581885, 46332238, 47401902, 38814902, 39736365, 24318394, 15329318, 35794552, 14913021, 8723328, 28102869, 27218765, 21310255 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2011-08-17"), "end-date": date("2011-12-15") } ] }
-{ "id": 9937957, "id-copy": 9937957, "alias": "Corey", "name": "CoreyTaggart", "user-since": datetime("2005-11-25T16:13:03.000Z"), "user-since-copy": datetime("2005-11-25T16:13:03.000Z"), "friend-ids": {{ 40105038, 9364511, 47362703, 1876955, 3505769, 41708385, 36179634, 7022850 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2005-09-24") } ] }
-{ "id": 9945208, "id-copy": 9945208, "alias": "Thelma", "name": "ThelmaGettemy", "user-since": datetime("2006-12-21T11:17:06.000Z"), "user-since-copy": datetime("2006-12-21T11:17:06.000Z"), "friend-ids": {{ 26578648, 43730418, 18099472, 11787057, 41534206, 16778979, 41142786, 25761045, 18556835, 25378849, 38984390, 37528215, 2531696 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2008-03-25") } ] }
-{ "id": 9952339, "id-copy": 9952339, "alias": "Dacia", "name": "DaciaStaymates", "user-since": datetime("2009-09-27T09:55:51.000Z"), "user-since-copy": datetime("2009-09-27T09:55:51.000Z"), "friend-ids": {{ 5177020, 46967179, 24156959, 17828131, 41565753, 1929360, 33761670, 27544454, 9964059, 25582191 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2000-10-12"), "end-date": date("2007-01-20") } ] }
-{ "id": 9962236, "id-copy": 9962236, "alias": "Craig", "name": "CraigKight", "user-since": datetime("2010-02-15T15:58:03.000Z"), "user-since-copy": datetime("2010-02-15T15:58:03.000Z"), "friend-ids": {{ 45604304, 40911167, 39517053, 6912584, 898627, 8412812, 33530827, 30135549, 14762146, 46313211, 21143796, 39820220, 11462372, 23575315 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-02-05"), "end-date": date("2008-01-04") } ] }
-{ "id": 9968869, "id-copy": 9968869, "alias": "Shemika", "name": "ShemikaNickolson", "user-since": datetime("2005-02-20T10:34:04.000Z"), "user-since-copy": datetime("2005-02-20T10:34:04.000Z"), "friend-ids": {{ 30287118, 877645, 9968776, 31800907 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2009-07-23") } ] }
-{ "id": 9975778, "id-copy": 9975778, "alias": "Marmaduke", "name": "MarmadukeElizabeth", "user-since": datetime("2012-07-18T02:21:55.000Z"), "user-since-copy": datetime("2012-07-18T02:21:55.000Z"), "friend-ids": {{ 17424696, 34807936, 8912699, 40534595, 36049658, 31706902, 7626256, 16178188, 36944385, 47878361, 8190132, 34365280, 13576207, 42728095 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2006-08-19") } ] }
-{ "id": 9988417, "id-copy": 9988417, "alias": "Coline", "name": "ColineLane", "user-since": datetime("2010-01-01T00:12:39.000Z"), "user-since-copy": datetime("2010-01-01T00:12:39.000Z"), "friend-ids": {{ 17656229, 42804152 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2012-05-01") } ] }
-{ "id": 9993001, "id-copy": 9993001, "alias": "Herbie", "name": "HerbieStall", "user-since": datetime("2010-06-14T03:01:11.000Z"), "user-since-copy": datetime("2010-06-14T03:01:11.000Z"), "friend-ids": {{ 12003033, 40923715, 34166285, 47927261, 638933, 17338590 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2009-07-12") } ] }
-{ "id": 10054327, "id-copy": 10054327, "alias": "Poppy", "name": "PoppyKellogg", "user-since": datetime("2010-03-28T09:43:49.000Z"), "user-since-copy": datetime("2010-03-28T09:43:49.000Z"), "friend-ids": {{ 10785684, 26545687, 942400, 18147517, 12133643, 17848751, 40864121, 18975370, 26159158, 42348235, 21795276, 40155922, 35240759 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2012-03-24") } ] }
-{ "id": 10065250, "id-copy": 10065250, "alias": "Debbie", "name": "DebbieBrinigh", "user-since": datetime("2012-01-05T15:05:48.000Z"), "user-since-copy": datetime("2012-01-05T15:05:48.000Z"), "friend-ids": {{ 23794420, 31166549, 3372724, 35955226, 45241312, 33488036, 17353508, 10094234, 12751868 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2000-06-28"), "end-date": date("2005-06-03") } ] }
-{ "id": 10066711, "id-copy": 10066711, "alias": "Nichelle", "name": "NichelleErschoff", "user-since": datetime("2009-11-10T21:17:50.000Z"), "user-since-copy": datetime("2009-11-10T21:17:50.000Z"), "friend-ids": {{ 19024226, 24428716, 24428406, 10686682, 46410623, 45809403, 33158503 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2004-06-21"), "end-date": date("2005-08-01") } ] }
-{ "id": 10069987, "id-copy": 10069987, "alias": "Andrina", "name": "AndrinaFisher", "user-since": datetime("2012-07-21T07:28:30.000Z"), "user-since-copy": datetime("2012-07-21T07:28:30.000Z"), "friend-ids": {{ 42024943, 39627436, 28414443, 36703363, 45477433, 37499278, 28548620, 6687009, 22700392, 47812034, 16805789, 33222895, 36328879, 20191886, 32457353, 14008353 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2004-12-11"), "end-date": date("2004-09-07") } ] }
-{ "id": 10073440, "id-copy": 10073440, "alias": "Mat", "name": "MatHasely", "user-since": datetime("2007-02-15T12:28:32.000Z"), "user-since-copy": datetime("2007-02-15T12:28:32.000Z"), "friend-ids": {{ 18317132, 16303558, 35197704, 41199497, 17394418, 18594954, 13332602, 15164806, 20807780, 18284264, 17164369, 6418744, 26535302, 47287046, 7169299, 22825706, 34007482, 38108004, 14449725, 16993574, 28055503 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2005-09-04"), "end-date": date("2006-06-02") } ] }
-{ "id": 10079965, "id-copy": 10079965, "alias": "Mason", "name": "MasonReamer", "user-since": datetime("2008-08-10T02:16:36.000Z"), "user-since-copy": datetime("2008-08-10T02:16:36.000Z"), "friend-ids": {{ 37149190, 37736572, 35955709, 28586597, 45460389 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-01-18"), "end-date": date("2010-12-09") } ] }
-{ "id": 10087876, "id-copy": 10087876, "alias": "Carlyle", "name": "CarlyleMoberly", "user-since": datetime("2009-09-12T03:44:36.000Z"), "user-since-copy": datetime("2009-09-12T03:44:36.000Z"), "friend-ids": {{ 22254101, 16994379, 42146906, 28928982 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2012-07-24"), "end-date": date("2012-07-09") } ] }
-{ "id": 10126408, "id-copy": 10126408, "alias": "Pen", "name": "PenFleming", "user-since": datetime("2005-11-11T08:50:34.000Z"), "user-since-copy": datetime("2005-11-11T08:50:34.000Z"), "friend-ids": {{ 38072630, 45021886, 23988042, 41084533, 4743969, 7223979, 19120365, 44219284, 4691449, 21072839, 32536521, 36335527, 47376347, 16882811, 43140173, 7610811, 28217191, 25488874, 27968660, 13102347, 40169395, 25952056, 17249838, 30971677 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2011-10-04"), "end-date": date("2011-01-10") } ] }
-{ "id": 10135477, "id-copy": 10135477, "alias": "Jasmine", "name": "JasmineEva", "user-since": datetime("2009-04-03T11:48:27.000Z"), "user-since-copy": datetime("2009-04-03T11:48:27.000Z"), "friend-ids": {{ 3776073 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2000-11-14"), "end-date": date("2001-05-19") } ] }
-{ "id": 10148251, "id-copy": 10148251, "alias": "Ghislaine", "name": "GhislaineFowler", "user-since": datetime("2005-12-08T05:25:56.000Z"), "user-since-copy": datetime("2005-12-08T05:25:56.000Z"), "friend-ids": {{ 14692731, 29964772 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2008-12-27"), "end-date": date("2008-04-02") } ] }
-{ "id": 10177078, "id-copy": 10177078, "alias": "Fausto", "name": "FaustoLotherington", "user-since": datetime("2005-06-23T22:18:16.000Z"), "user-since-copy": datetime("2005-06-23T22:18:16.000Z"), "friend-ids": {{ 9405744, 13732034 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2007-12-27") } ] }
-{ "id": 10177300, "id-copy": 10177300, "alias": "Chase", "name": "ChaseKnapp", "user-since": datetime("2005-09-27T16:41:30.000Z"), "user-since-copy": datetime("2005-09-27T16:41:30.000Z"), "friend-ids": {{ 12805247, 6093464, 39416190, 35877238, 26583227, 37835412, 46337730, 18107636, 43948720, 21031949, 11688759, 13980476, 25486392, 20775628 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2006-03-07"), "end-date": date("2006-05-09") } ] }
-{ "id": 10178518, "id-copy": 10178518, "alias": "Rudyard", "name": "RudyardMcmullen", "user-since": datetime("2011-05-06T14:57:22.000Z"), "user-since-copy": datetime("2011-05-06T14:57:22.000Z"), "friend-ids": {{ 25647527, 14445589, 47924548, 24945241, 13505530, 39640007, 6132209, 815976, 31529708, 28281922, 17886251, 42402860, 18330827, 13619952 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2006-06-24") } ] }
-{ "id": 10179538, "id-copy": 10179538, "alias": "Orlando", "name": "OrlandoBaxter", "user-since": datetime("2006-02-06T08:33:07.000Z"), "user-since-copy": datetime("2006-02-06T08:33:07.000Z"), "friend-ids": {{ 6233497, 33888281, 44259464, 19279042, 22534429, 13084190, 38886041, 41675566, 3155617 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2009-07-06") } ] }
-{ "id": 10189600, "id-copy": 10189600, "alias": "Melisa", "name": "MelisaGarry", "user-since": datetime("2010-05-10T10:35:49.000Z"), "user-since-copy": datetime("2010-05-10T10:35:49.000Z"), "friend-ids": {{ 18172527, 26205741, 32077713, 41214698, 33783052, 5734397, 46101468, 30210046, 27425699 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-05-20"), "end-date": date("2011-07-20") } ] }
-{ "id": 10195063, "id-copy": 10195063, "alias": "Rose", "name": "RoseHatcher", "user-since": datetime("2008-10-11T02:17:54.000Z"), "user-since-copy": datetime("2008-10-11T02:17:54.000Z"), "friend-ids": {{ 9820231, 12294967, 46911959, 47936560, 7881400, 11585414, 45934029, 18009898, 11594812, 13760171, 41894550, 13254896, 28025170, 20007524, 13027888 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-03-26") } ] }
-{ "id": 10206877, "id-copy": 10206877, "alias": "Tammie", "name": "TammieBerry", "user-since": datetime("2009-10-14T12:57:11.000Z"), "user-since-copy": datetime("2009-10-14T12:57:11.000Z"), "friend-ids": {{ 23748102, 37944735, 42193629, 11409119, 41246083, 35024235 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2008-05-21") } ] }
-{ "id": 10247557, "id-copy": 10247557, "alias": "Shanita", "name": "ShanitaReed", "user-since": datetime("2006-08-01T23:58:30.000Z"), "user-since-copy": datetime("2006-08-01T23:58:30.000Z"), "friend-ids": {{ 39665727, 7906210, 46234266, 15304695, 4362978, 43689749, 11688287, 11377882, 33955818, 29447417, 23667673, 7373357, 45056089, 34964516, 13871603, 41976105, 10661879, 11112019, 17797460 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2004-12-08"), "end-date": date("2005-04-04") } ] }
-{ "id": 10287028, "id-copy": 10287028, "alias": "Wilfred", "name": "WilfredChurchill", "user-since": datetime("2007-08-01T14:14:25.000Z"), "user-since-copy": datetime("2007-08-01T14:14:25.000Z"), "friend-ids": {{ 38355737, 39891840, 41036196, 39165706, 1155288, 15280633, 9744287, 11567914, 11225763, 2297894, 14386027, 67174, 28097703, 28721858, 6504409, 6743503, 22860419, 17773814, 34697084, 5419586, 45771084 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2002-08-08") } ] }
-{ "id": 10297336, "id-copy": 10297336, "alias": "Gayelord", "name": "GayelordCypret", "user-since": datetime("2005-09-28T10:01:31.000Z"), "user-since-copy": datetime("2005-09-28T10:01:31.000Z"), "friend-ids": {{ 43657472, 21189656, 43018991, 42333420, 27203617, 12389046, 44062328, 15441240, 31806533, 44999377, 30592890, 12304605, 6752099, 9488471, 5719065, 16290550, 23175098, 6432261 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-05-15") } ] }
-{ "id": 10305280, "id-copy": 10305280, "alias": "Isabella", "name": "IsabellaWilo", "user-since": datetime("2007-01-03T11:54:28.000Z"), "user-since-copy": datetime("2007-01-03T11:54:28.000Z"), "friend-ids": {{ 46537100, 26395353, 23044918 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2005-01-06") } ] }
-{ "id": 10317160, "id-copy": 10317160, "alias": "Maria", "name": "MariaHair", "user-since": datetime("2006-05-21T16:06:00.000Z"), "user-since-copy": datetime("2006-05-21T16:06:00.000Z"), "friend-ids": {{ 7063473, 43027344, 2119671, 39231388, 34041933, 5141408, 20278936 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2005-10-20") } ] }
-{ "id": 10318882, "id-copy": 10318882, "alias": "Skyler", "name": "SkylerConrad", "user-since": datetime("2007-03-04T08:56:54.000Z"), "user-since-copy": datetime("2007-03-04T08:56:54.000Z"), "friend-ids": {{ 4254240, 3778434, 23914534, 16376376, 39143316, 37229152, 32778982, 30182686, 13077652, 20439638, 34086734, 12101909, 47011547, 28666460, 31034524, 47508299, 17267782, 1260337, 43500601, 914291, 1786773 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2009-01-15") } ] }
-{ "id": 10338907, "id-copy": 10338907, "alias": "Leah", "name": "LeahStroble", "user-since": datetime("2010-12-07T08:23:00.000Z"), "user-since-copy": datetime("2010-12-07T08:23:00.000Z"), "friend-ids": {{ 25263375, 47112518, 47910837, 14446727, 35708710, 41365949, 8534511, 34992353, 1706302, 21380997, 47197876, 29441929, 4157771, 8674755, 14520863, 22041433, 47176591, 4072306, 47354501 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2003-03-25") } ] }
-{ "id": 10348309, "id-copy": 10348309, "alias": "Bernard", "name": "BernardAltman", "user-since": datetime("2010-09-23T09:08:33.000Z"), "user-since-copy": datetime("2010-09-23T09:08:33.000Z"), "friend-ids": {{ 7859503, 40438517, 7050233, 41735514, 8274833, 12496793, 41853402, 23751827, 23485505, 35520895, 17406459, 20238814, 42333149 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2007-07-27") } ] }
-{ "id": 10349656, "id-copy": 10349656, "alias": "Woodrow", "name": "WoodrowRichter", "user-since": datetime("2006-09-18T16:22:12.000Z"), "user-since-copy": datetime("2006-09-18T16:22:12.000Z"), "friend-ids": {{ 12344306, 36484394, 30889842, 47572749, 42102868, 22350773, 7166034, 16132372, 45197714, 34516830, 47108654, 4399888, 24401048, 32578065, 16593311, 33394001, 7356357, 29943304, 30866764, 11942891 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2003-11-18"), "end-date": date("2004-10-16") } ] }
-{ "id": 10357477, "id-copy": 10357477, "alias": "Rosy", "name": "RosyMitchell", "user-since": datetime("2005-08-13T13:44:24.000Z"), "user-since-copy": datetime("2005-08-13T13:44:24.000Z"), "friend-ids": {{ 13370964, 4479736, 44060098, 28936173, 42239651, 18380035, 17854869, 36485096, 7662833 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-05-12") } ] }
-{ "id": 10391077, "id-copy": 10391077, "alias": "Tracy", "name": "TracyHiles", "user-since": datetime("2005-11-19T21:08:51.000Z"), "user-since-copy": datetime("2005-11-19T21:08:51.000Z"), "friend-ids": {{ 27119048, 1983772, 38766385, 35631268, 14736954, 7586158, 45840742, 27211063, 33946244, 1590669, 22363833, 19668917, 12778790, 31993728, 4498870, 68121, 13591025, 13285639 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2012-07-12") } ] }
-{ "id": 10392898, "id-copy": 10392898, "alias": "Rodger", "name": "RodgerLear", "user-since": datetime("2010-03-05T20:39:12.000Z"), "user-since-copy": datetime("2010-03-05T20:39:12.000Z"), "friend-ids": {{ 23638180, 34355575, 28958329, 17287883, 46069191, 4055459, 36969931, 13059600, 6957015, 41374655, 44549230, 1943320, 39878243 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2002-12-22") } ] }
-{ "id": 10398562, "id-copy": 10398562, "alias": "Brendon", "name": "BrendonMaclagan", "user-since": datetime("2012-02-23T06:18:49.000Z"), "user-since-copy": datetime("2012-02-23T06:18:49.000Z"), "friend-ids": {{ 39206829, 37980663, 36889290, 9114653, 26448451, 15142055, 23349234, 11668644, 22072984, 2091972, 957976, 26110137, 20947598, 32127830, 35850034, 39029675, 21265582, 26725192, 13963111, 4392994, 37042547 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2001-09-21") } ] }
-{ "id": 10422310, "id-copy": 10422310, "alias": "Edmundo", "name": "EdmundoShaw", "user-since": datetime("2012-07-02T11:10:15.000Z"), "user-since-copy": datetime("2012-07-02T11:10:15.000Z"), "friend-ids": {{ 4235436, 16381036, 12579129, 43280339, 16455681, 28445764, 10796826, 28577255, 15173785, 47982248, 11990921, 2093558, 6244669, 4830927, 34859603, 22246754, 45142656 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2011-01-27") } ] }
-{ "id": 10423588, "id-copy": 10423588, "alias": "Shirlene", "name": "ShirleneRuch", "user-since": datetime("2006-04-09T05:52:24.000Z"), "user-since-copy": datetime("2006-04-09T05:52:24.000Z"), "friend-ids": {{ 15418780, 12724265, 27282306, 13592995, 24753166, 32824252, 40619106, 27563604, 12337625, 45387219, 27749581, 44912564, 37470078, 19663516 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2003-06-17") } ] }
-{ "id": 10453837, "id-copy": 10453837, "alias": "Leila", "name": "LeilaHunter", "user-since": datetime("2007-12-08T12:41:34.000Z"), "user-since-copy": datetime("2007-12-08T12:41:34.000Z"), "friend-ids": {{ 2310862, 19014920 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2011-02-06") } ] }
-{ "id": 10472248, "id-copy": 10472248, "alias": "Harry", "name": "HarryDugmore", "user-since": datetime("2012-02-18T05:46:12.000Z"), "user-since-copy": datetime("2012-02-18T05:46:12.000Z"), "friend-ids": {{ 30193978, 30762534, 24660208, 29628319, 30687391, 39795396, 33525293, 23739628, 28969085, 30275276, 3497701, 17091988, 15259527, 25164171, 34052417, 4318314, 1876063, 29984074, 3421436, 16610126 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2012-01-19"), "end-date": date("2012-01-02") } ] }
-{ "id": 10484578, "id-copy": 10484578, "alias": "Troy", "name": "TroyWheeler", "user-since": datetime("2006-12-19T11:23:18.000Z"), "user-since-copy": datetime("2006-12-19T11:23:18.000Z"), "friend-ids": {{ 13536585, 23059550, 16602050, 12025612, 25014410, 13465266 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2011-10-23") } ] }
-{ "id": 10501429, "id-copy": 10501429, "alias": "Danielle", "name": "DanielleYoung", "user-since": datetime("2010-04-24T05:46:06.000Z"), "user-since-copy": datetime("2010-04-24T05:46:06.000Z"), "friend-ids": {{ 7960737, 27505427 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2002-07-27"), "end-date": date("2004-07-28") } ] }
-{ "id": 10505419, "id-copy": 10505419, "alias": "Anderson", "name": "AndersonSoames", "user-since": datetime("2009-04-01T01:24:07.000Z"), "user-since-copy": datetime("2009-04-01T01:24:07.000Z"), "friend-ids": {{ 25420744, 34012676, 8558565, 45471514, 12117008, 35275, 4952379, 46480100, 29394067, 15504329, 18153717, 8476606, 19867236, 35743164, 38523474, 6479207, 31151752, 19687338, 5379846, 32574974, 26920356 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2005-08-01") } ] }
-{ "id": 10508467, "id-copy": 10508467, "alias": "Quincey", "name": "QuinceyKettlewell", "user-since": datetime("2009-11-08T14:09:57.000Z"), "user-since-copy": datetime("2009-11-08T14:09:57.000Z"), "friend-ids": {{ 16037923, 33757766, 22829568, 34589661, 10645853, 43124745, 41785968, 27704416, 42381402, 11993654, 31993782, 37761743, 15571469, 33326934, 22719288, 18321279, 19252211, 42927515, 22390312, 37655021, 37511969, 47740024, 1015876 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2009-02-24") } ] }
-{ "id": 10529809, "id-copy": 10529809, "alias": "Aric", "name": "AricLauffer", "user-since": datetime("2007-05-18T09:08:29.000Z"), "user-since-copy": datetime("2007-05-18T09:08:29.000Z"), "friend-ids": {{ 36647795, 13183862, 5313167, 36450019, 46412788, 47789981, 4012027, 35872968, 3903895 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2011-09-22") } ] }
-{ "id": 10541299, "id-copy": 10541299, "alias": "Derrick", "name": "DerrickLarson", "user-since": datetime("2009-09-04T09:42:12.000Z"), "user-since-copy": datetime("2009-09-04T09:42:12.000Z"), "friend-ids": {{ 39544341, 9620318, 40218798, 34927427, 28533075, 44505091, 29066144, 31724565, 46052997, 3011652, 24709291, 24805644, 41125094, 14186985, 24967210, 32420881, 31162758, 2356654, 11854218, 47933360, 9668743, 26801113 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2000-11-25") } ] }
-{ "id": 10547020, "id-copy": 10547020, "alias": "Reita", "name": "ReitaBlunt", "user-since": datetime("2006-01-18T16:51:49.000Z"), "user-since-copy": datetime("2006-01-18T16:51:49.000Z"), "friend-ids": {{ 34373903, 36464697, 37171525, 19138424, 24675436, 16269152, 43940985, 2735762, 32760257, 42561749, 45516984, 39110107, 21610913, 1805884, 3342035, 40703512, 11665984, 29345992, 41497492, 30054924, 18098215 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-12-01") } ] }
-{ "id": 10548142, "id-copy": 10548142, "alias": "Dannie", "name": "DannieTillson", "user-since": datetime("2007-03-07T04:57:23.000Z"), "user-since-copy": datetime("2007-03-07T04:57:23.000Z"), "friend-ids": {{ 37443492, 21615683, 5655492, 24162015, 46418787, 46328489, 26669127, 38324141 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2012-01-03") } ] }
-{ "id": 10554112, "id-copy": 10554112, "alias": "Virgil", "name": "VirgilBickerson", "user-since": datetime("2006-03-14T07:07:42.000Z"), "user-since-copy": datetime("2006-03-14T07:07:42.000Z"), "friend-ids": {{ 21584501, 3506050, 31062036, 20425233, 6548274, 12613206, 16607156 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2004-08-25"), "end-date": date("2006-11-11") } ] }
-{ "id": 10561624, "id-copy": 10561624, "alias": "Marielle", "name": "MarielleBrandenburg", "user-since": datetime("2005-07-17T10:28:02.000Z"), "user-since-copy": datetime("2005-07-17T10:28:02.000Z"), "friend-ids": {{ 1231477, 14598987 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2000-03-06"), "end-date": date("2005-09-25") } ] }
-{ "id": 10579345, "id-copy": 10579345, "alias": "Rexana", "name": "RexanaSchaeffer", "user-since": datetime("2006-01-20T15:37:57.000Z"), "user-since-copy": datetime("2006-01-20T15:37:57.000Z"), "friend-ids": {{ 20070497, 44547094, 38571608, 30731404, 7825730, 8433351, 25090042, 38943273, 3599029, 28517891, 17427828, 6853394, 32856065, 46627870, 43885788 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-01-22") } ] }
-{ "id": 10580422, "id-copy": 10580422, "alias": "Travers", "name": "TraversSadley", "user-since": datetime("2011-02-09T08:22:49.000Z"), "user-since-copy": datetime("2011-02-09T08:22:49.000Z"), "friend-ids": {{ 36067992, 8651663, 43180149, 732576, 35709545, 30999437 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2009-05-27") } ] }
-{ "id": 10585294, "id-copy": 10585294, "alias": "Bryan", "name": "BryanEliza", "user-since": datetime("2005-02-03T16:20:19.000Z"), "user-since-copy": datetime("2005-02-03T16:20:19.000Z"), "friend-ids": {{ 6407647, 24838863, 45997254, 42728806, 37001718, 46932382 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-03-15"), "end-date": date("2008-04-24") } ] }
-{ "id": 10587655, "id-copy": 10587655, "alias": "Del", "name": "DelLester", "user-since": datetime("2006-04-22T06:14:51.000Z"), "user-since-copy": datetime("2006-04-22T06:14:51.000Z"), "friend-ids": {{ 41382268, 41043817, 37053482, 27889226, 5182442, 46241085, 39510378, 25972421, 6234359, 2782513, 27042023, 20476198 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2001-10-18") } ] }
-{ "id": 10591498, "id-copy": 10591498, "alias": "Mick", "name": "MickVeith", "user-since": datetime("2006-02-21T06:58:53.000Z"), "user-since-copy": datetime("2006-02-21T06:58:53.000Z"), "friend-ids": {{ 33872347, 40692511, 18563650 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2011-08-07"), "end-date": date("2011-01-10") } ] }
-{ "id": 10595164, "id-copy": 10595164, "alias": "Jerome", "name": "JeromeLacon", "user-since": datetime("2009-09-24T09:47:36.000Z"), "user-since-copy": datetime("2009-09-24T09:47:36.000Z"), "friend-ids": {{ 31538601 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-07-26") } ] }
-{ "id": 10601758, "id-copy": 10601758, "alias": "Blossom", "name": "BlossomClark", "user-since": datetime("2011-08-16T23:44:16.000Z"), "user-since-copy": datetime("2011-08-16T23:44:16.000Z"), "friend-ids": {{ 22624576, 6945784, 47816004, 8072206, 23953052, 22668193, 8668574, 2269602, 39137309, 38996903, 23516086, 31166264, 28322741, 46296094, 36547681, 7287738, 15727604, 13556387, 2624138 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-10-15"), "end-date": date("2008-07-17") } ] }
-{ "id": 10610356, "id-copy": 10610356, "alias": "Jason", "name": "JasonGearhart", "user-since": datetime("2010-03-05T22:57:20.000Z"), "user-since-copy": datetime("2010-03-05T22:57:20.000Z"), "friend-ids": {{ 6967239, 47468231, 29517365, 9206260 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-03-16"), "end-date": date("2012-06-19") } ] }
-{ "id": 10624381, "id-copy": 10624381, "alias": "Ryana", "name": "RyanaKimmons", "user-since": datetime("2007-09-04T15:42:08.000Z"), "user-since-copy": datetime("2007-09-04T15:42:08.000Z"), "friend-ids": {{ 36219003, 5135252, 24653726, 4767631, 21595268, 4154414, 31857818, 9711256, 20793102, 14509650 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2006-06-21") } ] }
-{ "id": 10635319, "id-copy": 10635319, "alias": "Rusty", "name": "RustyStange", "user-since": datetime("2010-08-17T17:30:37.000Z"), "user-since-copy": datetime("2010-08-17T17:30:37.000Z"), "friend-ids": {{ 28180565, 25608756 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2006-07-07") } ] }
-{ "id": 10658977, "id-copy": 10658977, "alias": "Danny", "name": "DannyBailey", "user-since": datetime("2006-12-12T12:28:17.000Z"), "user-since-copy": datetime("2006-12-12T12:28:17.000Z"), "friend-ids": {{ 27744791, 5839976, 37243832, 42061553, 15660549, 26723434, 25864049, 8038100, 47690286, 29206337, 6169296, 1933137, 6500848, 45632949, 6329147, 15602171, 13477556, 25033716, 9515038, 4081408, 42840830 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2009-12-14"), "end-date": date("2009-03-11") } ] }
-{ "id": 10671115, "id-copy": 10671115, "alias": "Montague", "name": "MontagueLangston", "user-since": datetime("2007-09-20T00:32:15.000Z"), "user-since-copy": datetime("2007-09-20T00:32:15.000Z"), "friend-ids": {{ 18236000, 47490167, 40246549, 25232933, 22604487, 36974958, 44747862, 2137180, 39244601, 39608406, 23319330, 21166788, 21726220, 12703943, 36564459, 8379538, 43010567, 24538004, 173522, 6132291, 21199763, 26285128, 2350066 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2001-01-07") } ] }
-{ "id": 10678567, "id-copy": 10678567, "alias": "Detta", "name": "DettaIronmonger", "user-since": datetime("2006-05-01T08:52:26.000Z"), "user-since-copy": datetime("2006-05-01T08:52:26.000Z"), "friend-ids": {{ 11098679, 15763619, 12715761, 10175990, 43581466, 4595173, 17163835, 44918467, 38256765, 13239047, 25476309, 9075112, 19581524, 46478013, 24168854, 34121818, 25604978, 21114089 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2007-06-08") } ] }
-{ "id": 10690066, "id-copy": 10690066, "alias": "Abraham", "name": "AbrahamWardle", "user-since": datetime("2006-04-08T20:27:10.000Z"), "user-since-copy": datetime("2006-04-08T20:27:10.000Z"), "friend-ids": {{ 18105973, 39839261, 27532181, 2565949, 37077592, 28929530 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2005-06-03"), "end-date": date("2006-12-02") } ] }
-{ "id": 10701727, "id-copy": 10701727, "alias": "Paulita", "name": "PaulitaHays", "user-since": datetime("2009-11-15T15:25:08.000Z"), "user-since-copy": datetime("2009-11-15T15:25:08.000Z"), "friend-ids": {{ 31869253, 13336594, 19116516, 30920596 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2001-12-10") } ] }
-{ "id": 10708477, "id-copy": 10708477, "alias": "Zacharias", "name": "ZachariasRandolph", "user-since": datetime("2008-07-13T16:12:33.000Z"), "user-since-copy": datetime("2008-07-13T16:12:33.000Z"), "friend-ids": {{ 18251027, 47694844, 25569678, 33130234, 7351010, 32617025, 40619749, 28576965, 34970660, 34320919, 17056847, 46007935, 244756, 3130710, 5218614, 6968874, 19440356, 448790, 3336700, 44725864, 24738046, 6159443, 14380294, 20289778 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2007-09-09") } ] }
-{ "id": 10721059, "id-copy": 10721059, "alias": "Amandine", "name": "AmandineRockwell", "user-since": datetime("2008-09-24T21:50:39.000Z"), "user-since-copy": datetime("2008-09-24T21:50:39.000Z"), "friend-ids": {{ 10360854, 15197739, 28812340, 12172446, 9354363, 23580760, 6364957, 20048548 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2003-12-06") } ] }
-{ "id": 10733305, "id-copy": 10733305, "alias": "Dakota", "name": "DakotaSmith", "user-since": datetime("2009-11-17T19:52:42.000Z"), "user-since-copy": datetime("2009-11-17T19:52:42.000Z"), "friend-ids": {{ 21984282, 14492326, 18724474, 17361116, 26773641, 32118673, 8295454, 6804824 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2007-05-28") } ] }
-{ "id": 10733617, "id-copy": 10733617, "alias": "Leonardo", "name": "LeonardoKight", "user-since": datetime("2008-10-20T17:30:29.000Z"), "user-since-copy": datetime("2008-10-20T17:30:29.000Z"), "friend-ids": {{ 39687903, 7235506, 34696496, 25995345, 18435380, 47473591, 15710408, 44232442, 39520147, 36384026, 25160887, 245860, 1195579, 4587411, 536916, 47052672, 33953823, 13203710 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2007-07-12"), "end-date": date("2010-03-16") } ] }
-{ "id": 10738477, "id-copy": 10738477, "alias": "Kenith", "name": "KenithLeichter", "user-since": datetime("2012-07-10T15:21:51.000Z"), "user-since-copy": datetime("2012-07-10T15:21:51.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2006-07-28"), "end-date": date("2009-06-03") } ] }
-{ "id": 10742182, "id-copy": 10742182, "alias": "Tel", "name": "TelBowchiew", "user-since": datetime("2009-09-23T02:51:14.000Z"), "user-since-copy": datetime("2009-09-23T02:51:14.000Z"), "friend-ids": {{ 17515416, 42010238, 23580669, 26008148, 35744494 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2006-10-05"), "end-date": date("2007-05-26") } ] }
-{ "id": 10745200, "id-copy": 10745200, "alias": "Kaety", "name": "KaetyOppenheimer", "user-since": datetime("2008-11-21T08:11:11.000Z"), "user-since-copy": datetime("2008-11-21T08:11:11.000Z"), "friend-ids": {{ 32006369, 4542624, 28242708, 20936957, 11063561, 31392192, 34444041, 754368, 37317926 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-06-07") } ] }
-{ "id": 10745974, "id-copy": 10745974, "alias": "Gavin", "name": "GavinWard", "user-since": datetime("2008-11-23T02:59:13.000Z"), "user-since-copy": datetime("2008-11-23T02:59:13.000Z"), "friend-ids": {{ 45290227, 46308273, 4478698, 27613190, 34907694, 36182643 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-01-01"), "end-date": date("2011-01-17") } ] }
-{ "id": 10751260, "id-copy": 10751260, "alias": "Chrysanta", "name": "ChrysantaSanforth", "user-since": datetime("2009-06-02T12:54:32.000Z"), "user-since-copy": datetime("2009-06-02T12:54:32.000Z"), "friend-ids": {{ 6064707, 44017707, 22957433, 38426343, 24694205, 1061085, 24827089, 12192854, 40718843 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2011-01-19"), "end-date": date("2011-10-02") } ] }
-{ "id": 10767553, "id-copy": 10767553, "alias": "Titty", "name": "TittyCross", "user-since": datetime("2009-02-08T11:38:56.000Z"), "user-since-copy": datetime("2009-02-08T11:38:56.000Z"), "friend-ids": {{ 10869392, 39422025, 23051606, 43241994, 6257807, 37258783, 26946341, 33120713, 6481181, 13410766, 34576024, 42401239, 28793792, 37331232, 5979767 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2000-12-26"), "end-date": date("2006-01-17") } ] }
-{ "id": 10777441, "id-copy": 10777441, "alias": "Rosaline", "name": "RosalineFaast", "user-since": datetime("2005-05-23T08:24:59.000Z"), "user-since-copy": datetime("2005-05-23T08:24:59.000Z"), "friend-ids": {{ 25088415, 36453219, 42450810, 6845863, 23568088, 34305276, 28849557, 41593223, 18542045, 37652004, 9159129, 42079452 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-01-04") } ] }
-{ "id": 10786129, "id-copy": 10786129, "alias": "Ardelle", "name": "ArdelleHoopengarner", "user-since": datetime("2012-05-27T08:36:37.000Z"), "user-since-copy": datetime("2012-05-27T08:36:37.000Z"), "friend-ids": {{ 44854493, 13697746, 8918104, 22353878, 46059542, 23393155, 37374548, 1531344, 31554501, 30390740, 10076243, 19028830, 46174212, 4991316, 30988902, 6717568 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-03-08") } ] }
-{ "id": 10789207, "id-copy": 10789207, "alias": "Lucinda", "name": "LucindaFillmore", "user-since": datetime("2009-11-13T18:35:41.000Z"), "user-since-copy": datetime("2009-11-13T18:35:41.000Z"), "friend-ids": {{ 10917581, 24902161, 29393856, 35293349, 31477965, 44139676, 18083704, 46487557 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2005-11-04") } ] }
-{ "id": 10808932, "id-copy": 10808932, "alias": "Sharita", "name": "SharitaGregory", "user-since": datetime("2006-09-17T04:48:23.000Z"), "user-since-copy": datetime("2006-09-17T04:48:23.000Z"), "friend-ids": {{ 41622567, 16559791, 6346693, 18540237, 14753253, 23252825, 17163196, 46962665, 26442426, 14344279, 17332246, 36154890, 22814241, 22709064, 32887290, 42853122, 23782934, 27425228, 22941847 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2008-06-08"), "end-date": date("2011-01-28") } ] }
-{ "id": 10809322, "id-copy": 10809322, "alias": "Alden", "name": "AldenHiggens", "user-since": datetime("2011-02-06T01:31:58.000Z"), "user-since-copy": datetime("2011-02-06T01:31:58.000Z"), "friend-ids": {{ 44750450, 24564153, 42513064, 33316253, 21036452, 27132567, 29231674, 18040424, 36564417, 17474605, 14126628, 18988855, 35594147, 35685289, 40967850 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2002-09-26") } ] }
-{ "id": 10811875, "id-copy": 10811875, "alias": "Giovanni", "name": "GiovanniWarner", "user-since": datetime("2009-05-28T04:20:11.000Z"), "user-since-copy": datetime("2009-05-28T04:20:11.000Z"), "friend-ids": {{ 8005226, 21432611, 4037183, 40486007, 40666777, 24385549, 3686021, 12188144, 33646224, 46365125, 44351069, 34408172, 35904411, 4322876, 18767645, 10007322 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2005-07-18"), "end-date": date("2011-10-24") } ] }
-{ "id": 10832305, "id-copy": 10832305, "alias": "Briony", "name": "BrionyBaldwin", "user-since": datetime("2011-03-03T22:00:38.000Z"), "user-since-copy": datetime("2011-03-03T22:00:38.000Z"), "friend-ids": {{ 20436897, 36519715, 35325917, 31686319, 2644929, 3401668, 39344422, 18601722, 40274111, 30032679, 9312830, 5581755, 41164101, 35883066, 8274432, 4315219, 26200418, 43810182, 44718149, 6387153, 43086214, 39558538, 36036905, 25715671 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2007-01-21"), "end-date": date("2008-02-25") } ] }
-{ "id": 10833472, "id-copy": 10833472, "alias": "Monica", "name": "MonicaRyals", "user-since": datetime("2009-02-14T18:52:57.000Z"), "user-since-copy": datetime("2009-02-14T18:52:57.000Z"), "friend-ids": {{ 34417058, 24053823, 28067368, 16205470, 24168710, 9064471 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2003-12-03"), "end-date": date("2006-03-07") } ] }
-{ "id": 10840990, "id-copy": 10840990, "alias": "Libby", "name": "LibbyHayhurst", "user-since": datetime("2009-10-28T22:52:04.000Z"), "user-since-copy": datetime("2009-10-28T22:52:04.000Z"), "friend-ids": {{ 32146321, 47850956, 42432761, 28856789, 18595962, 23408710, 37015546 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2003-01-20") } ] }
-{ "id": 10867624, "id-copy": 10867624, "alias": "Fredric", "name": "FredricKimmons", "user-since": datetime("2005-05-14T23:08:00.000Z"), "user-since-copy": datetime("2005-05-14T23:08:00.000Z"), "friend-ids": {{ 25574899, 26822046, 3408550, 40738004, 3813112, 33045116, 9229839, 28557630, 36781441, 23585776 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2010-02-25"), "end-date": date("2011-07-06") } ] }
-{ "id": 10868761, "id-copy": 10868761, "alias": "Peronel", "name": "PeronelGongaware", "user-since": datetime("2010-01-25T14:26:30.000Z"), "user-since-copy": datetime("2010-01-25T14:26:30.000Z"), "friend-ids": {{ 28271989, 41567995, 31926358, 16420360, 15775849, 44023747, 39099521, 4517209, 39890594, 39784644, 43247769, 25427216, 46426794, 37704581, 46477208, 3213706 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2011-12-16") } ] }
-{ "id": 10869727, "id-copy": 10869727, "alias": "Jacquetta", "name": "JacquettaMaugham", "user-since": datetime("2010-07-11T22:43:19.000Z"), "user-since-copy": datetime("2010-07-11T22:43:19.000Z"), "friend-ids": {{ 36109878, 46889968, 19648550, 14051620, 14645938, 14933447, 33880415 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2002-12-09") } ] }
-{ "id": 10874791, "id-copy": 10874791, "alias": "Haydee", "name": "HaydeeGarratt", "user-since": datetime("2007-04-14T00:19:00.000Z"), "user-since-copy": datetime("2007-04-14T00:19:00.000Z"), "friend-ids": {{ 12247794, 10306863, 33161811, 43877113, 37745696 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2008-03-07"), "end-date": date("2011-12-27") } ] }
-{ "id": 10894411, "id-copy": 10894411, "alias": "Lacy", "name": "LacyShaw", "user-since": datetime("2006-04-06T00:11:24.000Z"), "user-since-copy": datetime("2006-04-06T00:11:24.000Z"), "friend-ids": {{ 4203591, 28370134, 5239468, 12951448, 39355113, 9126812, 5662652, 4633221, 11954172, 33269236, 11545355, 14018236, 21980886, 34750979, 22877356 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-04-12") } ] }
-{ "id": 10911220, "id-copy": 10911220, "alias": "Laurice", "name": "LauriceDuncan", "user-since": datetime("2008-08-05T15:55:34.000Z"), "user-since-copy": datetime("2008-08-05T15:55:34.000Z"), "friend-ids": {{ 212109 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2001-02-03") } ] }
-{ "id": 10911274, "id-copy": 10911274, "alias": "Bridgette", "name": "BridgetteBenford", "user-since": datetime("2007-02-15T06:18:45.000Z"), "user-since-copy": datetime("2007-02-15T06:18:45.000Z"), "friend-ids": {{ 10909520, 14433605 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2012-01-14") } ] }
-{ "id": 10915261, "id-copy": 10915261, "alias": "Lyle", "name": "LyleMuller", "user-since": datetime("2010-10-16T16:36:46.000Z"), "user-since-copy": datetime("2010-10-16T16:36:46.000Z"), "friend-ids": {{ 28409003, 7495999, 10776059, 23825626, 44321306, 15679301, 36736470, 24070644, 14041140, 4784196, 19462533, 47300197, 33544003 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-09-25") } ] }
-{ "id": 10925071, "id-copy": 10925071, "alias": "Gil", "name": "GilFocell", "user-since": datetime("2005-11-08T20:28:01.000Z"), "user-since-copy": datetime("2005-11-08T20:28:01.000Z"), "friend-ids": {{ 9416716, 42743353, 43396785, 44271346, 32924780, 44752785, 19741326, 39315503, 25154503, 29170056, 15457515, 14764269, 47861907, 15230067, 15326613, 6336542, 44127013, 1048087, 34624221, 19951452, 12778135 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2009-01-07") } ] }
-{ "id": 10936273, "id-copy": 10936273, "alias": "Hans", "name": "HansMench", "user-since": datetime("2008-08-08T12:00:48.000Z"), "user-since-copy": datetime("2008-08-08T12:00:48.000Z"), "friend-ids": {{ 36800139 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2005-04-15"), "end-date": date("2009-08-05") } ] }
-{ "id": 10937893, "id-copy": 10937893, "alias": "Katheleen", "name": "KatheleenEisenmann", "user-since": datetime("2012-06-17T05:15:08.000Z"), "user-since-copy": datetime("2012-06-17T05:15:08.000Z"), "friend-ids": {{ 30129247, 865896, 35091601, 19852276, 43238329, 46057691, 30405091, 3723169, 6577863, 12648596, 34726408, 19178848, 18365491, 28604299, 29242262, 12826786, 19046213, 23320700, 9318080, 35996590, 24812162, 9639554, 33615920, 6507511 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2006-07-26") } ] }
-{ "id": 10940377, "id-copy": 10940377, "alias": "Lory", "name": "LoryElless", "user-since": datetime("2011-03-21T19:07:17.000Z"), "user-since-copy": datetime("2011-03-21T19:07:17.000Z"), "friend-ids": {{ 38950352, 10596357, 43176277, 27274342, 27082326 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2007-04-19") } ] }
-{ "id": 10948003, "id-copy": 10948003, "alias": "August", "name": "AugustHatch", "user-since": datetime("2006-04-11T03:32:56.000Z"), "user-since-copy": datetime("2006-04-11T03:32:56.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2008-12-16"), "end-date": date("2009-01-21") } ] }
-{ "id": 10968562, "id-copy": 10968562, "alias": "Fox", "name": "FoxBillimek", "user-since": datetime("2012-03-24T07:32:17.000Z"), "user-since-copy": datetime("2012-03-24T07:32:17.000Z"), "friend-ids": {{ 8459327, 11505750, 30952882, 30467951, 6329439, 33947538, 19579432, 25135787, 41391398, 32456626, 6310287, 31211659 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2009-01-17") } ] }
-{ "id": 11016043, "id-copy": 11016043, "alias": "Ellis", "name": "EllisVorrasi", "user-since": datetime("2009-08-26T16:43:17.000Z"), "user-since-copy": datetime("2009-08-26T16:43:17.000Z"), "friend-ids": {{ 41000811, 12639978, 14487796, 39651858, 40189282, 7834125, 44416511, 28673665 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2008-01-21"), "end-date": date("2008-04-26") } ] }
-{ "id": 11027953, "id-copy": 11027953, "alias": "Angelika", "name": "AngelikaSanner", "user-since": datetime("2010-10-07T04:25:19.000Z"), "user-since-copy": datetime("2010-10-07T04:25:19.000Z"), "friend-ids": {{ 42662440, 6358862, 21758734, 28882210, 28157558, 39027509, 19068795, 45387055, 34737892, 32277859, 44713546, 24617807, 31067294, 12307376, 28568916, 31114183, 13997610, 15405045, 33587810, 32517419, 13452101, 8309328 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2006-02-25") } ] }
-{ "id": 11052748, "id-copy": 11052748, "alias": "Andriana", "name": "AndrianaYonkie", "user-since": datetime("2005-05-08T19:49:03.000Z"), "user-since-copy": datetime("2005-05-08T19:49:03.000Z"), "friend-ids": {{ 24372868, 41932219, 14088659, 33215970, 34384197, 16343164, 24230672, 20937997, 23129922, 33184913, 25421373, 12081379, 289577, 19330874, 31625333, 34885607, 34353478, 17694263, 34819024, 44837603 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2003-06-16"), "end-date": date("2008-02-15") } ] }
-{ "id": 11062330, "id-copy": 11062330, "alias": "Derick", "name": "DerickPennington", "user-since": datetime("2008-04-15T11:59:52.000Z"), "user-since-copy": datetime("2008-04-15T11:59:52.000Z"), "friend-ids": {{ 26471368, 22445928, 13709179, 16677606, 45234923, 5601330, 16510085, 27673980, 24365707, 42647605, 20473849, 40448252, 37480913, 38532114, 11022656, 799537, 38469920, 1291033, 31503804, 29154535, 5506108, 24609403, 35535409, 44197253 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-09-23") } ] }
-{ "id": 11066710, "id-copy": 11066710, "alias": "Caryl", "name": "CarylMaugham", "user-since": datetime("2007-02-10T03:38:03.000Z"), "user-since-copy": datetime("2007-02-10T03:38:03.000Z"), "friend-ids": {{ 41776362, 7370825, 35851510, 23733011, 27617379, 39377372, 3043067, 22122576, 11996852, 20708849, 40772627, 20108470, 4141780, 3724555, 31849764, 7347633 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2001-10-15") } ] }
-{ "id": 11087224, "id-copy": 11087224, "alias": "Zola", "name": "ZolaKnisely", "user-since": datetime("2005-11-18T05:30:00.000Z"), "user-since-copy": datetime("2005-11-18T05:30:00.000Z"), "friend-ids": {{ 6324130, 38065951, 14950455, 27869167, 32957819, 11157656, 10411400, 18072233, 35246039, 35345326, 23217009, 13495953, 18987122 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-05-04") } ] }
-{ "id": 11087839, "id-copy": 11087839, "alias": "Manfred", "name": "ManfredEdwards", "user-since": datetime("2009-10-01T09:12:15.000Z"), "user-since-copy": datetime("2009-10-01T09:12:15.000Z"), "friend-ids": {{ 7828089 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-07-25") } ] }
-{ "id": 11111890, "id-copy": 11111890, "alias": "Geordie", "name": "GeordieGraff", "user-since": datetime("2006-02-12T04:30:44.000Z"), "user-since-copy": datetime("2006-02-12T04:30:44.000Z"), "friend-ids": {{ 12852237, 10391003, 37679153, 6620205, 25381043, 19805548, 4534765, 11626709, 47369482, 15045527, 25177819, 15113002, 39634176, 40637870, 47662386, 8045236 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2005-06-18") } ] }
-{ "id": 11116594, "id-copy": 11116594, "alias": "Norwood", "name": "NorwoodErrett", "user-since": datetime("2008-10-04T16:36:27.000Z"), "user-since-copy": datetime("2008-10-04T16:36:27.000Z"), "friend-ids": {{ 30996403, 30788997, 22512789, 35425088, 12096858, 21391496, 41281428, 15854003, 47041757, 31205204, 36849089, 43015828, 27098245, 46735331, 9520980, 34482257, 36898055, 8962397 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2002-05-20") } ] }
-{ "id": 11117371, "id-copy": 11117371, "alias": "Jules", "name": "JulesRichardson", "user-since": datetime("2009-12-06T06:21:58.000Z"), "user-since-copy": datetime("2009-12-06T06:21:58.000Z"), "friend-ids": {{ 75701, 18653454, 5088871, 20583891, 46460448, 19742484, 2433030, 30869605, 9273775, 6556358 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2001-09-17"), "end-date": date("2006-06-05") } ] }
-{ "id": 11136910, "id-copy": 11136910, "alias": "Karl", "name": "KarlGarratt", "user-since": datetime("2006-12-22T01:58:50.000Z"), "user-since-copy": datetime("2006-12-22T01:58:50.000Z"), "friend-ids": {{ 753124, 31382435, 30698735, 25951267, 27027532, 34551403, 9451765, 37517863, 3719825, 37613952, 18670991, 39783690, 6592095, 27477830, 31739951, 24458195, 12317249 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-05-11") } ] }
-{ "id": 11147392, "id-copy": 11147392, "alias": "Sarina", "name": "SarinaFlickinger", "user-since": datetime("2011-09-26T12:41:56.000Z"), "user-since-copy": datetime("2011-09-26T12:41:56.000Z"), "friend-ids": {{ 17776087, 9254087, 14735666, 31097664, 36421253, 12595115, 40366588, 9491701, 29725314, 38852857, 46206259, 39281843, 36268114, 29939350, 804107, 36307361, 30999436, 47369074, 3820973, 46362092, 36413930, 8807546, 30260636, 15069463 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2005-03-15") } ] }
-{ "id": 11155816, "id-copy": 11155816, "alias": "Titty", "name": "TittyOneal", "user-since": datetime("2009-06-01T06:21:44.000Z"), "user-since-copy": datetime("2009-06-01T06:21:44.000Z"), "friend-ids": {{ 37016026, 32220220, 47720886, 10358045, 7678433, 22148913, 18800507, 17043803, 29852152, 11426875, 44761613, 32002053, 14686180, 26744098, 34991446, 38818677, 24977770 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2012-05-11"), "end-date": date("2012-05-08") } ] }
-{ "id": 11214976, "id-copy": 11214976, "alias": "Maxwell", "name": "MaxwellBailey", "user-since": datetime("2005-11-25T15:01:26.000Z"), "user-since-copy": datetime("2005-11-25T15:01:26.000Z"), "friend-ids": {{ 22027101, 5782023, 46909646, 27593651, 31079804, 31989634, 7337526, 34757530, 32792041 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2011-04-04") } ] }
-{ "id": 11241523, "id-copy": 11241523, "alias": "Gareth", "name": "GarethFylbrigg", "user-since": datetime("2011-01-05T16:02:25.000Z"), "user-since-copy": datetime("2011-01-05T16:02:25.000Z"), "friend-ids": {{ 45629812, 20113715, 13556523, 29410246, 37849964, 33688575, 35713924, 21492453, 32324177, 5765413, 4491937, 1592640, 2809253, 45152094, 36330032, 25347157, 199553, 16471761, 16621535, 20674800, 42682300, 11354218, 4830164 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2005-10-27"), "end-date": date("2005-12-10") } ] }
-{ "id": 11244283, "id-copy": 11244283, "alias": "Erica", "name": "EricaTilton", "user-since": datetime("2005-12-10T16:37:41.000Z"), "user-since-copy": datetime("2005-12-10T16:37:41.000Z"), "friend-ids": {{ 9476551, 22631836, 44127713, 32391437, 19413944, 4263930, 17603111, 24077268, 31120069, 30869992, 6040985, 3918705, 17640663, 22515182 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2002-02-05"), "end-date": date("2003-07-03") } ] }
-{ "id": 11246161, "id-copy": 11246161, "alias": "Jemima", "name": "JemimaJube", "user-since": datetime("2009-10-13T13:44:48.000Z"), "user-since-copy": datetime("2009-10-13T13:44:48.000Z"), "friend-ids": {{ 35264732, 26686176, 37947249, 9511009, 20544975, 21318354, 2417039, 15051823, 23702057, 34446389, 15435804, 42646090, 14791709 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2012-02-26") } ] }
-{ "id": 11253043, "id-copy": 11253043, "alias": "Joye", "name": "JoyeGadow", "user-since": datetime("2005-10-03T17:22:30.000Z"), "user-since-copy": datetime("2005-10-03T17:22:30.000Z"), "friend-ids": {{ 24978234, 7896483, 14560795, 18402417, 16619973, 5852675, 29679362, 19344221, 33721635, 14137068, 30581619, 9715250, 10966922, 24167091, 36509340 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2011-01-08"), "end-date": date("2011-08-10") } ] }
-{ "id": 11270020, "id-copy": 11270020, "alias": "Ursula", "name": "UrsulaSauter", "user-since": datetime("2006-09-17T06:18:31.000Z"), "user-since-copy": datetime("2006-09-17T06:18:31.000Z"), "friend-ids": {{ 13370394, 5537385, 6651824, 27208272, 3304500, 26518061, 44906267, 27803333, 8618582, 22074752, 20865682, 15343007 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2006-08-01") } ] }
-{ "id": 11271517, "id-copy": 11271517, "alias": "Amaryllis", "name": "AmaryllisNewlove", "user-since": datetime("2009-06-10T04:18:11.000Z"), "user-since-copy": datetime("2009-06-10T04:18:11.000Z"), "friend-ids": {{ 6594489, 17958014, 4087759, 38993546, 1741537, 8374107, 30133658, 33873746 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2004-12-21"), "end-date": date("2011-08-19") } ] }
-{ "id": 11273587, "id-copy": 11273587, "alias": "Timmy", "name": "TimmyBishop", "user-since": datetime("2011-11-08T13:46:03.000Z"), "user-since-copy": datetime("2011-11-08T13:46:03.000Z"), "friend-ids": {{ 42987870, 44400071, 27388256, 10579275, 12546323, 23276512, 382419, 4466999, 8068553, 33814105, 14872828, 35038629, 43462816, 44037440 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2001-08-08") } ] }
-{ "id": 11276305, "id-copy": 11276305, "alias": "Salome", "name": "SalomeGongaware", "user-since": datetime("2007-06-05T10:15:14.000Z"), "user-since-copy": datetime("2007-06-05T10:15:14.000Z"), "friend-ids": {{ 17354378, 35576200, 42905756, 44408264, 45572153, 18424890, 39234162, 42837501, 38464194, 45237502, 30396078, 16316605, 32231800, 35417394, 32796520, 13885091, 31520983, 4624403, 18144193, 45707906, 8211336, 2864876 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2002-03-16") } ] }
-{ "id": 11290870, "id-copy": 11290870, "alias": "Lanford", "name": "LanfordOsteen", "user-since": datetime("2009-03-04T15:04:12.000Z"), "user-since-copy": datetime("2009-03-04T15:04:12.000Z"), "friend-ids": {{ 4397941, 36140649, 12796618, 18235191, 8810154, 10521988, 6580979, 29578654, 46083953, 30113784, 25952539 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2009-08-06") } ] }
-{ "id": 11307037, "id-copy": 11307037, "alias": "Brett", "name": "BrettLeichter", "user-since": datetime("2011-02-24T01:38:23.000Z"), "user-since-copy": datetime("2011-02-24T01:38:23.000Z"), "friend-ids": {{ 16273758, 36959770, 26721660 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2011-11-23") } ] }
-{ "id": 11327029, "id-copy": 11327029, "alias": "Mallory", "name": "MalloryHughes", "user-since": datetime("2007-08-06T22:11:46.000Z"), "user-since-copy": datetime("2007-08-06T22:11:46.000Z"), "friend-ids": {{ 38924183, 22042572, 21014848, 46309217, 1120998, 19755064, 4413438, 38855205, 17626985, 5727472, 1293238 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2006-02-28"), "end-date": date("2006-08-24") } ] }
-{ "id": 11341747, "id-copy": 11341747, "alias": "Margaux", "name": "MargauxBynum", "user-since": datetime("2009-01-16T19:54:27.000Z"), "user-since-copy": datetime("2009-01-16T19:54:27.000Z"), "friend-ids": {{ 27056110, 1770280, 17190314, 18164827, 32684926, 32410281, 27173037, 16864868, 4664026, 31170366, 4296651 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2008-08-20") } ] }
-{ "id": 11362531, "id-copy": 11362531, "alias": "Garey", "name": "GareyChapman", "user-since": datetime("2005-10-13T04:24:29.000Z"), "user-since-copy": datetime("2005-10-13T04:24:29.000Z"), "friend-ids": {{ 20693565, 18896854, 17118168, 12285534, 21434048, 15453439, 42734432, 3627967, 30464042, 11556192, 22808282, 464074, 28100870, 29887664, 19046987, 34996619, 39964690, 22574200, 29497238 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2001-03-05") } ] }
-{ "id": 11366131, "id-copy": 11366131, "alias": "Cayley", "name": "CayleyGronko", "user-since": datetime("2005-03-06T13:24:19.000Z"), "user-since-copy": datetime("2005-03-06T13:24:19.000Z"), "friend-ids": {{ 26623267, 47792710, 27975124, 19721566, 45092752, 32954140, 25835098 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2006-09-14"), "end-date": date("2010-06-02") } ] }
-{ "id": 11378911, "id-copy": 11378911, "alias": "Courtney", "name": "CourtneyBashline", "user-since": datetime("2010-10-21T06:13:06.000Z"), "user-since-copy": datetime("2010-10-21T06:13:06.000Z"), "friend-ids": {{ 19627264, 13699162 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2002-06-21") } ] }
-{ "id": 11380807, "id-copy": 11380807, "alias": "Mckinley", "name": "MckinleyGeyer", "user-since": datetime("2008-02-17T13:01:21.000Z"), "user-since-copy": datetime("2008-02-17T13:01:21.000Z"), "friend-ids": {{ 16655526, 20048717, 15998744, 39702027, 28153175, 40825599, 38372618 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2010-11-26") } ] }
-{ "id": 11386210, "id-copy": 11386210, "alias": "Dale", "name": "DaleGreenwood", "user-since": datetime("2007-04-17T19:02:45.000Z"), "user-since-copy": datetime("2007-04-17T19:02:45.000Z"), "friend-ids": {{ 3669916 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2002-09-11") } ] }
-{ "id": 11404780, "id-copy": 11404780, "alias": "Carol", "name": "CarolCox", "user-since": datetime("2009-07-07T23:58:07.000Z"), "user-since-copy": datetime("2009-07-07T23:58:07.000Z"), "friend-ids": {{ 41450896, 12332484, 18515318, 39039576, 2336271, 47313837, 4655597, 40110200, 7357446, 24291515, 8898678, 28911118, 20372890, 1296082, 42558011, 5719716, 6830197 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2005-01-14") } ] }
-{ "id": 11412382, "id-copy": 11412382, "alias": "Gosse", "name": "GosseSutton", "user-since": datetime("2011-01-07T02:19:16.000Z"), "user-since-copy": datetime("2011-01-07T02:19:16.000Z"), "friend-ids": {{ 25790586, 42348812, 39275252, 32764855, 11642271, 15982736, 21971689, 13168697, 38246675, 40514837, 20840965 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2010-12-18"), "end-date": date("2011-01-09") } ] }
-{ "id": 11412640, "id-copy": 11412640, "alias": "Larry", "name": "LarryEisaman", "user-since": datetime("2005-04-23T10:38:04.000Z"), "user-since-copy": datetime("2005-04-23T10:38:04.000Z"), "friend-ids": {{ 15063821, 35006785, 18241384, 5967937, 45426140, 44234765, 3244540, 3222784, 36330320 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2001-07-05") } ] }
-{ "id": 11415055, "id-copy": 11415055, "alias": "Zavia", "name": "ZaviaLombardi", "user-since": datetime("2006-01-10T02:11:24.000Z"), "user-since-copy": datetime("2006-01-10T02:11:24.000Z"), "friend-ids": {{ 25953753, 952678, 31067065 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2003-06-27"), "end-date": date("2010-07-02") } ] }
-{ "id": 11427397, "id-copy": 11427397, "alias": "Oscar", "name": "OscarMillhouse", "user-since": datetime("2012-04-07T04:52:39.000Z"), "user-since-copy": datetime("2012-04-07T04:52:39.000Z"), "friend-ids": {{ 27577077, 26831616, 24024317, 24669981, 15864715, 41688094, 25689775, 19288762, 25015698, 24343183, 30170416, 39881555, 29378159, 6748762, 45948007 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2012-05-15") } ] }
-{ "id": 11428300, "id-copy": 11428300, "alias": "Major", "name": "MajorGreenawalt", "user-since": datetime("2006-12-02T06:43:13.000Z"), "user-since-copy": datetime("2006-12-02T06:43:13.000Z"), "friend-ids": {{ 8021918, 4810021, 34724015, 45030049, 36575685, 44527472 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2002-04-17") } ] }
-{ "id": 11445889, "id-copy": 11445889, "alias": "Milford", "name": "MilfordTeagarden", "user-since": datetime("2006-06-07T19:18:28.000Z"), "user-since-copy": datetime("2006-06-07T19:18:28.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "subtam", "start-date": date("2003-07-26") } ] }
-{ "id": 11448565, "id-copy": 11448565, "alias": "Martie", "name": "MartiePoley", "user-since": datetime("2010-07-02T14:37:46.000Z"), "user-since-copy": datetime("2010-07-02T14:37:46.000Z"), "friend-ids": {{ 45198632, 14347405, 14595348, 4990646, 44745176, 21949325, 9155582, 3970455, 10097690, 35781298, 46746615, 35535590, 16561713, 31169880, 22467369 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2006-02-08") } ] }
-{ "id": 11468158, "id-copy": 11468158, "alias": "Pamelia", "name": "PameliaShaner", "user-since": datetime("2005-07-11T18:28:07.000Z"), "user-since-copy": datetime("2005-07-11T18:28:07.000Z"), "friend-ids": {{ 8892753, 24751024, 7162523, 38425260, 8752332, 23371746, 6673241, 22278741, 46403700 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2006-02-04") } ] }
-{ "id": 11471689, "id-copy": 11471689, "alias": "Bevis", "name": "BevisWhishaw", "user-since": datetime("2011-03-05T23:14:53.000Z"), "user-since-copy": datetime("2011-03-05T23:14:53.000Z"), "friend-ids": {{ 27818002, 43784015, 39101258, 28170566, 38541659, 43935487, 907437, 25457112, 4731176, 35304801, 30364855, 33197014, 27028915, 21746182, 47624076, 41599425, 8592245 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2000-04-04"), "end-date": date("2009-05-08") } ] }
-{ "id": 11474374, "id-copy": 11474374, "alias": "Waldo", "name": "WaldoKnapp", "user-since": datetime("2008-08-17T21:17:28.000Z"), "user-since-copy": datetime("2008-08-17T21:17:28.000Z"), "friend-ids": {{ 33358772, 16499546, 8631001, 6045567, 45554236, 36229482, 354579, 11884970, 23657774, 32568373 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-11-18") } ] }
-{ "id": 11481961, "id-copy": 11481961, "alias": "Ralph", "name": "RalphMinnie", "user-since": datetime("2008-09-03T03:36:09.000Z"), "user-since-copy": datetime("2008-09-03T03:36:09.000Z"), "friend-ids": {{ 28795092, 15427393, 13323116, 6103928, 22507606, 38931008, 8419762, 30922606, 11217439, 41769747, 19668638, 26796252, 26750627, 4855539, 11170229, 30124829, 16596482, 15728547, 46139530, 43784722, 20640234, 22313927, 16136087, 39688415 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2006-02-13") } ] }
-{ "id": 11488420, "id-copy": 11488420, "alias": "Rik", "name": "RikSell", "user-since": datetime("2011-04-24T10:10:24.000Z"), "user-since-copy": datetime("2011-04-24T10:10:24.000Z"), "friend-ids": {{ 37808691, 28841986, 27850488, 28093210, 9165013, 45941806, 5194022, 39773028, 45473967, 44833113, 27429268 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2002-09-23"), "end-date": date("2010-06-23") } ] }
-{ "id": 11490220, "id-copy": 11490220, "alias": "Ernestine", "name": "ErnestineWheeler", "user-since": datetime("2005-01-27T23:36:35.000Z"), "user-since-copy": datetime("2005-01-27T23:36:35.000Z"), "friend-ids": {{ 12995063, 40353122, 11162426, 42762839, 9575788, 7725738, 29883894, 48002015, 5516807, 12731814, 33203496, 44912740, 19681146, 5849671, 4702317 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2008-06-16"), "end-date": date("2011-12-01") } ] }
-{ "id": 11529952, "id-copy": 11529952, "alias": "Charles", "name": "CharlesHarrow", "user-since": datetime("2008-11-24T19:27:12.000Z"), "user-since-copy": datetime("2008-11-24T19:27:12.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2005-06-08"), "end-date": date("2011-10-27") } ] }
-{ "id": 11533327, "id-copy": 11533327, "alias": "Miguel", "name": "MiguelSteiner", "user-since": datetime("2007-12-08T18:21:30.000Z"), "user-since-copy": datetime("2007-12-08T18:21:30.000Z"), "friend-ids": {{ 41619494, 4881397, 29302201, 26654760, 9690024, 15599321, 37163728, 2420315, 46258007, 15076674, 6757461 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2001-08-19"), "end-date": date("2008-10-15") } ] }
-{ "id": 11536078, "id-copy": 11536078, "alias": "Scot", "name": "ScotSwartzbaugh", "user-since": datetime("2007-06-02T13:28:19.000Z"), "user-since-copy": datetime("2007-06-02T13:28:19.000Z"), "friend-ids": {{ 160897, 11035428, 35908585, 14713740, 16036400, 21530456, 31659920, 33439685, 42771513, 42899492, 42315848, 17885118, 12371932, 47219421, 45350312, 33755309, 30284897, 34557464, 21531204, 26093690 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2002-02-23"), "end-date": date("2005-03-24") } ] }
-{ "id": 11538001, "id-copy": 11538001, "alias": "Milo", "name": "MiloGarland", "user-since": datetime("2007-09-12T09:40:42.000Z"), "user-since-copy": datetime("2007-09-12T09:40:42.000Z"), "friend-ids": {{ 7363153, 7252759 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2011-09-03"), "end-date": date("2011-10-27") } ] }
-{ "id": 11540278, "id-copy": 11540278, "alias": "Flora", "name": "FloraSaltser", "user-since": datetime("2007-11-20T08:52:26.000Z"), "user-since-copy": datetime("2007-11-20T08:52:26.000Z"), "friend-ids": {{ 44172124, 43836609, 2821020, 356092, 25456578, 14806637, 19970466, 15369859, 23267393, 34480680, 42574031, 39606777, 17221367, 19617483, 1364901, 21402012, 4999365, 31098654, 34512618, 44652673, 14757091, 9755310, 39190510 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2012-07-07") } ] }
-{ "id": 11542174, "id-copy": 11542174, "alias": "Pacey", "name": "PaceyTripp", "user-since": datetime("2011-11-07T08:36:12.000Z"), "user-since-copy": datetime("2011-11-07T08:36:12.000Z"), "friend-ids": {{ 35602078, 32622628, 34826581, 34837077, 41522736, 14908313, 42986568 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2006-07-08") } ] }
-{ "id": 11542519, "id-copy": 11542519, "alias": "Colten", "name": "ColtenDemuth", "user-since": datetime("2012-02-09T01:22:04.000Z"), "user-since-copy": datetime("2012-02-09T01:22:04.000Z"), "friend-ids": {{ 15666280, 36489446, 45424145, 47509110, 24198688, 42545568, 30526545, 43828073, 26402530, 23632737, 20385217, 35055795, 38789042, 34967858, 521531, 47834820, 20307524 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2008-04-10") } ] }
-{ "id": 11570326, "id-copy": 11570326, "alias": "Linden", "name": "LindenFilby", "user-since": datetime("2007-08-16T03:11:11.000Z"), "user-since-copy": datetime("2007-08-16T03:11:11.000Z"), "friend-ids": {{ 6549689, 15243636, 3147666 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2010-02-23"), "end-date": date("2010-04-22") } ] }
-{ "id": 11573350, "id-copy": 11573350, "alias": "Sommer", "name": "SommerGregory", "user-since": datetime("2007-08-25T21:50:51.000Z"), "user-since-copy": datetime("2007-08-25T21:50:51.000Z"), "friend-ids": {{ 6622046, 40071999, 24631984, 42427860, 13378139, 27659078, 32813734, 20145238, 15342806, 9562288, 24211264, 29951003, 3620479, 43701781, 22474191, 6298296, 4047189, 27133942, 8058121, 9928231, 31835361, 6234235, 6100660, 1575061 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-12-09"), "end-date": date("2010-01-16") } ] }
-{ "id": 11587666, "id-copy": 11587666, "alias": "Kathi", "name": "KathiJenner", "user-since": datetime("2012-02-20T01:58:30.000Z"), "user-since-copy": datetime("2012-02-20T01:58:30.000Z"), "friend-ids": {{ 37156773, 10519382, 11009989, 47883115, 13123467, 36990044, 8554049, 47075065, 11896169, 42580126, 43261036, 15337748, 35985068, 44438965, 33507413, 40063633, 32559158, 32202309, 25536635 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2001-01-01") } ] }
-{ "id": 11610913, "id-copy": 11610913, "alias": "Vic", "name": "VicDiegel", "user-since": datetime("2008-08-03T21:05:21.000Z"), "user-since-copy": datetime("2008-08-03T21:05:21.000Z"), "friend-ids": {{ 15275871, 8304749, 7803583, 45134147, 36058489, 7180792, 2104280, 4322584, 39304177, 43050196, 32955811, 4161448, 3187410, 47263593 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-03-11") } ] }
-{ "id": 11626990, "id-copy": 11626990, "alias": "Filiberto", "name": "FilibertoFonblanque", "user-since": datetime("2006-05-18T07:38:32.000Z"), "user-since-copy": datetime("2006-05-18T07:38:32.000Z"), "friend-ids": {{ 41443868, 30006940, 14137070, 14868792, 47991977, 39513958, 32787637, 1389727, 28607710, 21537795, 42395037, 11730902, 25246772, 24475669, 35786951, 32795214 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2007-08-11") } ] }
-{ "id": 11659888, "id-copy": 11659888, "alias": "Nannie", "name": "NannieWoodworth", "user-since": datetime("2006-12-11T15:30:08.000Z"), "user-since-copy": datetime("2006-12-11T15:30:08.000Z"), "friend-ids": {{ 30803046, 33105462, 14783423, 5069473, 15960335 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2006-10-12") } ] }
-{ "id": 11668552, "id-copy": 11668552, "alias": "Kassandra", "name": "KassandraJames", "user-since": datetime("2010-09-27T18:12:59.000Z"), "user-since-copy": datetime("2010-09-27T18:12:59.000Z"), "friend-ids": {{ 27400643, 15449089, 802964, 45059523, 9603951, 20911122, 46243977, 45487995, 34528880, 16093159, 22484957, 3951663, 12349433, 7887502, 34786818, 13014384, 28307526, 30476565, 7746152, 17600641, 36877141, 4513081, 25065078 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-04"), "end-date": date("2012-08-25") } ] }
-{ "id": 11674741, "id-copy": 11674741, "alias": "Soon", "name": "SoonBillimek", "user-since": datetime("2009-03-02T12:08:16.000Z"), "user-since-copy": datetime("2009-03-02T12:08:16.000Z"), "friend-ids": {{ 26069920, 16634341, 13963293, 27425934, 19271848, 22444876, 42264629, 39307655, 21118192, 27961060, 12398172, 13202296, 23221559, 34323488, 1588557, 42672479, 19548482, 28266272, 6241122, 13633490 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2006-01-19"), "end-date": date("2011-03-25") } ] }
-{ "id": 11676574, "id-copy": 11676574, "alias": "Isidore", "name": "IsidoreCatlay", "user-since": datetime("2012-08-26T08:28:08.000Z"), "user-since-copy": datetime("2012-08-26T08:28:08.000Z"), "friend-ids": {{ 46189001 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2001-03-07") } ] }
-{ "id": 11698384, "id-copy": 11698384, "alias": "Bernetta", "name": "BernettaFiddler", "user-since": datetime("2012-06-20T20:05:46.000Z"), "user-since-copy": datetime("2012-06-20T20:05:46.000Z"), "friend-ids": {{ 12203676 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2000-03-06") } ] }
-{ "id": 11708152, "id-copy": 11708152, "alias": "Gil", "name": "GilElsas", "user-since": datetime("2009-04-08T15:40:59.000Z"), "user-since-copy": datetime("2009-04-08T15:40:59.000Z"), "friend-ids": {{ 14661698, 22657473, 28892770, 39654430, 46338819, 44974094, 38564659, 24819725, 21550883, 37711934, 37285158, 20050610, 19163447, 10974750, 47513067, 43771947, 23633824 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2002-09-21"), "end-date": date("2011-03-11") } ] }
-{ "id": 11709478, "id-copy": 11709478, "alias": "Jonty", "name": "JontyCurry", "user-since": datetime("2006-09-08T22:15:05.000Z"), "user-since-copy": datetime("2006-09-08T22:15:05.000Z"), "friend-ids": {{ 1684909, 3914449, 16704128, 11890093, 44073634, 24897496 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2006-03-01") } ] }
-{ "id": 11713315, "id-copy": 11713315, "alias": "Chung", "name": "ChungStroble", "user-since": datetime("2005-10-20T22:59:27.000Z"), "user-since-copy": datetime("2005-10-20T22:59:27.000Z"), "friend-ids": {{ 13105744, 9160760, 37104436, 33688116, 31455484, 44428287 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2002-12-03"), "end-date": date("2010-10-06") } ] }
-{ "id": 11720794, "id-copy": 11720794, "alias": "Alisha", "name": "AlishaTue", "user-since": datetime("2010-08-11T01:17:31.000Z"), "user-since-copy": datetime("2010-08-11T01:17:31.000Z"), "friend-ids": {{ 6380101, 43972052, 6557931, 42465959, 21268624, 35831867, 45839471, 37781645, 34750475, 35886124, 4491900 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2001-02-02") } ] }
-{ "id": 11725939, "id-copy": 11725939, "alias": "Clover", "name": "CloverAlice", "user-since": datetime("2007-07-12T05:17:52.000Z"), "user-since-copy": datetime("2007-07-12T05:17:52.000Z"), "friend-ids": {{ 24426905, 6647137, 25463555, 11443041, 10549599, 35925634, 4053835, 11813301, 6976204, 26680887, 29934690, 7935338, 45092791, 30510709 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2000-05-04"), "end-date": date("2000-08-24") } ] }
-{ "id": 11735830, "id-copy": 11735830, "alias": "Maryvonne", "name": "MaryvonneHarrold", "user-since": datetime("2007-12-03T06:30:43.000Z"), "user-since-copy": datetime("2007-12-03T06:30:43.000Z"), "friend-ids": {{ 27842540, 46624942, 21701969, 33750891, 28523702, 38840881, 1497785, 32357938, 19740312, 1880841, 41116687, 35621654, 46917268, 14610853, 33099367, 8710534 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2004-10-04") } ] }
-{ "id": 11758474, "id-copy": 11758474, "alias": "Xavier", "name": "XavierAtweeke", "user-since": datetime("2011-10-03T12:35:37.000Z"), "user-since-copy": datetime("2011-10-03T12:35:37.000Z"), "friend-ids": {{ 30110740, 41016650, 23732518, 14585316, 34474077, 47591093, 10803514, 8912354, 43455040, 21960801, 31978150, 40693811, 14585416, 36411476, 20556412, 44978412, 7266670, 506620, 7686872 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2004-03-07") } ] }
-{ "id": 11774587, "id-copy": 11774587, "alias": "Shari", "name": "ShariMortland", "user-since": datetime("2012-07-21T10:15:22.000Z"), "user-since-copy": datetime("2012-07-21T10:15:22.000Z"), "friend-ids": {{ 17661326, 29399532, 38328734, 38063295, 46008807, 29873254, 4407085, 27903240 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2005-05-18") } ] }
-{ "id": 11783038, "id-copy": 11783038, "alias": "Cecily", "name": "CecilyRamsey", "user-since": datetime("2011-01-20T23:39:28.000Z"), "user-since-copy": datetime("2011-01-20T23:39:28.000Z"), "friend-ids": {{ 30228589, 45494315, 36823967, 2965036, 37243358, 7140131, 8303981, 10041948, 41439178, 24261471, 16906521, 46190105, 45392996, 21067630, 26632248, 44955893 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2000-03-25"), "end-date": date("2010-06-25") } ] }
-{ "id": 11786815, "id-copy": 11786815, "alias": "Micheal", "name": "MichealTreeby", "user-since": datetime("2008-06-04T14:59:23.000Z"), "user-since-copy": datetime("2008-06-04T14:59:23.000Z"), "friend-ids": {{ 15590922, 1367468, 37464776, 21877607, 38646966, 46702919, 46771039, 4688915, 41827211, 6556380 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2003-09-17") } ] }
-{ "id": 11801005, "id-copy": 11801005, "alias": "Jacques", "name": "JacquesWhitling", "user-since": datetime("2007-05-20T05:42:21.000Z"), "user-since-copy": datetime("2007-05-20T05:42:21.000Z"), "friend-ids": {{ 45134681, 48016178 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2006-12-07") } ] }
-{ "id": 11804755, "id-copy": 11804755, "alias": "Humbert", "name": "HumbertArmitage", "user-since": datetime("2008-01-01T21:14:34.000Z"), "user-since-copy": datetime("2008-01-01T21:14:34.000Z"), "friend-ids": {{ 15498777, 1984479, 18672418, 13137212, 17931875, 10446256, 39250716, 9422828, 35469173, 35940705, 44217206 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2005-11-12") } ] }
-{ "id": 11818252, "id-copy": 11818252, "alias": "Sandee", "name": "SandeeBlair", "user-since": datetime("2008-12-22T20:09:56.000Z"), "user-since-copy": datetime("2008-12-22T20:09:56.000Z"), "friend-ids": {{ 35579096, 13690328, 19410347, 10601941, 13140634, 19728850 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2007-09-24") } ] }
-{ "id": 11821996, "id-copy": 11821996, "alias": "Latanya", "name": "LatanyaZalack", "user-since": datetime("2010-12-07T15:20:09.000Z"), "user-since-copy": datetime("2010-12-07T15:20:09.000Z"), "friend-ids": {{ 23521495, 43957220, 3823403, 34033770 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2008-04-17") } ] }
-{ "id": 11830663, "id-copy": 11830663, "alias": "Bettie", "name": "BettieKing", "user-since": datetime("2009-11-06T15:04:55.000Z"), "user-since-copy": datetime("2009-11-06T15:04:55.000Z"), "friend-ids": {{ 46068058, 35215092, 34850678, 9126970, 16472040, 20000261, 17610567, 37016763, 19830405, 38071058, 43961371, 13092410, 24867008, 12366628, 15539063, 15611017, 1343975, 43254018, 30838755, 30488641, 38027133, 5701592 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2003-04-10") } ] }
-{ "id": 11857618, "id-copy": 11857618, "alias": "Glenda", "name": "GlendaPyle", "user-since": datetime("2009-01-05T13:34:53.000Z"), "user-since-copy": datetime("2009-01-05T13:34:53.000Z"), "friend-ids": {{ 31083833, 39371819, 38336556, 7590988, 17022330, 8016611, 41444367, 13194826, 1589028, 37076285, 33481940, 22093098, 9959371, 35262849, 20744580, 33226729, 35025566, 46396680, 30247311, 6884899, 35691024, 40965552, 46106170 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2000-02-19") } ] }
-{ "id": 11886532, "id-copy": 11886532, "alias": "Tel", "name": "TelGardner", "user-since": datetime("2009-10-06T10:33:32.000Z"), "user-since-copy": datetime("2009-10-06T10:33:32.000Z"), "friend-ids": {{ 37243107, 36561786, 3939621, 13531917, 7768514, 31689833, 27145019, 9462172, 40579935, 32184519, 8668855, 26137893, 5582080, 4847233, 10244448, 42634758, 34911290, 10834989, 34800551, 14109743 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2010-07-24") } ] }
-{ "id": 11894854, "id-copy": 11894854, "alias": "Connor", "name": "ConnorWilliamson", "user-since": datetime("2011-09-16T22:24:17.000Z"), "user-since-copy": datetime("2011-09-16T22:24:17.000Z"), "friend-ids": {{ 19318451, 47946991, 1913830, 45324890, 47189256, 39211392, 6998884, 4344587, 24720830, 4355756, 19102058, 34241496, 39408673, 1360498, 7695088, 25754984, 21796436 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2007-09-19"), "end-date": date("2010-07-22") } ] }
-{ "id": 11899576, "id-copy": 11899576, "alias": "Raven", "name": "RavenAdams", "user-since": datetime("2011-12-02T12:46:45.000Z"), "user-since-copy": datetime("2011-12-02T12:46:45.000Z"), "friend-ids": {{ 33232775, 8985272, 34257645, 15577012, 3749136, 36721837, 17368752, 36931534, 30688133, 36202643, 8373322, 34639728, 10776563, 5758944, 19414939, 46764976, 29704238, 38970621, 9462886, 46724087, 29191126, 9001393 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2003-03-02") } ] }
-{ "id": 11899861, "id-copy": 11899861, "alias": "Jacki", "name": "JackiLeach", "user-since": datetime("2009-01-07T13:33:40.000Z"), "user-since-copy": datetime("2009-01-07T13:33:40.000Z"), "friend-ids": {{ 17554995, 17598007, 2855045, 4108843, 47202404, 42565398, 45821410, 32619673, 7988594, 7631349, 20552170, 13116128, 14526615, 17916951, 43018507, 18114607 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-06-24") } ] }
-{ "id": 11920078, "id-copy": 11920078, "alias": "Alane", "name": "AlaneRichter", "user-since": datetime("2005-04-12T04:06:03.000Z"), "user-since-copy": datetime("2005-04-12T04:06:03.000Z"), "friend-ids": {{ 18326190, 34366549, 13047472, 29553920, 6210406, 41865352, 26108964, 15042193, 33225025, 7014329, 11051157, 37032436, 8025322, 21902099, 22953955, 42645725, 29144585 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2006-04-24") } ] }
-{ "id": 11920375, "id-copy": 11920375, "alias": "Terance", "name": "TeranceSaylor", "user-since": datetime("2005-02-09T10:33:47.000Z"), "user-since-copy": datetime("2005-02-09T10:33:47.000Z"), "friend-ids": {{ 17869677, 39051840, 6852335, 6153367, 1318628, 9983745, 5401091, 32798056, 42870494, 10337793, 43570623, 3233493, 38297525, 43712104, 15430099, 36703995, 25022620, 3681464, 21499719, 33737350, 6602331, 35391438, 47011233 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2005-11-05"), "end-date": date("2011-04-20") } ] }
-{ "id": 11943412, "id-copy": 11943412, "alias": "Kizzie", "name": "KizzieBillimek", "user-since": datetime("2011-08-25T09:24:43.000Z"), "user-since-copy": datetime("2011-08-25T09:24:43.000Z"), "friend-ids": {{ 47433684, 41380366, 5933545, 6348490, 24429719, 22579519, 21550752, 4653838, 44131628, 7980571, 3208666, 35631166, 13693250, 41263305, 29172668, 24656473, 31110672, 11323134, 23674731, 37422602, 20327470, 13419973 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2012-03-18"), "end-date": date("2012-06-09") } ] }
-{ "id": 11972860, "id-copy": 11972860, "alias": "Isador", "name": "IsadorCattley", "user-since": datetime("2005-04-10T23:37:49.000Z"), "user-since-copy": datetime("2005-04-10T23:37:49.000Z"), "friend-ids": {{ 39841874, 9405322, 3110197, 39455453, 11331432, 31809217, 45852118, 12899824, 19561127, 3413313, 19872192, 13427579, 140732, 6913603 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2006-01-01"), "end-date": date("2009-11-22") } ] }
-{ "id": 11989228, "id-copy": 11989228, "alias": "Jaden", "name": "JadenKelley", "user-since": datetime("2006-11-12T15:45:55.000Z"), "user-since-copy": datetime("2006-11-12T15:45:55.000Z"), "friend-ids": {{ 39881086, 47143027, 9394301, 17338199, 16961896, 6602092, 46708527, 24050942, 20543677, 13309656 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2003-10-19") } ] }
-{ "id": 11990740, "id-copy": 11990740, "alias": "Vernon", "name": "VernonBarnes", "user-since": datetime("2005-05-25T09:07:06.000Z"), "user-since-copy": datetime("2005-05-25T09:07:06.000Z"), "friend-ids": {{ 44677447, 20354746, 30157224, 29686873, 9413456, 11656099, 25404439, 24706566, 45005726, 22096097, 29868918, 12109246, 38948331, 2643312, 41565707, 17566751, 8045341, 25358960, 43614095, 28262168, 14405467, 22519550 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2012-01-23") } ] }
-{ "id": 11996683, "id-copy": 11996683, "alias": "Ivy", "name": "IvyReddish", "user-since": datetime("2008-10-09T09:54:46.000Z"), "user-since-copy": datetime("2008-10-09T09:54:46.000Z"), "friend-ids": {{ 42344158, 40312093, 15782003 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2003-04-16") } ] }
-{ "id": 9004354, "id-copy": 9004354, "alias": "Deshawn", "name": "DeshawnGarneys", "user-since": datetime("2010-07-21T12:45:03.000Z"), "user-since-copy": datetime("2010-07-21T12:45:03.000Z"), "friend-ids": {{ 46096495, 1526403 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2011-07-08") } ] }
-{ "id": 9025786, "id-copy": 9025786, "alias": "Terrance", "name": "TerranceFinlay", "user-since": datetime("2009-12-28T02:19:23.000Z"), "user-since-copy": datetime("2009-12-28T02:19:23.000Z"), "friend-ids": {{ 45324679, 13507068, 46678304, 37010727, 44866157, 12584675, 34305776, 14467180, 37751377, 2448873, 32584169, 14120838, 8902593, 31955437, 13436805 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2012-02-19"), "end-date": date("2012-07-25") } ] }
-{ "id": 9029377, "id-copy": 9029377, "alias": "Boyce", "name": "BoyceAnderson", "user-since": datetime("2010-12-18T14:17:12.000Z"), "user-since-copy": datetime("2010-12-18T14:17:12.000Z"), "friend-ids": {{ 19260027, 21449100, 35898407, 34501982 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2007-06-25") } ] }
-{ "id": 9041578, "id-copy": 9041578, "alias": "Kristia", "name": "KristiaWillcox", "user-since": datetime("2012-01-09T10:29:02.000Z"), "user-since-copy": datetime("2012-01-09T10:29:02.000Z"), "friend-ids": {{ 29794000, 34515675, 3759231, 14418948, 35788028, 34225561, 20821065, 27582458, 4424723 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2005-06-04"), "end-date": date("2008-01-13") } ] }
-{ "id": 9041992, "id-copy": 9041992, "alias": "Royston", "name": "RoystonBatten", "user-since": datetime("2009-06-27T08:09:45.000Z"), "user-since-copy": datetime("2009-06-27T08:09:45.000Z"), "friend-ids": {{ 35666317, 30439304, 35405688, 2079220, 5996407, 40490306, 33188983, 24455609, 4293738, 29028817, 32566429, 10186823 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2002-04-04"), "end-date": date("2010-06-28") } ] }
-{ "id": 9046852, "id-copy": 9046852, "alias": "Mauro", "name": "MauroChase", "user-since": datetime("2011-04-18T20:18:58.000Z"), "user-since-copy": datetime("2011-04-18T20:18:58.000Z"), "friend-ids": {{ 28268506, 13880377, 18637778, 27129860, 47146036, 23136396, 34534506, 23274864, 38781071, 9644011, 34754620, 45178277, 33832472, 31871984, 47201051, 42153557, 12418584, 37615805, 35474951, 29273401, 4845352, 18687033 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2012-05-14"), "end-date": date("2012-06-25") } ] }
-{ "id": 9067279, "id-copy": 9067279, "alias": "Jeanine", "name": "JeanineEmrick", "user-since": datetime("2011-06-25T09:43:07.000Z"), "user-since-copy": datetime("2011-06-25T09:43:07.000Z"), "friend-ids": {{ 12884712, 45104617, 41134568, 15844605, 645264, 33182092, 16884335, 46281324, 3977219, 5682848, 441588, 26025738, 3165091, 21821928, 23073877 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2012-01-02") } ] }
-{ "id": 9069397, "id-copy": 9069397, "alias": "Manuel", "name": "ManuelTrevithick", "user-since": datetime("2009-01-25T00:11:22.000Z"), "user-since-copy": datetime("2009-01-25T00:11:22.000Z"), "friend-ids": {{ 1121944, 14645273, 16100117, 45331540, 17901062, 7344920, 22533580, 16386626, 4267586, 34975914, 28841442, 38737330, 31607047, 11785331, 9617022, 44328180, 30996836, 14315445, 18464409, 21038654, 14409120, 12230754, 25856707 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2011-10-12"), "end-date": date("2011-03-28") } ] }
-{ "id": 9077020, "id-copy": 9077020, "alias": "Marquis", "name": "MarquisBunten", "user-since": datetime("2008-08-23T04:31:07.000Z"), "user-since-copy": datetime("2008-08-23T04:31:07.000Z"), "friend-ids": {{ 16894897, 21101342, 27872737, 14878739, 47969914, 38986368, 20779589, 4491084, 21066166, 40159242, 25290828, 43152855, 41928030, 2282944 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2001-07-16") } ] }
-{ "id": 9082201, "id-copy": 9082201, "alias": "Alberic", "name": "AlbericCrawford", "user-since": datetime("2005-02-11T07:41:05.000Z"), "user-since-copy": datetime("2005-02-11T07:41:05.000Z"), "friend-ids": {{ 26925567, 6108069, 30484049, 4903722, 4579631, 21166966, 3892344, 6259030, 32887933, 7183018, 46041497, 23448710, 47887528, 3679587, 7140571, 47671072, 4554470, 23481403, 16738975, 4885244 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2006-10-10") } ] }
-{ "id": 9083791, "id-copy": 9083791, "alias": "Lashay", "name": "LashayLeonard", "user-since": datetime("2008-07-03T04:52:06.000Z"), "user-since-copy": datetime("2008-07-03T04:52:06.000Z"), "friend-ids": {{ 16762687, 32021842, 851915, 36102981, 3553783, 30756474, 12043049, 16852621, 35699568, 4425852, 35227725, 25748188, 9140215, 24886626, 1945167, 12733697, 20761965 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2003-05-19"), "end-date": date("2006-10-16") } ] }
-{ "id": 9087292, "id-copy": 9087292, "alias": "Kiersten", "name": "KierstenRawls", "user-since": datetime("2005-03-21T08:42:24.000Z"), "user-since-copy": datetime("2005-03-21T08:42:24.000Z"), "friend-ids": {{ 5551555, 2958358, 17900476, 23956783, 31634897, 12573318, 32475113, 47343698, 40929064, 39881831, 38067700, 3519291, 19229024, 4383684, 13932328, 16414275, 8654888, 16145374, 26880764 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-03-10") } ] }
-{ "id": 9098314, "id-copy": 9098314, "alias": "Terrance", "name": "TerranceWilkerson", "user-since": datetime("2010-07-01T06:01:32.000Z"), "user-since-copy": datetime("2010-07-01T06:01:32.000Z"), "friend-ids": {{ 32477103, 38306013, 36022406, 25594192, 10966661, 28537611, 5444323, 16012053, 43228208, 30344050, 22600011, 42820310, 37103995, 6359985 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2012-01-24") } ] }
-{ "id": 9112336, "id-copy": 9112336, "alias": "Marlin", "name": "MarlinRosenstiehl", "user-since": datetime("2010-09-26T04:27:50.000Z"), "user-since-copy": datetime("2010-09-26T04:27:50.000Z"), "friend-ids": {{ 10225686, 16259250, 11552542, 28661586, 8900635, 27988260 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2011-12-05") } ] }
-{ "id": 9133714, "id-copy": 9133714, "alias": "Wil", "name": "WilDale", "user-since": datetime("2009-12-04T18:40:04.000Z"), "user-since-copy": datetime("2009-12-04T18:40:04.000Z"), "friend-ids": {{ 40400811, 26528322 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2005-10-08"), "end-date": date("2007-03-23") } ] }
-{ "id": 9139966, "id-copy": 9139966, "alias": "Elwood", "name": "ElwoodDavis", "user-since": datetime("2009-04-25T20:38:07.000Z"), "user-since-copy": datetime("2009-04-25T20:38:07.000Z"), "friend-ids": {{ 28327906, 35534034, 3278109, 20721373, 40303614, 22594044, 3292862, 42117489, 18133788, 31771270, 43837818, 36567026 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2004-09-03"), "end-date": date("2011-07-03") } ] }
-{ "id": 9151357, "id-copy": 9151357, "alias": "Clover", "name": "CloverTedrow", "user-since": datetime("2012-04-04T22:46:03.000Z"), "user-since-copy": datetime("2012-04-04T22:46:03.000Z"), "friend-ids": {{ 47959325, 11808875, 46311157, 33138600, 15486362, 27921017, 32586367, 24379643, 14793815, 5841252, 22249573, 2147304, 47811082, 40329394, 4601822, 27977744, 45733056 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2002-09-07"), "end-date": date("2006-08-04") } ] }
-{ "id": 9160906, "id-copy": 9160906, "alias": "Cathryn", "name": "CathrynReamer", "user-since": datetime("2010-10-08T06:24:51.000Z"), "user-since-copy": datetime("2010-10-08T06:24:51.000Z"), "friend-ids": {{ 30962953 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2006-10-28"), "end-date": date("2010-03-14") } ] }
-{ "id": 9174313, "id-copy": 9174313, "alias": "Hal", "name": "HalHasely", "user-since": datetime("2008-01-28T17:01:16.000Z"), "user-since-copy": datetime("2008-01-28T17:01:16.000Z"), "friend-ids": {{ 9058102, 40616538, 45706325, 991699, 37832260, 4793008, 36372035, 23272432, 36685642, 2621984, 9576806, 14325601, 41449409, 16499609, 20610820, 1564035, 20738111, 19735088, 31942764, 34813086 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-10-16") } ] }
-{ "id": 9179122, "id-copy": 9179122, "alias": "Zach", "name": "ZachMilliron", "user-since": datetime("2011-07-28T01:09:04.000Z"), "user-since-copy": datetime("2011-07-28T01:09:04.000Z"), "friend-ids": {{ 40552138, 19204406, 46806031, 18794200, 45071131, 40119114, 23955279, 11126709, 37101358, 23332998, 1172034, 41496458, 32278235, 30949991, 148070, 6360227, 7378339, 33611217 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2000-06-26") } ] }
-{ "id": 9187549, "id-copy": 9187549, "alias": "Lenny", "name": "LennyField", "user-since": datetime("2008-09-11T10:50:10.000Z"), "user-since-copy": datetime("2008-09-11T10:50:10.000Z"), "friend-ids": {{ 26505249, 4392946, 32062169, 45628101, 22865593, 4982483, 13425537, 18846467, 36122039, 2998293, 19787439, 22246499, 43133873, 30573462, 36272473, 41691126, 43929640, 43759980, 25546305 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2004-08-13"), "end-date": date("2010-03-08") } ] }
-{ "id": 9203731, "id-copy": 9203731, "alias": "Phoebe", "name": "PhoebeCoates", "user-since": datetime("2008-04-27T01:42:34.000Z"), "user-since-copy": datetime("2008-04-27T01:42:34.000Z"), "friend-ids": {{ 25611465, 519838, 22814080, 46015954, 7805914, 12757618, 36785422, 25727822, 32042543 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2003-11-11"), "end-date": date("2005-08-19") } ] }
-{ "id": 9216376, "id-copy": 9216376, "alias": "Stanford", "name": "StanfordBurney", "user-since": datetime("2010-04-24T23:03:06.000Z"), "user-since-copy": datetime("2010-04-24T23:03:06.000Z"), "friend-ids": {{ 15567770, 24839882, 163708, 45725879, 43621238, 27363995, 46782727, 21660511, 37585197, 17426559, 47247057, 41831246, 23944363, 1608826, 25831838, 41140458, 37108898, 19739056, 7475981, 17807472, 3126856, 40257768, 44873566 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2004-10-04") } ] }
-{ "id": 9219955, "id-copy": 9219955, "alias": "Audrey", "name": "AudreyOmara", "user-since": datetime("2011-06-04T15:31:25.000Z"), "user-since-copy": datetime("2011-06-04T15:31:25.000Z"), "friend-ids": {{ 28209595, 29907721, 18295175, 18631813, 3380755, 20244076, 43026452, 42394327, 10914853, 27224999 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2003-03-24") } ] }
-{ "id": 9232504, "id-copy": 9232504, "alias": "Lesley", "name": "LesleyHujsak", "user-since": datetime("2008-07-07T13:30:22.000Z"), "user-since-copy": datetime("2008-07-07T13:30:22.000Z"), "friend-ids": {{ 42058063, 24501683, 26865036, 180621 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-01-04"), "end-date": date("2011-02-07") } ] }
-{ "id": 9233794, "id-copy": 9233794, "alias": "Jeffrey", "name": "JeffreyThrockmorton", "user-since": datetime("2005-04-23T04:24:31.000Z"), "user-since-copy": datetime("2005-04-23T04:24:31.000Z"), "friend-ids": {{ 29565308, 29107229, 35495609, 27358360, 24507795, 18583779, 16799427, 3571959, 6539875, 32120867, 17248402, 12227155, 37995559, 29425657, 20855502 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2000-04-22"), "end-date": date("2010-05-28") } ] }
-{ "id": 9234529, "id-copy": 9234529, "alias": "Xavior", "name": "XaviorBarnes", "user-since": datetime("2010-08-26T12:06:44.000Z"), "user-since-copy": datetime("2010-08-26T12:06:44.000Z"), "friend-ids": {{ 19552290, 24018104, 43285028, 33954718, 18084047, 18675363, 17369450, 36533551, 46779811, 46943171, 17609996, 14171942, 10468121, 33831228, 9905114, 11839935, 41387228 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2007-12-24") } ] }
-{ "id": 9239515, "id-copy": 9239515, "alias": "Precious", "name": "PreciousWeingarten", "user-since": datetime("2006-07-03T10:28:56.000Z"), "user-since-copy": datetime("2006-07-03T10:28:56.000Z"), "friend-ids": {{ 20459132, 9181399, 30604442, 45266959, 31805782, 8190732, 46444663, 46572075, 43980715, 42547186, 21087158, 38075989, 32228414, 25466991, 4929897, 33467622, 35742242, 7150399, 16997658, 18543557, 11799062 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2003-02-15") } ] }
-{ "id": 9269422, "id-copy": 9269422, "alias": "Roddy", "name": "RoddyFriedline", "user-since": datetime("2007-03-26T23:41:29.000Z"), "user-since-copy": datetime("2007-03-26T23:41:29.000Z"), "friend-ids": {{ 31923430, 19739952, 30983882, 10630795 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2001-06-03") } ] }
-{ "id": 9292738, "id-copy": 9292738, "alias": "Walter", "name": "WalterWain", "user-since": datetime("2012-05-03T10:41:22.000Z"), "user-since-copy": datetime("2012-05-03T10:41:22.000Z"), "friend-ids": {{ 1834068, 38777276, 43381631, 32380769, 23994313, 37459142, 21015234, 23788270, 33191448, 31111521, 21788604, 39349512, 20638072, 17300228, 4712935, 36205876, 27740958, 27236154 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2011-03-28") } ] }
-{ "id": 9332161, "id-copy": 9332161, "alias": "Lavinia", "name": "LaviniaLineman", "user-since": datetime("2006-02-07T20:39:55.000Z"), "user-since-copy": datetime("2006-02-07T20:39:55.000Z"), "friend-ids": {{ 21419337, 31581364 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-07-05") } ] }
-{ "id": 9341008, "id-copy": 9341008, "alias": "Gus", "name": "GusGearhart", "user-since": datetime("2012-05-23T13:19:57.000Z"), "user-since-copy": datetime("2012-05-23T13:19:57.000Z"), "friend-ids": {{ 20124243, 19722425, 20605718, 21833401, 18276801, 46184199, 40454562, 22828817, 44122338, 4485860, 34209581, 19783645, 44454238, 1353350, 37958534, 33547730, 2456119, 3023314, 44828467, 46655836, 33144170, 16864855, 41938662 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2002-07-07") } ] }
-{ "id": 9354127, "id-copy": 9354127, "alias": "Seymour", "name": "SeymourFlick", "user-since": datetime("2011-06-17T06:00:11.000Z"), "user-since-copy": datetime("2011-06-17T06:00:11.000Z"), "friend-ids": {{ 7662170, 25563062, 18178019, 32667220, 12254954, 7192061, 18829113, 8959008, 1692176, 28852587, 17130396, 12781461, 4083182, 11054115, 10558861, 13876198 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2007-11-23") } ] }
-{ "id": 9356098, "id-copy": 9356098, "alias": "Juliana", "name": "JulianaAnderson", "user-since": datetime("2007-04-26T20:13:07.000Z"), "user-since-copy": datetime("2007-04-26T20:13:07.000Z"), "friend-ids": {{ 3850702, 46858392, 20177889, 34485932, 20958453, 26839176, 23562562, 47962945, 43961803, 19857248, 29816714, 14695228, 35327929, 16196977, 11908428, 30035277, 23919929 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2009-06-04"), "end-date": date("2009-05-05") } ] }
-{ "id": 9369847, "id-copy": 9369847, "alias": "Jeffrey", "name": "JeffreyArchibald", "user-since": datetime("2011-07-11T23:43:52.000Z"), "user-since-copy": datetime("2011-07-11T23:43:52.000Z"), "friend-ids": {{ 44928062, 45653705 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2010-03-25") } ] }
-{ "id": 9407710, "id-copy": 9407710, "alias": "Todd", "name": "ToddStall", "user-since": datetime("2009-09-21T02:18:16.000Z"), "user-since-copy": datetime("2009-09-21T02:18:16.000Z"), "friend-ids": {{ 46998635, 14217621, 8062100, 47498395, 37234901, 41039045, 37635206, 42173831, 24149948 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2009-09-27"), "end-date": date("2009-07-21") } ] }
-{ "id": 9418882, "id-copy": 9418882, "alias": "Laurine", "name": "LaurineCowart", "user-since": datetime("2012-06-14T22:26:09.000Z"), "user-since-copy": datetime("2012-06-14T22:26:09.000Z"), "friend-ids": {{ 19430214, 17084414, 12678029, 1783933, 42580022, 26274674, 13661281, 31117329, 19971039, 43840305, 42672247, 17088417, 31128028, 41009670, 16020772 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2002-03-20") } ] }
-{ "id": 9421798, "id-copy": 9421798, "alias": "Jaqueline", "name": "JaquelineHasely", "user-since": datetime("2011-06-06T16:32:03.000Z"), "user-since-copy": datetime("2011-06-06T16:32:03.000Z"), "friend-ids": {{ 17911249, 45887650, 15916739, 42045244, 42824039, 4802136, 43709530, 41533233, 13714833, 33000412, 29627102, 43277560, 3727319, 19030370, 47600623, 27902511, 13460397, 34825938, 9726577, 10062858, 34721080, 6725312, 21572679 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2004-02-17") } ] }
-{ "id": 9426544, "id-copy": 9426544, "alias": "Joshawa", "name": "JoshawaHiles", "user-since": datetime("2012-04-28T09:48:20.000Z"), "user-since-copy": datetime("2012-04-28T09:48:20.000Z"), "friend-ids": {{ 16780903 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2002-07-01") } ] }
-{ "id": 9440452, "id-copy": 9440452, "alias": "Maria", "name": "MariaField", "user-since": datetime("2010-04-06T15:15:24.000Z"), "user-since-copy": datetime("2010-04-06T15:15:24.000Z"), "friend-ids": {{ 35137543, 24166956, 45255343, 10050289, 27769291, 40368984, 38146662, 43123957, 10442976, 46931482, 447566, 14148069, 39035817, 32169234, 35607837, 8648749, 3741547, 31840808, 3029722, 40917859 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-04-27"), "end-date": date("2012-05-11") } ] }
-{ "id": 9449881, "id-copy": 9449881, "alias": "Veola", "name": "VeolaSchaeffer", "user-since": datetime("2005-06-15T04:27:55.000Z"), "user-since-copy": datetime("2005-06-15T04:27:55.000Z"), "friend-ids": {{ 15932585, 16875491, 977001, 15650783, 30629770, 9735829, 35435062, 2023808, 21909452, 29327288, 24004438, 41780113, 10546865, 17514287, 16690971, 23762008, 21853049, 12673064, 35992661, 30579445, 21341455, 2338670 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2001-09-07") } ] }
-{ "id": 9450532, "id-copy": 9450532, "alias": "Troy", "name": "TroyKoepple", "user-since": datetime("2011-05-10T09:56:46.000Z"), "user-since-copy": datetime("2011-05-10T09:56:46.000Z"), "friend-ids": {{ 42029412, 18025243, 715282, 501115, 38550360, 39016114, 31451417, 38836992, 13665836, 17286159, 28850827, 17241066, 41893804, 39172781, 4523003, 28542863, 25386847, 44039032, 19593806, 607220, 26442265, 47847281 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-12-26"), "end-date": date("2004-04-05") } ] }
-{ "id": 9483769, "id-copy": 9483769, "alias": "Marketta", "name": "MarkettaSchere", "user-since": datetime("2006-04-02T05:48:16.000Z"), "user-since-copy": datetime("2006-04-02T05:48:16.000Z"), "friend-ids": {{ 15151816, 38432593, 14501842, 21508230, 20201815, 35434395, 46212890, 9387767, 35469959, 6671088, 38888798, 10719563, 36944652, 36703732, 9646545, 29287523, 24156038, 24502755 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-07-20"), "end-date": date("2006-03-10") } ] }
-{ "id": 9503761, "id-copy": 9503761, "alias": "Demelza", "name": "DemelzaLaw", "user-since": datetime("2010-12-17T06:40:19.000Z"), "user-since-copy": datetime("2010-12-17T06:40:19.000Z"), "friend-ids": {{ 34126746, 5537488, 609154, 35877951, 36237612 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2005-10-22") } ] }
-{ "id": 9516883, "id-copy": 9516883, "alias": "Delsie", "name": "DelsieKuster", "user-since": datetime("2005-11-20T06:18:01.000Z"), "user-since-copy": datetime("2005-11-20T06:18:01.000Z"), "friend-ids": {{ 7211399, 31355269, 10052966, 11255272, 11976144, 13036749, 28228775, 3501290, 35668522, 21064471, 47089958, 20725508, 16768149, 39282691, 44096922, 12469594, 8258231, 36373387, 14994345, 32022989, 28975684, 29840860 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2008-11-11"), "end-date": date("2008-03-06") } ] }
-{ "id": 9518128, "id-copy": 9518128, "alias": "Jerrie", "name": "JerrieFonblanque", "user-since": datetime("2008-06-08T02:51:53.000Z"), "user-since-copy": datetime("2008-06-08T02:51:53.000Z"), "friend-ids": {{ 41051692, 21547608, 41749297, 21528434, 28012731, 43579854, 9172140, 17908381, 10276804, 12277383, 38454166, 6950146, 11878198, 24415804, 46218827, 33013212, 44735001, 36395459, 38515534, 16015324, 21085620, 20338207 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-01-14") } ] }
-{ "id": 9522265, "id-copy": 9522265, "alias": "Brendon", "name": "BrendonLing", "user-since": datetime("2012-08-11T12:01:34.000Z"), "user-since-copy": datetime("2012-08-11T12:01:34.000Z"), "friend-ids": {{ 32770998, 43037450, 13481444, 36411834, 21704194 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2012-08-30") } ] }
-{ "id": 9543280, "id-copy": 9543280, "alias": "Isabell", "name": "IsabellGaskins", "user-since": datetime("2009-12-05T01:29:24.000Z"), "user-since-copy": datetime("2009-12-05T01:29:24.000Z"), "friend-ids": {{ 9815607, 43778761, 25835208, 40078303, 28971077, 9802833, 17822058, 12655680, 37398606, 11387722, 5483134, 11506312, 36341116, 13511812, 3504784, 11655484, 18350098, 15365006, 32814750 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-07-01"), "end-date": date("2007-08-14") } ] }
-{ "id": 9545626, "id-copy": 9545626, "alias": "Russell", "name": "RussellKeilbach", "user-since": datetime("2010-05-20T15:10:25.000Z"), "user-since-copy": datetime("2010-05-20T15:10:25.000Z"), "friend-ids": {{ 40592323, 28819303 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2003-04-18") } ] }
-{ "id": 9568750, "id-copy": 9568750, "alias": "Daley", "name": "DaleyHarshman", "user-since": datetime("2012-01-17T10:38:07.000Z"), "user-since-copy": datetime("2012-01-17T10:38:07.000Z"), "friend-ids": {{ 18932212, 37118057, 37586464, 12686041, 21083532, 27598912 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2007-11-07") } ] }
-{ "id": 9596080, "id-copy": 9596080, "alias": "Yolonda", "name": "YolondaUlery", "user-since": datetime("2012-03-02T19:57:32.000Z"), "user-since-copy": datetime("2012-03-02T19:57:32.000Z"), "friend-ids": {{ 22382589, 22012001, 13142890, 44320162, 10358341, 14975, 43101433, 10324321, 14791134, 25984312, 11075173, 44140537, 40528755, 27384004, 40022140, 10650900, 37789740, 6928495, 22130557, 47679224, 40973393, 37190617, 35395183 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2012-03-03") } ] }
-{ "id": 9597526, "id-copy": 9597526, "alias": "Emory", "name": "EmoryThorley", "user-since": datetime("2006-01-19T22:44:03.000Z"), "user-since-copy": datetime("2006-01-19T22:44:03.000Z"), "friend-ids": {{ 420066, 8047878, 20510786, 1639671, 22923859, 27319995, 3624690, 18526424, 45857863, 2830065, 4588990, 25531572, 17878497, 47796172, 41309806, 34307425, 10084701, 1659934, 38218970, 44720636, 43501970, 610796, 35455526, 2080900 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2011-06-18"), "end-date": date("2011-09-10") } ] }
-{ "id": 9602242, "id-copy": 9602242, "alias": "Marc", "name": "MarcDimsdale", "user-since": datetime("2005-10-03T23:32:18.000Z"), "user-since-copy": datetime("2005-10-03T23:32:18.000Z"), "friend-ids": {{ 34004502, 24469994, 2140538, 1486939, 6895407, 13348535, 22384921, 11662871, 21398307, 33070732, 45602509, 26146770, 24148813, 45988030, 22184030, 855669, 36390708, 30883354, 26360628, 29836897, 28696575 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2004-05-15"), "end-date": date("2008-01-19") } ] }
-{ "id": 9638248, "id-copy": 9638248, "alias": "Azucena", "name": "AzucenaEmrick", "user-since": datetime("2005-12-04T00:15:40.000Z"), "user-since-copy": datetime("2005-12-04T00:15:40.000Z"), "friend-ids": {{ 37210744, 43097413, 2901403, 24492031, 7887583, 42518446, 28555003, 20402754, 5506767, 22982986, 21168589, 37638670, 30930177, 43662522, 45627167, 13450586, 36757137, 46663990 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2002-07-07"), "end-date": date("2006-06-11") } ] }
-{ "id": 9646474, "id-copy": 9646474, "alias": "Lilac", "name": "LilacWoodworth", "user-since": datetime("2009-12-17T02:42:51.000Z"), "user-since-copy": datetime("2009-12-17T02:42:51.000Z"), "friend-ids": {{ 47784123, 45348808, 36392712, 9381262, 10215254, 1461251, 23038092, 44549001, 39097217, 41152823, 31758517, 19401493, 39964393, 46307214, 41683224, 39011968, 5014398, 482179, 3789628, 46257340, 36041029, 10903757, 5980810, 31935548 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2004-10-25"), "end-date": date("2005-05-04") } ] }
-{ "id": 9665848, "id-copy": 9665848, "alias": "Shannah", "name": "ShannahDale", "user-since": datetime("2006-08-09T02:09:51.000Z"), "user-since-copy": datetime("2006-08-09T02:09:51.000Z"), "friend-ids": {{ 19512022, 25217933, 21742776, 35558948, 5893317, 2441637, 6907563, 36626257, 3366834, 25069218, 5753530, 45604388, 33908296, 1048890, 5720452, 7923351, 43424884, 43184720, 29744229, 10349400, 15273614, 15283237, 41997307 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2010-12-28"), "end-date": date("2010-09-17") } ] }
-{ "id": 9674677, "id-copy": 9674677, "alias": "Skye", "name": "SkyeTomlinson", "user-since": datetime("2006-02-02T19:15:10.000Z"), "user-since-copy": datetime("2006-02-02T19:15:10.000Z"), "friend-ids": {{ 24282798, 5600117, 33292938, 19518197, 11735189, 22867735, 8029689, 11269147, 7443311, 45905216, 12859442, 26944030 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2011-05-07"), "end-date": date("2011-04-19") } ] }
-{ "id": 9677293, "id-copy": 9677293, "alias": "Owen", "name": "OwenHoenshell", "user-since": datetime("2005-06-28T02:54:49.000Z"), "user-since-copy": datetime("2005-06-28T02:54:49.000Z"), "friend-ids": {{ 1016713, 4999321, 27107303, 15587298 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2010-01-11") } ] }
-{ "id": 9698980, "id-copy": 9698980, "alias": "Leland", "name": "LelandReiss", "user-since": datetime("2012-05-23T04:40:29.000Z"), "user-since-copy": datetime("2012-05-23T04:40:29.000Z"), "friend-ids": {{ 7623016, 12672253, 42612513, 44457047, 46981337, 1098470, 23122899, 15019916, 45345438, 30272843, 43546610 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-11-27") } ] }
-{ "id": 9733942, "id-copy": 9733942, "alias": "Andra", "name": "AndraConrad", "user-since": datetime("2007-01-23T01:20:01.000Z"), "user-since-copy": datetime("2007-01-23T01:20:01.000Z"), "friend-ids": {{ 42791827, 36987912, 12650269, 5310067, 33419819, 36880069, 1146970, 20314, 10762565, 20657888, 31871678, 42279496, 9831201, 4223369, 46820320, 21703772, 1326858, 21739453, 20082273, 12950360 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2002-03-21") } ] }
-{ "id": 9744016, "id-copy": 9744016, "alias": "Kasha", "name": "KashaMueller", "user-since": datetime("2011-03-16T17:17:31.000Z"), "user-since-copy": datetime("2011-03-16T17:17:31.000Z"), "friend-ids": {{ 15857660, 46791109, 10310040, 42863950, 19533508, 32561502, 4367358, 31952243, 7130063, 19536081, 19870534, 3642001, 910385, 28668446, 33204842, 13210089, 2805429 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2000-11-01") } ] }
-{ "id": 9748939, "id-copy": 9748939, "alias": "April", "name": "AprilCourtney", "user-since": datetime("2008-02-10T17:35:28.000Z"), "user-since-copy": datetime("2008-02-10T17:35:28.000Z"), "friend-ids": {{ 43018591, 38860193, 26524230, 23704979, 2293321, 18201469, 41569073, 26942967, 16348102, 20218840, 30888146, 7584389, 11355443, 3703344 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2001-08-26") } ] }
-{ "id": 9769501, "id-copy": 9769501, "alias": "Geffrey", "name": "GeffreyBurch", "user-since": datetime("2005-08-28T03:10:56.000Z"), "user-since-copy": datetime("2005-08-28T03:10:56.000Z"), "friend-ids": {{ 21060169, 45384418, 20564855, 24708101, 30231, 29383832, 9200835, 822161, 29674263, 619991, 38797966, 14299510, 13545166, 33027152 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2004-01-03"), "end-date": date("2006-04-13") } ] }
-{ "id": 9773836, "id-copy": 9773836, "alias": "Harris", "name": "HarrisAshmore", "user-since": datetime("2005-11-09T08:38:57.000Z"), "user-since-copy": datetime("2005-11-09T08:38:57.000Z"), "friend-ids": {{ 8138978, 18579002, 42663609, 12096643, 38992166, 36937135, 19634600, 2103929, 37072923, 25031081, 13379299, 11238246, 23166598, 19181943, 45382447, 8237252, 30986231, 29591835 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2000-02-12") } ] }
-{ "id": 9802330, "id-copy": 9802330, "alias": "Kirby", "name": "KirbyKnopsnider", "user-since": datetime("2011-12-18T01:10:12.000Z"), "user-since-copy": datetime("2011-12-18T01:10:12.000Z"), "friend-ids": {{ 3703876, 46564552, 9263120, 39930137, 36202804, 45164241, 7778394, 2527495, 2831079, 33834588, 42759211, 2766215, 36344152, 5218620, 1190357, 30615313, 25434877, 43958817, 23617510 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2008-02-01") } ] }
-{ "id": 9804196, "id-copy": 9804196, "alias": "Micheal", "name": "MichealEiford", "user-since": datetime("2009-05-21T02:55:17.000Z"), "user-since-copy": datetime("2009-05-21T02:55:17.000Z"), "friend-ids": {{ 31376257, 19749408, 5790154, 17891222, 15712036, 40911870, 40765983, 38804584, 24619388, 10957577, 35370581, 39352927, 6063001, 23702369, 14716580, 46589395, 35232946 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2003-07-27") } ] }
-{ "id": 9814867, "id-copy": 9814867, "alias": "Pacey", "name": "PaceyBranson", "user-since": datetime("2011-07-05T06:49:42.000Z"), "user-since-copy": datetime("2011-07-05T06:49:42.000Z"), "friend-ids": {{ 7196953 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2005-11-19"), "end-date": date("2007-12-03") } ] }
-{ "id": 9822973, "id-copy": 9822973, "alias": "Melia", "name": "MeliaWentzel", "user-since": datetime("2012-07-17T05:10:30.000Z"), "user-since-copy": datetime("2012-07-17T05:10:30.000Z"), "friend-ids": {{ 2563633, 27918474, 42233962, 40497985, 4437912, 43013491, 47283180, 20434605, 25309336, 11299381, 20584869, 15093618, 14273412, 46920368, 5868827, 40191100, 44286983, 11787568, 44551406 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2001-07-07") } ] }
-{ "id": 9856990, "id-copy": 9856990, "alias": "Claud", "name": "ClaudBaird", "user-since": datetime("2006-10-10T11:48:09.000Z"), "user-since-copy": datetime("2006-10-10T11:48:09.000Z"), "friend-ids": {{ 41756695, 15842897, 29797715, 13771892, 21179308, 42974840, 22223660, 35004748, 35597685, 45300254, 31116834, 42699991, 9704157, 23181215, 14806554, 8198556, 16256974, 16360634, 34736641 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2008-07-23") } ] }
-{ "id": 9878209, "id-copy": 9878209, "alias": "Duana", "name": "DuanaGettemy", "user-since": datetime("2007-03-05T19:06:27.000Z"), "user-since-copy": datetime("2007-03-05T19:06:27.000Z"), "friend-ids": {{ 5530171, 22409344, 22742046, 14418589, 27149252 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-07") } ] }
-{ "id": 9880603, "id-copy": 9880603, "alias": "Davis", "name": "DavisRitter", "user-since": datetime("2009-12-18T18:55:46.000Z"), "user-since-copy": datetime("2009-12-18T18:55:46.000Z"), "friend-ids": {{ 10790833, 43529865, 23457220, 6745186, 22333440, 39380793, 2096806, 44121543, 29345888, 46499780, 31896682, 35084540, 6060378, 27402271, 18954641 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2002-01-11") } ] }
-{ "id": 9886819, "id-copy": 9886819, "alias": "Phoebe", "name": "PhoebeBarnes", "user-since": datetime("2010-12-26T07:30:15.000Z"), "user-since-copy": datetime("2010-12-26T07:30:15.000Z"), "friend-ids": {{ 24361962, 43750816, 46566991, 4790101, 38827567, 6893116, 41555542, 35877264, 18479056, 22186674, 10954414, 43453344, 11903159, 12257863, 45299776 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2000-01-02"), "end-date": date("2008-05-24") } ] }
-{ "id": 9904822, "id-copy": 9904822, "alias": "Judith", "name": "JudithChristman", "user-since": datetime("2005-05-19T14:43:44.000Z"), "user-since-copy": datetime("2005-05-19T14:43:44.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "highfax", "start-date": date("2002-05-06") } ] }
-{ "id": 9917008, "id-copy": 9917008, "alias": "Clancy", "name": "ClancyHector", "user-since": datetime("2007-09-25T20:55:57.000Z"), "user-since-copy": datetime("2007-09-25T20:55:57.000Z"), "friend-ids": {{ 37754545, 37579706, 39121342, 28434988, 3927416, 3794736, 17107964, 20761621, 20497172, 28562441, 4310488, 35121288, 2380560, 32434056 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2011-06-24") } ] }
-{ "id": 9939937, "id-copy": 9939937, "alias": "Margeret", "name": "MargeretWhite", "user-since": datetime("2008-10-10T22:07:17.000Z"), "user-since-copy": datetime("2008-10-10T22:07:17.000Z"), "friend-ids": {{ 12369844, 34252449, 12412010, 16942281, 25231122, 42326296, 27054531, 8338820, 25466132, 10175756, 23763550, 40035149, 41030740, 36493305, 19615682, 30813330, 24869907, 6934392, 31309446, 2545800, 463498, 3089623, 12714051, 38317605 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2012-01-19") } ] }
-{ "id": 9945166, "id-copy": 9945166, "alias": "Lilly", "name": "LillyPirl", "user-since": datetime("2009-10-26T11:59:59.000Z"), "user-since-copy": datetime("2009-10-26T11:59:59.000Z"), "friend-ids": {{ 44569094, 5885974, 43165146, 40353390, 45117914, 35995608, 22535699, 46288114, 47171829, 14193764, 45832182, 4957844, 2623547, 37294528 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2009-12-15"), "end-date": date("2011-11-20") } ] }
-{ "id": 9952342, "id-copy": 9952342, "alias": "Christal", "name": "ChristalMcmichaels", "user-since": datetime("2008-02-13T13:25:45.000Z"), "user-since-copy": datetime("2008-02-13T13:25:45.000Z"), "friend-ids": {{ 12290348, 1563117, 10883525, 17285406, 3798829, 3734533, 13084348, 31001579, 23655942, 44480002, 11803789, 8240833, 42718608, 41919526, 37582304, 10494964, 10815416, 10676699, 9376307 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2011-05-16") } ] }
-{ "id": 9970132, "id-copy": 9970132, "alias": "Garrett", "name": "GarrettPery", "user-since": datetime("2007-03-03T11:19:29.000Z"), "user-since-copy": datetime("2007-03-03T11:19:29.000Z"), "friend-ids": {{ 25744707, 31991833, 37406793, 30461766, 24815522, 3640470, 13669903, 17663561, 19222132, 29107132, 42516393, 40032051, 24029037, 28047983, 45579233 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2001-01-12") } ] }
-{ "id": 9979750, "id-copy": 9979750, "alias": "Reginald", "name": "ReginaldAltman", "user-since": datetime("2007-04-04T08:51:58.000Z"), "user-since-copy": datetime("2007-04-04T08:51:58.000Z"), "friend-ids": {{ 2988287 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2002-01-28") } ] }
-{ "id": 9985393, "id-copy": 9985393, "alias": "Whitaker", "name": "WhitakerMang", "user-since": datetime("2007-11-28T09:34:34.000Z"), "user-since-copy": datetime("2007-11-28T09:34:34.000Z"), "friend-ids": {{ 24107735, 37165967, 31305236, 35313360, 9261860, 32724193, 34416346, 8143882, 9029425, 26723829, 4545824 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2000-08-23"), "end-date": date("2008-08-06") } ] }
-{ "id": 9986206, "id-copy": 9986206, "alias": "Tatiana", "name": "TatianaAlbright", "user-since": datetime("2006-03-21T10:00:55.000Z"), "user-since-copy": datetime("2006-03-21T10:00:55.000Z"), "friend-ids": {{ 42869099, 40178170, 13922993, 28844962, 26206785, 41293581, 17131809, 1583964, 47236558, 2656158, 11008100, 3994698, 23764118, 14275676, 4922979, 28466879, 16454954, 3620561, 42044685, 12665882, 18354684 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2012-04-24") } ] }
-{ "id": 10000456, "id-copy": 10000456, "alias": "Love", "name": "LoveHawker", "user-since": datetime("2011-03-01T20:42:05.000Z"), "user-since-copy": datetime("2011-03-01T20:42:05.000Z"), "friend-ids": {{ 33646270, 5736885, 35243769, 35528678, 43954964, 44975821, 1839952, 24025196, 1108928 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2010-11-23"), "end-date": date("2011-03-07") } ] }
-{ "id": 10001080, "id-copy": 10001080, "alias": "Garrett", "name": "GarrettBode", "user-since": datetime("2005-10-25T18:07:35.000Z"), "user-since-copy": datetime("2005-10-25T18:07:35.000Z"), "friend-ids": {{ 35858744, 16426061, 11473961, 4769664, 29038930, 33070686, 46271872, 42593454, 36202882, 46642640, 22243678, 20222041, 29014540, 7389258, 7172909, 12787979, 146736, 21081030, 21615179, 2936936, 44934891 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2007-06-24") } ] }
-{ "id": 10002907, "id-copy": 10002907, "alias": "Maegan", "name": "MaeganErschoff", "user-since": datetime("2011-10-15T18:08:56.000Z"), "user-since-copy": datetime("2011-10-15T18:08:56.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2009-02-06"), "end-date": date("2011-05-20") } ] }
-{ "id": 10045915, "id-copy": 10045915, "alias": "Mona", "name": "MonaMarshall", "user-since": datetime("2005-08-24T06:03:43.000Z"), "user-since-copy": datetime("2005-08-24T06:03:43.000Z"), "friend-ids": {{ 34157870, 1960568, 39038094, 2842182, 12353591, 44464974, 45836337, 4831806, 18179039, 21060089, 15776264, 41865218, 5999176, 18197780 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-07-28") } ] }
-{ "id": 10089976, "id-copy": 10089976, "alias": "Marion", "name": "MarionThomlinson", "user-since": datetime("2006-06-27T14:11:49.000Z"), "user-since-copy": datetime("2006-06-27T14:11:49.000Z"), "friend-ids": {{ 39404598, 46190974, 43413339, 41250692, 4194349, 5150083, 35574492, 30896673, 15969653, 41889132, 38801872, 17834003, 42587459, 42269051, 20206793, 46257713, 2735409, 28567746, 6641216, 3627253, 15945805, 33861471, 9997931, 38242090 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2011-11-22"), "end-date": date("2011-06-01") } ] }
-{ "id": 10108534, "id-copy": 10108534, "alias": "Moriah", "name": "MoriahMitchell", "user-since": datetime("2005-11-13T21:32:41.000Z"), "user-since-copy": datetime("2005-11-13T21:32:41.000Z"), "friend-ids": {{ 30372632 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-05-07") } ] }
-{ "id": 10114891, "id-copy": 10114891, "alias": "Destinee", "name": "DestineeLeech", "user-since": datetime("2006-06-05T09:32:17.000Z"), "user-since-copy": datetime("2006-06-05T09:32:17.000Z"), "friend-ids": {{ 9925448, 28685906, 3305693, 11131758, 10477741, 19058196, 25921997, 38543939, 20851041 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2001-09-24") } ] }
-{ "id": 10118077, "id-copy": 10118077, "alias": "Elizbeth", "name": "ElizbethPfeifer", "user-since": datetime("2011-09-08T11:58:48.000Z"), "user-since-copy": datetime("2011-09-08T11:58:48.000Z"), "friend-ids": {{ 18001251, 40309720, 10119557, 37766102, 22202316, 2805709, 693628, 5524288, 21415560, 45687644, 23912525, 25418741, 22816155, 26787291, 30518473, 27701649 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2002-03-15"), "end-date": date("2004-11-03") } ] }
-{ "id": 10133458, "id-copy": 10133458, "alias": "Kati", "name": "KatiPennington", "user-since": datetime("2011-01-28T10:51:37.000Z"), "user-since-copy": datetime("2011-01-28T10:51:37.000Z"), "friend-ids": {{ 41299906, 11523198, 8344474, 36086944, 34330342, 43585884, 6751565, 23415221, 32275829, 43645200 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2005-11-11") } ] }
-{ "id": 10136659, "id-copy": 10136659, "alias": "Robt", "name": "RobtKooser", "user-since": datetime("2008-11-08T19:22:49.000Z"), "user-since-copy": datetime("2008-11-08T19:22:49.000Z"), "friend-ids": {{ 22245145, 29285750, 9880896 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-02-07") } ] }
-{ "id": 10178182, "id-copy": 10178182, "alias": "Jen", "name": "JenOtis", "user-since": datetime("2007-08-09T09:42:29.000Z"), "user-since-copy": datetime("2007-08-09T09:42:29.000Z"), "friend-ids": {{ 26278603, 27983753, 13714345, 35452213, 27849291, 21838200, 1008530, 27777115, 27069057, 35804914, 34598070, 10076890, 12795361, 16653787, 2916026, 27047674, 8630755, 29822673 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2005-10-10") } ] }
-{ "id": 10185346, "id-copy": 10185346, "alias": "Noah", "name": "NoahAshmore", "user-since": datetime("2006-04-04T14:33:43.000Z"), "user-since-copy": datetime("2006-04-04T14:33:43.000Z"), "friend-ids": {{ 15819384, 46052301, 7102428, 7977240, 30337629, 31480307, 30013142, 4192580, 34814572, 6841517, 2253788, 31150059, 505825, 27897490, 11402219 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-06-06") } ] }
-{ "id": 10202302, "id-copy": 10202302, "alias": "Camila", "name": "CamilaKelley", "user-since": datetime("2010-04-17T06:57:52.000Z"), "user-since-copy": datetime("2010-04-17T06:57:52.000Z"), "friend-ids": {{ 21392718, 41703679, 41044232, 47307848, 13912958, 45329595, 33360889, 24572594, 23726460, 9181899, 42227287, 26565775, 12665691, 12244453, 26966326, 3189268, 41340076, 33904406, 38048631, 22870005 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2000-10-09") } ] }
-{ "id": 10205539, "id-copy": 10205539, "alias": "Raeburn", "name": "RaeburnWire", "user-since": datetime("2007-04-28T23:05:24.000Z"), "user-since-copy": datetime("2007-04-28T23:05:24.000Z"), "friend-ids": {{ 13609724, 40251506 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2000-08-09") } ] }
-{ "id": 10207636, "id-copy": 10207636, "alias": "Stewart", "name": "StewartHamilton", "user-since": datetime("2008-11-06T21:44:47.000Z"), "user-since-copy": datetime("2008-11-06T21:44:47.000Z"), "friend-ids": {{ 25417411, 7322723, 13495699, 47274757, 44964322, 4993843, 36429109, 11904558, 18759232, 45446850, 40537858, 40487724, 36200691, 6846408, 7421262, 2225424, 12997194 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2012-08-28"), "end-date": date("2012-08-29") } ] }
-{ "id": 10219465, "id-copy": 10219465, "alias": "Ros", "name": "RosSurrency", "user-since": datetime("2010-04-20T12:07:16.000Z"), "user-since-copy": datetime("2010-04-20T12:07:16.000Z"), "friend-ids": {{ 14365151, 47786936, 41386448, 10958072, 34068903, 28844652, 16749120, 16920092, 7474357, 35730197, 13732713, 26185093, 19486844, 13720196, 7483494, 16709415, 32998666, 31641404, 42939361, 20750447, 44343030, 17559252, 13810932 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-12-12"), "end-date": date("2010-05-04") } ] }
-{ "id": 10222144, "id-copy": 10222144, "alias": "Alvina", "name": "AlvinaTanner", "user-since": datetime("2007-10-15T04:24:14.000Z"), "user-since-copy": datetime("2007-10-15T04:24:14.000Z"), "friend-ids": {{ 44207447, 29837430, 407059, 4562324, 970458, 31348025, 16439061, 13011150, 23510630, 21529259, 8279487, 28052530, 36551405, 17492050, 17983056, 11834104, 242520, 9279232, 4179609, 28407763, 23038009, 36977762, 8779957, 15040402 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2006-05-27") } ] }
-{ "id": 10224400, "id-copy": 10224400, "alias": "Malvina", "name": "MalvinaPery", "user-since": datetime("2009-01-25T03:41:22.000Z"), "user-since-copy": datetime("2009-01-25T03:41:22.000Z"), "friend-ids": {{ 17095877, 17062955, 13129292, 31635980, 32747924, 902714, 32032985, 44944935, 30544897, 44429244 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2001-01-11"), "end-date": date("2011-04-10") } ] }
-{ "id": 10227844, "id-copy": 10227844, "alias": "Simon", "name": "SimonCoates", "user-since": datetime("2008-09-18T06:23:35.000Z"), "user-since-copy": datetime("2008-09-18T06:23:35.000Z"), "friend-ids": {{ 5847048, 15554997, 1367924, 17223026, 31605674, 38148868, 15521228, 37540102, 4103855, 39184726, 26130198, 43081715, 35929397, 28963043, 10703925 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2011-07-02") } ] }
-{ "id": 10230604, "id-copy": 10230604, "alias": "Courtney", "name": "CourtneyCountryman", "user-since": datetime("2012-03-05T08:49:56.000Z"), "user-since-copy": datetime("2012-03-05T08:49:56.000Z"), "friend-ids": {{ 28617094, 31170285, 26700577, 43586990, 12809105, 8131401, 15644912, 38127923, 7871621, 13276397, 41863539, 3715524, 13404150, 12834697, 237361, 41295097, 29471386, 19859329, 14312407, 79917, 42547367, 9661712, 30110962, 29137807 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2001-06-09"), "end-date": date("2004-06-04") } ] }
-{ "id": 10250857, "id-copy": 10250857, "alias": "Kandi", "name": "KandiFranks", "user-since": datetime("2010-11-24T19:47:41.000Z"), "user-since-copy": datetime("2010-11-24T19:47:41.000Z"), "friend-ids": {{ 44991748, 27655130, 7925482, 33419150, 18275478 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2002-04-08") } ] }
-{ "id": 10251805, "id-copy": 10251805, "alias": "Jericho", "name": "JerichoBaird", "user-since": datetime("2005-07-02T12:57:18.000Z"), "user-since-copy": datetime("2005-07-02T12:57:18.000Z"), "friend-ids": {{ 5748549, 47013396, 15858292, 458526, 28324553, 22401875, 21726858, 38878600, 29844738, 14547049, 11432495, 9227475 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2005-06-09"), "end-date": date("2011-11-01") } ] }
-{ "id": 10257028, "id-copy": 10257028, "alias": "Gary", "name": "GaryThompson", "user-since": datetime("2009-01-23T04:15:30.000Z"), "user-since-copy": datetime("2009-01-23T04:15:30.000Z"), "friend-ids": {{ 46006273, 33435458, 40976127, 42353737, 37166855, 14882549, 27357892, 31126471, 38151307, 38721200 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2008-05-03"), "end-date": date("2011-09-08") } ] }
-{ "id": 10258114, "id-copy": 10258114, "alias": "Chuck", "name": "ChuckGibson", "user-since": datetime("2012-07-20T03:48:15.000Z"), "user-since-copy": datetime("2012-07-20T03:48:15.000Z"), "friend-ids": {{ 32318205, 37049120, 26298456, 3281723, 14892306, 29998569, 29992020, 36383932, 15333422, 29670243 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2006-10-11"), "end-date": date("2011-09-02") } ] }
-{ "id": 10267057, "id-copy": 10267057, "alias": "Thomas", "name": "ThomasCook", "user-since": datetime("2008-03-02T23:04:31.000Z"), "user-since-copy": datetime("2008-03-02T23:04:31.000Z"), "friend-ids": {{ 23744020, 25995598, 40459051, 27658275, 10133202, 11434833, 29790727, 1672639, 19652058, 18554997, 37878642, 48016133, 46599310, 37105777, 36004129, 6402365, 9889815, 29589019, 1497208, 19269802, 43383394, 30936085 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2008-05-14"), "end-date": date("2008-07-10") } ] }
-{ "id": 10269349, "id-copy": 10269349, "alias": "Oneida", "name": "OneidaJube", "user-since": datetime("2010-11-18T02:17:28.000Z"), "user-since-copy": datetime("2010-11-18T02:17:28.000Z"), "friend-ids": {{ 12058841, 5816839, 33989309, 42710608, 27128355, 22765769, 30666197, 9009086, 7254731, 41783149, 10080163, 38431373, 35086196, 3607650 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2000-12-10") } ] }
-{ "id": 10271479, "id-copy": 10271479, "alias": "Leah", "name": "LeahKoepple", "user-since": datetime("2007-10-26T15:57:39.000Z"), "user-since-copy": datetime("2007-10-26T15:57:39.000Z"), "friend-ids": {{ 317362, 43304286, 35630504, 16014770, 43567734, 37946435, 7728583, 45620359, 43235478, 17133820, 22926471, 27438784, 43521614, 235789, 43107565, 21967424, 39119573, 1688079, 5463246, 10081045 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2012-06-14") } ] }
-{ "id": 10278607, "id-copy": 10278607, "alias": "Brenden", "name": "BrendenLombardi", "user-since": datetime("2012-02-13T05:59:40.000Z"), "user-since-copy": datetime("2012-02-13T05:59:40.000Z"), "friend-ids": {{ 2820692, 43529738, 38518064, 29672334, 24653037, 39717291, 14213502, 23982828, 47123006, 34213620, 5993185, 10068793, 47512414, 40682283, 26631237, 23442819, 9215972, 9003752, 31259126, 8467245, 32821220, 8582002, 42606040 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2001-06-16"), "end-date": date("2008-09-11") } ] }
-{ "id": 10280533, "id-copy": 10280533, "alias": "Normand", "name": "NormandAckerley", "user-since": datetime("2008-05-18T00:44:35.000Z"), "user-since-copy": datetime("2008-05-18T00:44:35.000Z"), "friend-ids": {{ 46908522, 2002203, 15632192, 3790633, 21300428, 15452344, 34478785, 18864214, 32842683, 10486268, 2496859 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2010-12-07") } ] }
-{ "id": 10300027, "id-copy": 10300027, "alias": "Cassie", "name": "CassieCarmichael", "user-since": datetime("2007-02-17T16:12:21.000Z"), "user-since-copy": datetime("2007-02-17T16:12:21.000Z"), "friend-ids": {{ 18690821, 9246387, 5425670, 8058755, 32156367, 29092478 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-03-06") } ] }
-{ "id": 10301008, "id-copy": 10301008, "alias": "Edgardo", "name": "EdgardoWheeler", "user-since": datetime("2012-04-27T03:11:16.000Z"), "user-since-copy": datetime("2012-04-27T03:11:16.000Z"), "friend-ids": {{ 44525957, 2368018 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2004-07-02"), "end-date": date("2009-04-13") } ] }
-{ "id": 10320979, "id-copy": 10320979, "alias": "Giuseppe", "name": "GiuseppePorter", "user-since": datetime("2006-10-21T21:56:23.000Z"), "user-since-copy": datetime("2006-10-21T21:56:23.000Z"), "friend-ids": {{ 34102109, 41585396, 8170669, 7376463, 11841426, 6745396, 35637670, 38513040, 26085708, 7577827, 4793535, 31185038, 9126, 502656, 18672743, 27688404, 19846788, 47731814, 42609593 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2012-06-08") } ] }
-{ "id": 10323868, "id-copy": 10323868, "alias": "Floyd", "name": "FloydCostello", "user-since": datetime("2007-12-17T05:45:55.000Z"), "user-since-copy": datetime("2007-12-17T05:45:55.000Z"), "friend-ids": {{ 16296950, 29360254, 19980961, 43395913, 46984972, 2696536, 9715184, 10851075, 25657111, 46730259, 9182621, 31950695, 46717390, 16664917, 38439464, 6987406, 28167105, 10608129, 11375117, 4306430, 31737185, 29321535, 7420588 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2000-08-21") } ] }
-{ "id": 10337950, "id-copy": 10337950, "alias": "Bibi", "name": "BibiCattley", "user-since": datetime("2007-11-16T11:08:34.000Z"), "user-since-copy": datetime("2007-11-16T11:08:34.000Z"), "friend-ids": {{ 24399247, 18391359, 18215808, 36042641, 19360937, 2039633, 17280287, 22159187, 31245932, 4767019, 3299881, 12321916, 22533524, 18760130, 31303729, 39565694, 21606207, 8226305, 16276064 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2003-02-25"), "end-date": date("2008-08-20") } ] }
-{ "id": 10365688, "id-copy": 10365688, "alias": "Innocent", "name": "InnocentBlatenberger", "user-since": datetime("2008-11-09T13:57:34.000Z"), "user-since-copy": datetime("2008-11-09T13:57:34.000Z"), "friend-ids": {{ 27902413, 27226238, 35017422, 28154221 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2011-09-13"), "end-date": date("2011-02-05") } ] }
-{ "id": 10367503, "id-copy": 10367503, "alias": "Tory", "name": "ToryBender", "user-since": datetime("2012-01-17T03:20:23.000Z"), "user-since-copy": datetime("2012-01-17T03:20:23.000Z"), "friend-ids": {{ 12035968, 32370161, 7506904, 40525754, 44978940, 28927429, 47139832, 9164811, 29534171, 3789973 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2009-02-05") } ] }
-{ "id": 10380031, "id-copy": 10380031, "alias": "Otha", "name": "OthaHaines", "user-since": datetime("2005-08-08T04:10:50.000Z"), "user-since-copy": datetime("2005-08-08T04:10:50.000Z"), "friend-ids": {{ 2710866, 28894512, 36379679, 32545673, 38671874, 16746916, 39103475, 19783615, 17514492, 42617267, 7461114, 17712393, 43474200, 3806350, 5065542, 35722940 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2011-05-07") } ] }
-{ "id": 10384705, "id-copy": 10384705, "alias": "Santos", "name": "SantosJames", "user-since": datetime("2011-05-07T11:54:13.000Z"), "user-since-copy": datetime("2011-05-07T11:54:13.000Z"), "friend-ids": {{ 43937179, 34015979, 7417213, 14660995, 19725400, 3931428, 7318379, 48016396, 44068471, 4577462, 38302695, 16520658, 40487183, 31181305, 11750148, 42688348, 42071075, 10641987, 28860865, 27686448, 40844612, 10817134 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2007-12-18") } ] }
-{ "id": 10391179, "id-copy": 10391179, "alias": "Raymond", "name": "RaymondHoopengarner", "user-since": datetime("2006-04-06T18:32:20.000Z"), "user-since-copy": datetime("2006-04-06T18:32:20.000Z"), "friend-ids": {{ 35664656, 36940003, 35836359, 25322876, 45895708, 14553421 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2005-08-05"), "end-date": date("2007-01-09") } ] }
-{ "id": 10394632, "id-copy": 10394632, "alias": "Marlin", "name": "MarlinLogue", "user-since": datetime("2011-08-28T14:57:40.000Z"), "user-since-copy": datetime("2011-08-28T14:57:40.000Z"), "friend-ids": {{ 45667126 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2004-07-03"), "end-date": date("2009-05-09") } ] }
-{ "id": 10400386, "id-copy": 10400386, "alias": "Marion", "name": "MarionBuck", "user-since": datetime("2006-06-22T03:35:25.000Z"), "user-since-copy": datetime("2006-06-22T03:35:25.000Z"), "friend-ids": {{ 35854700, 8766966, 41860546, 25745457, 12225165, 15412904, 39841282, 5879215, 24965438, 4636142, 43652954, 36414405, 34931848, 38550959, 30395999, 44263220, 8167212, 35555246, 11177002, 29078503 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2000-08-28") } ] }
-{ "id": 10413097, "id-copy": 10413097, "alias": "Lindsay", "name": "LindsayDoverspike", "user-since": datetime("2005-03-24T22:42:49.000Z"), "user-since-copy": datetime("2005-03-24T22:42:49.000Z"), "friend-ids": {{ 773762, 43764188, 23133486, 27099138, 38010544, 38283504, 38432745, 32450505, 34499948, 38200436, 44093983, 41684052, 41353940, 29027114, 2947798, 25212070, 9522627, 18680730, 13060818, 41586559 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2010-01-23"), "end-date": date("2011-01-14") } ] }
-{ "id": 10415575, "id-copy": 10415575, "alias": "Amabel", "name": "AmabelRoose", "user-since": datetime("2011-05-28T10:47:28.000Z"), "user-since-copy": datetime("2011-05-28T10:47:28.000Z"), "friend-ids": {{ 22120342, 22881927, 39043768, 27695122, 8669783, 25973892 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2000-03-14") } ] }
-{ "id": 10417531, "id-copy": 10417531, "alias": "Eileen", "name": "EileenCrissman", "user-since": datetime("2009-10-13T21:36:38.000Z"), "user-since-copy": datetime("2009-10-13T21:36:38.000Z"), "friend-ids": {{ 911579, 3590209, 15646563, 31960066, 14495212, 44915460, 42713118, 1962949, 44935091, 6578467, 21896024, 41455809, 25543039, 28884330, 44289305, 15569750, 32580470, 46016098, 9828368 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2003-06-11"), "end-date": date("2005-10-02") } ] }
-{ "id": 10451932, "id-copy": 10451932, "alias": "Kory", "name": "KoryRomanoff", "user-since": datetime("2008-09-27T13:29:18.000Z"), "user-since-copy": datetime("2008-09-27T13:29:18.000Z"), "friend-ids": {{ 21328124, 47569968, 22569123, 34316877, 36016117, 19944396, 34862141, 14875173, 3888684, 25235679, 7930355, 24991146, 2862320, 9552488, 23394143, 6292732, 23109993 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2008-04-25"), "end-date": date("2010-03-18") } ] }
-{ "id": 10458316, "id-copy": 10458316, "alias": "Nivek", "name": "NivekHarper", "user-since": datetime("2009-06-27T16:14:07.000Z"), "user-since-copy": datetime("2009-06-27T16:14:07.000Z"), "friend-ids": {{ 28377255, 40295259, 41434117, 37075748, 12913111, 1533923, 393103, 31161713, 13106373, 924904, 14927212, 7552938, 8299772, 28404911, 45464821, 34440085, 36216015, 2915789, 13470222, 34755494, 29250423 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2002-03-28"), "end-date": date("2010-12-09") } ] }
-{ "id": 10479073, "id-copy": 10479073, "alias": "Rhianna", "name": "RhiannaWerry", "user-since": datetime("2009-09-17T19:42:47.000Z"), "user-since-copy": datetime("2009-09-17T19:42:47.000Z"), "friend-ids": {{ 30293616, 42971604, 8411318, 37648744, 27412687, 17821200, 45008072 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2004-11-02"), "end-date": date("2011-06-24") } ] }
-{ "id": 10479190, "id-copy": 10479190, "alias": "Carmine", "name": "CarmineMortland", "user-since": datetime("2011-06-18T02:57:13.000Z"), "user-since-copy": datetime("2011-06-18T02:57:13.000Z"), "friend-ids": {{ 36090597, 35550849, 19614765, 34665409, 7740163, 12824683, 12997403, 32586142, 10137983, 44900811, 30392212, 43177710, 47792212 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2002-02-23") } ] }
-{ "id": 10492168, "id-copy": 10492168, "alias": "Savannah", "name": "SavannahRobinson", "user-since": datetime("2008-05-02T04:19:01.000Z"), "user-since-copy": datetime("2008-05-02T04:19:01.000Z"), "friend-ids": {{ 40126719, 38171650, 1474355, 6983398, 7918678, 45578368, 3210188, 29374863, 37758187, 2415003, 13746140, 44168763, 45798029, 17203664, 46309082, 21338452, 17217009, 24916114 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-07-20"), "end-date": date("2009-03-01") } ] }
-{ "id": 10493269, "id-copy": 10493269, "alias": "Anya", "name": "AnyaWoodward", "user-since": datetime("2009-03-08T07:08:04.000Z"), "user-since-copy": datetime("2009-03-08T07:08:04.000Z"), "friend-ids": {{ 2357333 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2001-05-04") } ] }
-{ "id": 10494370, "id-copy": 10494370, "alias": "Maria", "name": "MariaToke", "user-since": datetime("2009-12-06T17:40:38.000Z"), "user-since-copy": datetime("2009-12-06T17:40:38.000Z"), "friend-ids": {{ 28240347, 34042532 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2001-08-08"), "end-date": date("2008-07-09") } ] }
-{ "id": 10503262, "id-copy": 10503262, "alias": "Suzanne", "name": "SuzanneFonblanque", "user-since": datetime("2012-03-16T20:22:06.000Z"), "user-since-copy": datetime("2012-03-16T20:22:06.000Z"), "friend-ids": {{ 17868500, 500991, 7701699, 45401842, 16746916, 24217608, 46250003, 17567888, 28186634 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2001-01-18"), "end-date": date("2005-08-07") } ] }
-{ "id": 10509676, "id-copy": 10509676, "alias": "Dinorah", "name": "DinorahRopes", "user-since": datetime("2009-12-05T06:00:03.000Z"), "user-since-copy": datetime("2009-12-05T06:00:03.000Z"), "friend-ids": {{ 13297859, 17139775, 6500776, 46867326, 18510471, 20417055, 39500392, 2482383, 3361807, 14184772, 24928547, 14390842, 40519232, 14991589, 21242930, 24964529, 38160860, 25523267, 4709228, 13473948, 15850888, 30150938, 5984402, 26343874 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2010-08-11") } ] }
-{ "id": 10514095, "id-copy": 10514095, "alias": "Chantelle", "name": "ChantelleCatleay", "user-since": datetime("2008-10-23T00:05:15.000Z"), "user-since-copy": datetime("2008-10-23T00:05:15.000Z"), "friend-ids": {{ 11871759, 1505524, 45483061, 31479407, 15112731, 41816114, 22650998 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2008-06-14") } ] }
-{ "id": 10540825, "id-copy": 10540825, "alias": "Jayna", "name": "JaynaRowe", "user-since": datetime("2008-01-09T12:09:19.000Z"), "user-since-copy": datetime("2008-01-09T12:09:19.000Z"), "friend-ids": {{ 20315422, 9358699, 6204561, 40594838, 46678685, 34224970, 47262413, 42477325, 7591560, 39986319, 9438124, 30292072, 11187685, 27885, 47428887, 9535830, 36979072, 14613793 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2005-11-10") } ] }
-{ "id": 10567702, "id-copy": 10567702, "alias": "Zelda", "name": "ZeldaRitter", "user-since": datetime("2010-09-27T12:52:54.000Z"), "user-since-copy": datetime("2010-09-27T12:52:54.000Z"), "friend-ids": {{ 28336161, 20248788, 24723848, 8856879, 16831898, 7643547, 42868543, 23023606, 7531861, 36186817, 29113040, 995506, 32607538, 18755505, 44683178, 24627205, 39736850, 43535271, 385416, 40525568 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-11-27"), "end-date": date("2011-08-16") } ] }
-{ "id": 10582339, "id-copy": 10582339, "alias": "Randall", "name": "RandallDrabble", "user-since": datetime("2006-09-08T10:08:58.000Z"), "user-since-copy": datetime("2006-09-08T10:08:58.000Z"), "friend-ids": {{ 32686522, 24466673, 14026712, 31573032, 14639819, 19975138, 30208386, 24174917, 7234882, 9431452, 18256175, 18934583, 31539286, 46107937, 32747992, 28900739, 40079932, 40674667, 33527888, 45927633, 22350243, 14260823, 19696930, 17970296 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2008-12-13") } ] }
-{ "id": 10602166, "id-copy": 10602166, "alias": "Karine", "name": "KarineAdams", "user-since": datetime("2006-03-03T20:36:12.000Z"), "user-since-copy": datetime("2006-03-03T20:36:12.000Z"), "friend-ids": {{ 4463206, 23962283, 34321170, 10546383, 39886106, 37478996 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2000-10-28"), "end-date": date("2010-04-26") } ] }
-{ "id": 10623790, "id-copy": 10623790, "alias": "Leon", "name": "LeonSouthern", "user-since": datetime("2006-08-26T12:47:17.000Z"), "user-since-copy": datetime("2006-08-26T12:47:17.000Z"), "friend-ids": {{ 15974929, 10054172, 9775689, 22060162, 41777649, 13548836, 10842789, 45455670, 32027368, 45268626, 40570545, 18214851, 47559589, 38267347, 41101925, 45749689, 29277572, 47828706, 45708476, 33769625 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2012-06-22"), "end-date": date("2012-06-05") } ] }
-{ "id": 10650526, "id-copy": 10650526, "alias": "Gertie", "name": "GertieWallace", "user-since": datetime("2010-07-16T05:33:07.000Z"), "user-since-copy": datetime("2010-07-16T05:33:07.000Z"), "friend-ids": {{ 35934417, 43053648, 35859770, 43704932, 35605486, 17212020, 21235775, 26783725, 17450538, 42996452, 15873053, 36331217, 18524993, 45483950, 1549676, 24801562, 46527491 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2003-08-16") } ] }
-{ "id": 10661566, "id-copy": 10661566, "alias": "Cathy", "name": "CathyKight", "user-since": datetime("2007-07-17T18:53:31.000Z"), "user-since-copy": datetime("2007-07-17T18:53:31.000Z"), "friend-ids": {{ 19477294, 31919442, 6947933, 16858850, 21921187, 21214480, 19616226, 2133662, 42362248, 7534944, 12953803, 41148200, 30043772, 38130157, 36623612, 45371575, 25019205, 10260656 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2008-12-09"), "end-date": date("2008-01-04") } ] }
-{ "id": 10662082, "id-copy": 10662082, "alias": "Colbert", "name": "ColbertFylbrigg", "user-since": datetime("2005-04-09T18:04:54.000Z"), "user-since-copy": datetime("2005-04-09T18:04:54.000Z"), "friend-ids": {{ 25358191, 27442450, 16828484, 16821866, 7010321, 35271072, 32519925, 15521808, 35168957, 36812363, 18888093, 45727757, 30009499, 31505405, 27925036, 47549214, 20290733, 18290760, 36238437, 32377676 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-04-02") } ] }
-{ "id": 10663741, "id-copy": 10663741, "alias": "Gaylord", "name": "GaylordWynne", "user-since": datetime("2007-09-07T09:15:35.000Z"), "user-since-copy": datetime("2007-09-07T09:15:35.000Z"), "friend-ids": {{ 34508923, 28228552, 7714885, 16525247, 30914675, 8152699, 26553788, 8070452, 45739728 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2006-05-18"), "end-date": date("2008-04-07") } ] }
-{ "id": 10700431, "id-copy": 10700431, "alias": "Lessie", "name": "LessieRobinson", "user-since": datetime("2011-02-03T18:31:41.000Z"), "user-since-copy": datetime("2011-02-03T18:31:41.000Z"), "friend-ids": {{ 8174251, 46379649, 3507858, 13269282, 38334885, 12074283, 34128956, 46802811, 37285621, 15203773, 17611824, 47823053, 28609781, 31377970, 11077457, 3771375, 27529933, 170454, 38682017 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2002-01-08"), "end-date": date("2006-06-08") } ] }
-{ "id": 10712731, "id-copy": 10712731, "alias": "Abigail", "name": "AbigailKunkle", "user-since": datetime("2011-07-20T07:10:43.000Z"), "user-since-copy": datetime("2011-07-20T07:10:43.000Z"), "friend-ids": {{ 35920648, 38798778, 17160209, 16674423, 44247736, 45731986, 29605307, 123608, 46926535, 41274265, 36397206, 16900492, 19895463, 10043680, 42549381, 21006240, 13037274, 25867242, 34428167, 953419, 2284340, 32321044, 2351589, 30797666 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2002-08-11"), "end-date": date("2002-12-01") } ] }
-{ "id": 10735369, "id-copy": 10735369, "alias": "Cody", "name": "CodySchaeffer", "user-since": datetime("2008-07-03T05:27:24.000Z"), "user-since-copy": datetime("2008-07-03T05:27:24.000Z"), "friend-ids": {{ 15534779, 12333665, 10468027, 3865324, 39537208, 16999101, 9009757, 318331, 30123714, 10137427, 16481424 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2009-01-19") } ] }
-{ "id": 10739446, "id-copy": 10739446, "alias": "Urban", "name": "UrbanHair", "user-since": datetime("2010-12-28T02:29:19.000Z"), "user-since-copy": datetime("2010-12-28T02:29:19.000Z"), "friend-ids": {{ 31947556, 39058269, 43315882, 40575729, 4079275, 40689246, 22639555, 1422452, 28051313, 41854009, 30810426, 37406811, 20834349, 46933622, 28218698, 17239481, 33458180 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2006-05-21") } ] }
-{ "id": 10749553, "id-copy": 10749553, "alias": "Rolland", "name": "RollandMunshower", "user-since": datetime("2005-12-26T19:26:32.000Z"), "user-since-copy": datetime("2005-12-26T19:26:32.000Z"), "friend-ids": {{ 27080985, 4355429, 17027260, 30203290, 37292858, 1935550, 467329, 24265915, 4926329, 28586308, 27299677, 25356918, 14171255, 319307, 15014794 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2011-04-21") } ] }
-{ "id": 10777072, "id-copy": 10777072, "alias": "Fairy", "name": "FairyAgg", "user-since": datetime("2011-08-22T17:08:52.000Z"), "user-since-copy": datetime("2011-08-22T17:08:52.000Z"), "friend-ids": {{ 30447177, 24535470, 1763903, 4456057, 35013322 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2009-02-19") } ] }
-{ "id": 10783822, "id-copy": 10783822, "alias": "Emerald", "name": "EmeraldMillard", "user-since": datetime("2008-08-07T16:33:44.000Z"), "user-since-copy": datetime("2008-08-07T16:33:44.000Z"), "friend-ids": {{ 22464360, 7890894, 18256597, 33659179, 24554534, 30962087, 29716339, 23689397, 45113518, 19997635 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-06-10"), "end-date": date("2006-12-02") } ] }
-{ "id": 10799674, "id-copy": 10799674, "alias": "Dolores", "name": "DoloresPolson", "user-since": datetime("2006-03-24T00:54:47.000Z"), "user-since-copy": datetime("2006-03-24T00:54:47.000Z"), "friend-ids": {{ 40482317, 21393644, 151122, 13958566, 6524741, 1269094, 34703787, 38215473, 20258639, 144407, 23903205, 46922014, 26741209, 34932062, 1043581, 14090176, 45243069, 19226320, 33271281, 20215000, 46383495, 42405679, 42360649 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2009-07-18") } ] }
-{ "id": 10804771, "id-copy": 10804771, "alias": "Delicia", "name": "DeliciaPittman", "user-since": datetime("2008-04-12T01:07:13.000Z"), "user-since-copy": datetime("2008-04-12T01:07:13.000Z"), "friend-ids": {{ 35228090 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2006-08-16") } ] }
-{ "id": 10836430, "id-copy": 10836430, "alias": "Kaycee", "name": "KayceeCatleay", "user-since": datetime("2007-05-18T07:19:02.000Z"), "user-since-copy": datetime("2007-05-18T07:19:02.000Z"), "friend-ids": {{ 40568633, 44667158, 18923311, 34987631, 29306332, 38711535, 43999451, 3179954, 9799980, 3451381, 23204288, 17797804, 2164448, 16697308, 24697554, 45250786, 10029328, 27871642 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-04-25") } ] }
-{ "id": 10837876, "id-copy": 10837876, "alias": "Tianna", "name": "TiannaOppenheimer", "user-since": datetime("2006-05-14T01:19:23.000Z"), "user-since-copy": datetime("2006-05-14T01:19:23.000Z"), "friend-ids": {{ 8389212, 20540523, 37708985, 22298925, 5938365, 34705514, 39174355, 44283530, 44597508, 37912034, 45434053, 47086440, 6559664, 12451920, 47639456, 39030619, 24239344, 2566247, 27102794 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2008-04-03") } ] }
-{ "id": 10847359, "id-copy": 10847359, "alias": "Leone", "name": "LeoneWood", "user-since": datetime("2005-07-28T14:24:43.000Z"), "user-since-copy": datetime("2005-07-28T14:24:43.000Z"), "friend-ids": {{ 7650486, 39843416, 43272193, 47152762, 45218041, 45422234, 46812876, 18098636, 47174431, 19091549, 1405281, 46699360, 37961345, 43323551, 46824225, 30700451, 10188790, 16642374, 26570751 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2005-01-22") } ] }
-{ "id": 10851133, "id-copy": 10851133, "alias": "Wilbur", "name": "WilburDiegel", "user-since": datetime("2005-08-20T01:37:10.000Z"), "user-since-copy": datetime("2005-08-20T01:37:10.000Z"), "friend-ids": {{ 44811869, 15362002, 5320359, 4756538, 40097009, 905334, 44595717, 3685695, 35645656, 2090160, 35124514, 21715286, 26713020, 5816017, 15598653, 6425314, 10423130, 29593106, 14054734, 1780417, 38517315, 25570577, 5038946 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2004-05-04") } ] }
-{ "id": 10878553, "id-copy": 10878553, "alias": "Fido", "name": "FidoWillcox", "user-since": datetime("2007-01-10T01:06:54.000Z"), "user-since-copy": datetime("2007-01-10T01:06:54.000Z"), "friend-ids": {{ 28379360, 45087756, 15173549, 15693878, 23925453, 44178250, 26895550, 35260808, 9946110 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2012-02-09"), "end-date": date("2012-06-24") } ] }
-{ "id": 10896556, "id-copy": 10896556, "alias": "Kimberleigh", "name": "KimberleighWoolery", "user-since": datetime("2005-05-12T17:22:37.000Z"), "user-since-copy": datetime("2005-05-12T17:22:37.000Z"), "friend-ids": {{ 6300953, 46149018, 25478406, 577782, 38073266, 11461118, 10240145, 686269, 37990652, 26865957 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2007-05-03") } ] }
-{ "id": 10904125, "id-copy": 10904125, "alias": "Jarred", "name": "JarredRopes", "user-since": datetime("2005-11-09T09:53:06.000Z"), "user-since-copy": datetime("2005-11-09T09:53:06.000Z"), "friend-ids": {{ 26810, 23763346, 5064508, 26124598 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2007-12-28"), "end-date": date("2009-04-23") } ] }
-{ "id": 10905721, "id-copy": 10905721, "alias": "Tibby", "name": "TibbyPriebe", "user-since": datetime("2010-04-09T18:32:02.000Z"), "user-since-copy": datetime("2010-04-09T18:32:02.000Z"), "friend-ids": {{ 18406663, 1072532, 16897765 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2007-05-05"), "end-date": date("2007-03-06") } ] }
-{ "id": 10913971, "id-copy": 10913971, "alias": "Marylyn", "name": "MarylynBuehler", "user-since": datetime("2008-03-02T11:14:18.000Z"), "user-since-copy": datetime("2008-03-02T11:14:18.000Z"), "friend-ids": {{ 36555710, 21041383, 37895483, 11392039, 5195346, 12022072, 5206222, 37834919, 434970, 4441054, 39212196, 12773393 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2000-01-11") } ] }
-{ "id": 10936798, "id-copy": 10936798, "alias": "Chang", "name": "ChangBriner", "user-since": datetime("2011-01-21T02:58:13.000Z"), "user-since-copy": datetime("2011-01-21T02:58:13.000Z"), "friend-ids": {{ 44173597, 3293094, 47813131, 8981206, 36324479, 16594808, 20038389, 11223092, 7224123, 10682354, 7270314, 5170866, 10241023, 43090387, 21910381, 36504407, 18319458, 19534667, 14493618, 11394344, 5990164, 35322441 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2004-12-09"), "end-date": date("2006-08-28") } ] }
-{ "id": 10948315, "id-copy": 10948315, "alias": "Munro", "name": "MunroDiegel", "user-since": datetime("2006-11-24T10:55:36.000Z"), "user-since-copy": datetime("2006-11-24T10:55:36.000Z"), "friend-ids": {{ 46912879, 47760999, 8438850, 12005776, 7286415, 41598308, 42462653, 2040525, 8432844, 39644931 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2011-08-08"), "end-date": date("2011-09-27") } ] }
-{ "id": 10955896, "id-copy": 10955896, "alias": "Felton", "name": "FeltonRiggle", "user-since": datetime("2010-08-18T08:55:19.000Z"), "user-since-copy": datetime("2010-08-18T08:55:19.000Z"), "friend-ids": {{ 9250996, 46302470, 16921353, 21053478, 40274566, 25492381, 7743899 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-09-10"), "end-date": date("2009-01-22") } ] }
-{ "id": 10967305, "id-copy": 10967305, "alias": "Harrietta", "name": "HarriettaClewett", "user-since": datetime("2008-05-11T02:34:28.000Z"), "user-since-copy": datetime("2008-05-11T02:34:28.000Z"), "friend-ids": {{ 3346670, 25522849, 46919524, 22773543, 8985252, 43521041, 14951485, 45977993, 21285106, 17023357, 615364, 23079537, 23459313, 31663735, 24201883, 39321873, 47183802, 26870642, 34447310, 4848880, 17078809, 14119447, 39460378 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2012-05-12"), "end-date": date("2012-06-25") } ] }
-{ "id": 10975810, "id-copy": 10975810, "alias": "Davin", "name": "DavinKifer", "user-since": datetime("2005-08-19T20:23:07.000Z"), "user-since-copy": datetime("2005-08-19T20:23:07.000Z"), "friend-ids": {{ 20162027, 7842505, 3191764, 11487126, 44589086, 14959953, 18826364, 18917713, 37717273, 24319173, 1393081, 19608709, 47932966, 37681921, 47734310, 21616345, 21035793, 9650227, 43642280, 21890130, 17249802, 27944839 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2011-12-17"), "end-date": date("2011-12-01") } ] }
-{ "id": 10989949, "id-copy": 10989949, "alias": "Kaylyn", "name": "KaylynElder", "user-since": datetime("2011-01-13T12:02:13.000Z"), "user-since-copy": datetime("2011-01-13T12:02:13.000Z"), "friend-ids": {{ 22698118, 31639011, 11500577, 13007617, 26781164, 20827141, 9916306, 26415081, 14027605, 19305199, 45276489, 17632806, 42243983 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2006-01-05") } ] }
-{ "id": 10992421, "id-copy": 10992421, "alias": "Ashleigh", "name": "AshleighStroh", "user-since": datetime("2009-10-20T03:03:48.000Z"), "user-since-copy": datetime("2009-10-20T03:03:48.000Z"), "friend-ids": {{ 34581685, 36997971, 29555907, 34868441, 31092587, 9963667, 60170, 19708784, 26201942, 27806479, 40464656, 27628428, 5144660, 44794976, 9937339 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2001-11-04") } ] }
-{ "id": 11001610, "id-copy": 11001610, "alias": "Keven", "name": "KevenWildman", "user-since": datetime("2006-09-07T02:21:33.000Z"), "user-since-copy": datetime("2006-09-07T02:21:33.000Z"), "friend-ids": {{ 14316856, 4291050 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-06-20"), "end-date": date("2012-06-09") } ] }
-{ "id": 11003527, "id-copy": 11003527, "alias": "Clitus", "name": "ClitusDickinson", "user-since": datetime("2007-10-18T04:59:18.000Z"), "user-since-copy": datetime("2007-10-18T04:59:18.000Z"), "friend-ids": {{ 26264340, 47892511, 18715043, 43994375, 42874707, 44696774, 7281939 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2006-09-15") } ] }
-{ "id": 11010904, "id-copy": 11010904, "alias": "Chang", "name": "ChangSteele", "user-since": datetime("2009-02-24T01:43:56.000Z"), "user-since-copy": datetime("2009-02-24T01:43:56.000Z"), "friend-ids": {{ 19212881, 4019921, 24976558, 47613555, 26049623, 17656988, 24011085, 31763054, 21741933, 31356824, 9651386, 35034682, 5665574, 31306405, 38922156, 9837341, 31865250, 12415354 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2005-09-20"), "end-date": date("2005-05-28") } ] }
-{ "id": 11016238, "id-copy": 11016238, "alias": "Justy", "name": "JustyShaner", "user-since": datetime("2008-06-17T22:08:29.000Z"), "user-since-copy": datetime("2008-06-17T22:08:29.000Z"), "friend-ids": {{ 23689951, 17071721, 9194411, 34128749, 46316500, 31173605, 32802286, 26107462, 6561314, 9993897, 14746369, 7297148, 41466258 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2003-12-12"), "end-date": date("2007-04-12") } ] }
-{ "id": 11022826, "id-copy": 11022826, "alias": "Virgee", "name": "VirgeeHolts", "user-since": datetime("2012-01-17T22:54:54.000Z"), "user-since-copy": datetime("2012-01-17T22:54:54.000Z"), "friend-ids": {{ 40134062, 13624785, 23477090, 26708578, 18967215, 21325604, 15522457, 25873528 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2004-05-09"), "end-date": date("2010-06-15") } ] }
-{ "id": 11022889, "id-copy": 11022889, "alias": "Aubrey", "name": "AubreyMccallum", "user-since": datetime("2009-08-17T02:42:54.000Z"), "user-since-copy": datetime("2009-08-17T02:42:54.000Z"), "friend-ids": {{ 22265320, 4304911, 3403321, 20791603, 31499855, 22278594, 14580040, 31651270, 14509751, 13733306, 10947101, 7713960 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2001-01-11") } ] }
-{ "id": 11049715, "id-copy": 11049715, "alias": "Carlo", "name": "CarloBrooks", "user-since": datetime("2005-03-23T21:46:06.000Z"), "user-since-copy": datetime("2005-03-23T21:46:06.000Z"), "friend-ids": {{ 8214850, 7465603, 15385071, 32299168, 5993026, 3262895, 24995417, 25987462, 10230501, 12537459, 44597291, 33492282, 30758369, 15589085, 6799067, 23023304, 42597416, 10978280, 40668626, 25650335, 37336071 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-09-15"), "end-date": date("2011-09-03") } ] }
-{ "id": 11059435, "id-copy": 11059435, "alias": "Lucina", "name": "LucinaDurstine", "user-since": datetime("2007-04-14T19:19:23.000Z"), "user-since-copy": datetime("2007-04-14T19:19:23.000Z"), "friend-ids": {{ 18983436, 36225185, 42601602, 22134709, 20671612 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2009-06-15") } ] }
-{ "id": 11068231, "id-copy": 11068231, "alias": "Dinah", "name": "DinahSwink", "user-since": datetime("2012-05-02T04:24:33.000Z"), "user-since-copy": datetime("2012-05-02T04:24:33.000Z"), "friend-ids": {{ 31542440, 17451543, 32642661, 27867264, 32718667, 43042567, 7921827 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2003-04-10"), "end-date": date("2003-10-03") } ] }
-{ "id": 11072782, "id-copy": 11072782, "alias": "Jewel", "name": "JewelSchreckengost", "user-since": datetime("2012-06-04T18:20:29.000Z"), "user-since-copy": datetime("2012-06-04T18:20:29.000Z"), "friend-ids": {{ 47896348, 34649239, 38135221, 19731900, 14383059, 3639686, 28133949, 1326525, 415048, 34486382, 32809579, 31754806, 33563370 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-03-06") } ] }
-{ "id": 11089501, "id-copy": 11089501, "alias": "Antonette", "name": "AntonetteBrandenburg", "user-since": datetime("2010-01-02T05:42:44.000Z"), "user-since-copy": datetime("2010-01-02T05:42:44.000Z"), "friend-ids": {{ 18054329, 21707156, 1570987, 17610288, 32279976, 10880989, 37459189, 9057880, 46495123, 29331373, 20615029, 22282366, 22218648, 15950453, 30669615, 46097959, 16640911, 15896647 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2004-10-01"), "end-date": date("2009-02-20") } ] }
-{ "id": 11116465, "id-copy": 11116465, "alias": "Read", "name": "ReadOppenheimer", "user-since": datetime("2012-08-23T03:38:20.000Z"), "user-since-copy": datetime("2012-08-23T03:38:20.000Z"), "friend-ids": {{ 18679034, 12828526, 13510152, 28052139, 20367021, 30392195, 41580515, 2644015, 29573423, 22838698 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2009-11-05") } ] }
-{ "id": 11130676, "id-copy": 11130676, "alias": "Krystal", "name": "KrystalDavis", "user-since": datetime("2008-08-18T00:59:11.000Z"), "user-since-copy": datetime("2008-08-18T00:59:11.000Z"), "friend-ids": {{ 44775993, 31503397, 32012007, 16923302, 37099907, 14276165, 40040126, 38310068 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2003-11-21") } ] }
-{ "id": 11130781, "id-copy": 11130781, "alias": "Kenia", "name": "KeniaMiller", "user-since": datetime("2008-05-27T02:28:18.000Z"), "user-since-copy": datetime("2008-05-27T02:28:18.000Z"), "friend-ids": {{ 43139868, 16103105, 25352928, 23612973, 9645914, 20517323, 40438742, 47972276, 7395189, 44164898, 2805123, 33235701, 39846510, 21170026, 14223369, 14077979 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-06-24"), "end-date": date("2011-04-08") } ] }
-{ "id": 11131138, "id-copy": 11131138, "alias": "Maximillian", "name": "MaximillianSloan", "user-since": datetime("2009-12-26T13:02:42.000Z"), "user-since-copy": datetime("2009-12-26T13:02:42.000Z"), "friend-ids": {{ 4007900, 16474597, 36917058, 46709116, 35833748, 7074328, 6125321, 40646485, 23690629, 3251896, 3973740, 17863849, 9389737, 26501803, 4207105 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2010-10-16") } ] }
-{ "id": 11131756, "id-copy": 11131756, "alias": "Sharlene", "name": "SharleneFinlay", "user-since": datetime("2006-01-11T00:34:50.000Z"), "user-since-copy": datetime("2006-01-11T00:34:50.000Z"), "friend-ids": {{ 47024803, 17225785, 29871165, 14503159, 22992924, 38939801, 44563447, 101625, 40957129, 24838380, 7187619, 45283524, 31617405, 517806, 28714183, 32966332, 24006006 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2008-02-16"), "end-date": date("2011-09-12") } ] }
-{ "id": 11135899, "id-copy": 11135899, "alias": "Bailey", "name": "BaileyMoonshower", "user-since": datetime("2011-08-28T07:36:28.000Z"), "user-since-copy": datetime("2011-08-28T07:36:28.000Z"), "friend-ids": {{ 29802790, 16418079 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2010-05-17") } ] }
-{ "id": 11139106, "id-copy": 11139106, "alias": "Faith", "name": "FaithHicks", "user-since": datetime("2008-01-08T07:44:36.000Z"), "user-since-copy": datetime("2008-01-08T07:44:36.000Z"), "friend-ids": {{ 5409553, 11995627, 30724106, 17065157, 29513453, 38627025, 34382279, 36487812, 4292416, 19328709, 42169589, 18029462, 20202054, 8738011, 18339448, 2522742, 35366856, 10669527, 44287935, 47124982, 25912125, 38893810, 42212137, 22227146 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2000-11-15"), "end-date": date("2002-10-01") } ] }
-{ "id": 11145823, "id-copy": 11145823, "alias": "Rebeccah", "name": "RebeccahTodd", "user-since": datetime("2007-03-25T15:13:08.000Z"), "user-since-copy": datetime("2007-03-25T15:13:08.000Z"), "friend-ids": {{ 46132741, 11527757, 27573172, 45663865, 45572803, 30569464, 31892238 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-04-07") } ] }
-{ "id": 11147050, "id-copy": 11147050, "alias": "Karena", "name": "KarenaTanner", "user-since": datetime("2007-03-17T08:50:48.000Z"), "user-since-copy": datetime("2007-03-17T08:50:48.000Z"), "friend-ids": {{ 39952587, 2518830, 30305705, 21365609, 45914603, 2590495, 8595660 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2000-11-13"), "end-date": date("2009-01-10") } ] }
-{ "id": 11158711, "id-copy": 11158711, "alias": "Gwendolen", "name": "GwendolenBousum", "user-since": datetime("2007-07-06T10:35:24.000Z"), "user-since-copy": datetime("2007-07-06T10:35:24.000Z"), "friend-ids": {{ 22558162, 31443428, 22992355, 19452651, 23323540, 41272500, 17328954, 37489389, 35041092, 42476655 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2005-05-23") } ] }
-{ "id": 11179192, "id-copy": 11179192, "alias": "Derren", "name": "DerrenClose", "user-since": datetime("2008-04-28T09:18:19.000Z"), "user-since-copy": datetime("2008-04-28T09:18:19.000Z"), "friend-ids": {{ 43947479, 30154889, 10673575, 8056171, 28691242, 22881730, 15291446, 7331632, 32819016, 35194153 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-10-24"), "end-date": date("2006-08-12") } ] }
-{ "id": 11187373, "id-copy": 11187373, "alias": "Garfield", "name": "GarfieldWible", "user-since": datetime("2009-06-19T05:22:16.000Z"), "user-since-copy": datetime("2009-06-19T05:22:16.000Z"), "friend-ids": {{ 24453777, 20841948, 12224610, 30351943, 17826670, 36119836, 27850423, 4004658, 42610631, 25893845, 46022891, 33018964, 37844844, 1705377, 38811008, 36802000 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2000-02-22") } ] }
-{ "id": 11190361, "id-copy": 11190361, "alias": "Jancis", "name": "JancisFeufer", "user-since": datetime("2005-08-04T13:00:03.000Z"), "user-since-copy": datetime("2005-08-04T13:00:03.000Z"), "friend-ids": {{ 29421411, 15938833, 13248806, 1321174, 32401361, 34058563, 39735399, 35531531, 2631116, 1167996, 18366452, 45021961, 246133 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2003-08-27") } ] }
-{ "id": 11196118, "id-copy": 11196118, "alias": "Carson", "name": "CarsonBusk", "user-since": datetime("2006-07-23T07:08:34.000Z"), "user-since-copy": datetime("2006-07-23T07:08:34.000Z"), "friend-ids": {{ 36454884, 31755449, 44569587 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2007-08-13") } ] }
-{ "id": 11220541, "id-copy": 11220541, "alias": "Phyllida", "name": "PhyllidaRing", "user-since": datetime("2012-03-01T06:11:58.000Z"), "user-since-copy": datetime("2012-03-01T06:11:58.000Z"), "friend-ids": {{ 609357, 45820919, 17439004, 16790980, 27878958, 13930012, 20759108, 23987257, 29330180, 9298668, 10644382, 2596101, 29705735, 13371057, 41709459, 6973880, 41608321, 41344973, 9555209, 37508452, 26445359, 7693361, 12059348 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-12-05"), "end-date": date("2009-09-16") } ] }
-{ "id": 11230663, "id-copy": 11230663, "alias": "Caryl", "name": "CarylSmail", "user-since": datetime("2006-03-17T16:52:51.000Z"), "user-since-copy": datetime("2006-03-17T16:52:51.000Z"), "friend-ids": {{ 32153460, 21186863, 24199212, 25220508, 26590053, 42433121, 35372685 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-12-05") } ] }
-{ "id": 11233525, "id-copy": 11233525, "alias": "Syd", "name": "SydSauter", "user-since": datetime("2010-12-18T02:44:55.000Z"), "user-since-copy": datetime("2010-12-18T02:44:55.000Z"), "friend-ids": {{ 6312313, 17431246, 36729581, 3715101, 39534341, 10333995, 36042764, 14014852, 27375328, 17089631, 24066240, 42616402, 34049424, 29807262, 25669160, 43435752, 46702290, 27418631, 13587383, 14811241 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2010-06-21") } ] }
-{ "id": 11235340, "id-copy": 11235340, "alias": "Maurice", "name": "MauriceHayhurst", "user-since": datetime("2008-12-24T05:11:37.000Z"), "user-since-copy": datetime("2008-12-24T05:11:37.000Z"), "friend-ids": {{ 36045307, 37144109, 37142113, 38379399, 21011762, 30698208, 3185430, 24698099, 39750599, 1820110, 19740583, 5658727, 33165497, 27066109, 20299488, 26484094, 17984991, 9623240, 15287433, 32468842, 34023148, 16744372, 30389952, 40305465 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2011-05-19"), "end-date": date("2011-11-15") } ] }
-{ "id": 11244439, "id-copy": 11244439, "alias": "Francene", "name": "FranceneArmstrong", "user-since": datetime("2009-11-12T19:32:27.000Z"), "user-since-copy": datetime("2009-11-12T19:32:27.000Z"), "friend-ids": {{ 27784445, 37528954, 14014093, 18695376 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-06-26") } ] }
-{ "id": 11252185, "id-copy": 11252185, "alias": "Quintin", "name": "QuintinMcdonald", "user-since": datetime("2010-09-27T08:09:51.000Z"), "user-since-copy": datetime("2010-09-27T08:09:51.000Z"), "friend-ids": {{ 17231767, 1840658, 32389773, 31328720, 18446903, 48007173, 40417004, 41543048, 4774035, 43047815, 24232919, 936390, 20744224, 39536211, 34205950, 38429209, 399190, 38425767, 8776604, 10360244, 28414116, 15735235, 6431904 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-10-04") } ] }
-{ "id": 11262439, "id-copy": 11262439, "alias": "Alexandra", "name": "AlexandraStocker", "user-since": datetime("2010-08-28T03:48:52.000Z"), "user-since-copy": datetime("2010-08-28T03:48:52.000Z"), "friend-ids": {{ 16331707 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2010-12-04") } ] }
-{ "id": 11281576, "id-copy": 11281576, "alias": "Louisa", "name": "LouisaWheeler", "user-since": datetime("2005-01-19T05:34:26.000Z"), "user-since-copy": datetime("2005-01-19T05:34:26.000Z"), "friend-ids": {{ 29655724, 29204886, 24086191, 36260050, 502778, 368888, 42853595, 40434954, 46768026, 17096472, 33160972, 15621748, 46246949, 14174435, 99088, 44271646, 3676253, 11744063, 21957250, 34611796, 32735521, 45352911, 6097178, 3796892 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-03-05") } ] }
-{ "id": 11287666, "id-copy": 11287666, "alias": "Darian", "name": "DarianHurst", "user-since": datetime("2009-05-11T03:33:37.000Z"), "user-since-copy": datetime("2009-05-11T03:33:37.000Z"), "friend-ids": {{ 34901893, 38687373, 30369991, 44597588, 41413513, 24197212, 36791517, 19949174, 23092611, 29695794, 7024108, 25202811, 10231736, 3754404, 15863600, 30772236, 21615658 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2012-04-12"), "end-date": date("2012-05-07") } ] }
-{ "id": 11289733, "id-copy": 11289733, "alias": "Jettie", "name": "JettieElinor", "user-since": datetime("2006-03-02T09:44:17.000Z"), "user-since-copy": datetime("2006-03-02T09:44:17.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2002-07-25"), "end-date": date("2005-01-16") } ] }
-{ "id": 11302930, "id-copy": 11302930, "alias": "Eustace", "name": "EustaceKava", "user-since": datetime("2011-08-24T18:08:32.000Z"), "user-since-copy": datetime("2011-08-24T18:08:32.000Z"), "friend-ids": {{ 31173988, 7044500, 11649679, 34385410, 3097267, 24759223, 20452579, 7436501, 4500062, 765860, 14592959, 582267, 25586360, 6035361, 38333776, 47384154, 22158173 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2004-05-24") } ] }
-{ "id": 11313361, "id-copy": 11313361, "alias": "Lashawn", "name": "LashawnSchuth", "user-since": datetime("2006-08-24T02:37:43.000Z"), "user-since-copy": datetime("2006-08-24T02:37:43.000Z"), "friend-ids": {{ 3844342, 31605302, 11335667, 3890958 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2001-04-04"), "end-date": date("2006-12-03") } ] }
-{ "id": 11316178, "id-copy": 11316178, "alias": "Carlene", "name": "CarleneArchibald", "user-since": datetime("2007-09-02T16:24:57.000Z"), "user-since-copy": datetime("2007-09-02T16:24:57.000Z"), "friend-ids": {{ 45522809, 33213012, 2265630, 27087141, 7247502, 38659338, 33327692, 43927391, 41809132, 4738869, 9663680, 45809341, 38204579, 17145650, 23991333, 9915598, 28129675, 47406993, 37554697 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2007-12-15"), "end-date": date("2008-06-02") } ] }
-{ "id": 11321269, "id-copy": 11321269, "alias": "Wilford", "name": "WilfordFuhrer", "user-since": datetime("2012-01-25T14:53:32.000Z"), "user-since-copy": datetime("2012-01-25T14:53:32.000Z"), "friend-ids": {{ 6210425, 27216911, 3113058, 28094966, 119775, 805604, 43386400, 46812881, 22339620, 46498863, 26422270, 43219229, 40022359, 39446155 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2001-07-06") } ] }
-{ "id": 11357614, "id-copy": 11357614, "alias": "Denys", "name": "DenysMcintosh", "user-since": datetime("2006-01-15T22:32:48.000Z"), "user-since-copy": datetime("2006-01-15T22:32:48.000Z"), "friend-ids": {{ 10713170, 21699820, 14949046, 7935772, 21404351, 21078565, 15867691, 41676271, 2655928, 22987809, 16585582, 8318693, 46886662, 15081903, 47617713, 6317213, 32997127 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2008-08-28") } ] }
-{ "id": 11373598, "id-copy": 11373598, "alias": "Dina", "name": "DinaDriggers", "user-since": datetime("2010-01-06T22:56:18.000Z"), "user-since-copy": datetime("2010-01-06T22:56:18.000Z"), "friend-ids": {{ 8839886, 10146989, 10877857, 11710726, 5699142, 27984085, 12834284 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2012-07-25") } ] }
-{ "id": 11381089, "id-copy": 11381089, "alias": "Earlene", "name": "EarleneAmmons", "user-since": datetime("2010-03-24T05:25:35.000Z"), "user-since-copy": datetime("2010-03-24T05:25:35.000Z"), "friend-ids": {{ 25392364, 36996951, 16110083, 9799716, 22893553, 28551996, 7706432, 14225386, 15633254, 39395931, 46707062, 37226919, 8532306, 3765988, 20939685, 31136325, 45222021, 15355741, 8760941, 12045616, 6890610, 13560532, 44914868, 37744233 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2000-06-10") } ] }
-{ "id": 11427025, "id-copy": 11427025, "alias": "Kyran", "name": "KyranKlockman", "user-since": datetime("2007-11-24T11:35:40.000Z"), "user-since-copy": datetime("2007-11-24T11:35:40.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-06-10"), "end-date": date("2008-10-25") } ] }
-{ "id": 11435779, "id-copy": 11435779, "alias": "Jonty", "name": "JontyLarson", "user-since": datetime("2012-04-11T08:34:47.000Z"), "user-since-copy": datetime("2012-04-11T08:34:47.000Z"), "friend-ids": {{ 37343432, 9979565, 14647518, 32490112, 26673699, 22447290, 40923710, 47426439 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2010-08-24"), "end-date": date("2011-06-21") } ] }
-{ "id": 11454253, "id-copy": 11454253, "alias": "Fairy", "name": "FairyFoster", "user-since": datetime("2007-05-04T11:48:12.000Z"), "user-since-copy": datetime("2007-05-04T11:48:12.000Z"), "friend-ids": {{ 15077027, 13719617, 3663639, 16159577, 29937764, 11018999, 36883485, 35967804, 16558412, 19456409, 33156277, 8763694, 9279896 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2012-07-10") } ] }
-{ "id": 11455492, "id-copy": 11455492, "alias": "Cymbeline", "name": "CymbelineEliza", "user-since": datetime("2010-05-03T21:32:10.000Z"), "user-since-copy": datetime("2010-05-03T21:32:10.000Z"), "friend-ids": {{ 27738860, 21711920, 47805508, 33507501, 22648267, 1006513, 23617648, 20104970, 8132761, 14963107, 19477123 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2002-07-25") } ] }
-{ "id": 11515477, "id-copy": 11515477, "alias": "Kassandra", "name": "KassandraByers", "user-since": datetime("2005-05-24T10:27:06.000Z"), "user-since-copy": datetime("2005-05-24T10:27:06.000Z"), "friend-ids": {{ 23979652, 25789717, 7769765, 30747470, 30667193, 22447318, 42934938, 24601934, 31839813, 18960206, 30913033, 39059809, 18213877, 3731518, 10573130, 37902022 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-01-13") } ] }
-{ "id": 11515828, "id-copy": 11515828, "alias": "Christa", "name": "ChristaWain", "user-since": datetime("2007-05-01T13:32:18.000Z"), "user-since-copy": datetime("2007-05-01T13:32:18.000Z"), "friend-ids": {{ 9081871, 27897837, 47641133, 1224070, 41007475, 39553691, 10757036, 28663201, 44842180, 24894191, 42128523, 30703082, 27281648, 9786943 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2012-05-04") } ] }
-{ "id": 11529730, "id-copy": 11529730, "alias": "Linwood", "name": "LinwoodZadovsky", "user-since": datetime("2007-03-13T03:41:20.000Z"), "user-since-copy": datetime("2007-03-13T03:41:20.000Z"), "friend-ids": {{ 23516069, 24312236, 23750591, 36982495, 36483830 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2008-01-25") } ] }
-{ "id": 11534575, "id-copy": 11534575, "alias": "Sena", "name": "SenaWeidemann", "user-since": datetime("2008-05-25T01:11:53.000Z"), "user-since-copy": datetime("2008-05-25T01:11:53.000Z"), "friend-ids": {{ 8564372, 20258364, 35812476, 36877724, 30983504, 17757915, 42833517 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2003-04-22") } ] }
-{ "id": 11559613, "id-copy": 11559613, "alias": "Mick", "name": "MickWilkinson", "user-since": datetime("2005-12-23T15:11:33.000Z"), "user-since-copy": datetime("2005-12-23T15:11:33.000Z"), "friend-ids": {{ 4641355 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2000-06-03") } ] }
-{ "id": 11587057, "id-copy": 11587057, "alias": "Meagan", "name": "MeaganHays", "user-since": datetime("2012-08-15T21:45:05.000Z"), "user-since-copy": datetime("2012-08-15T21:45:05.000Z"), "friend-ids": {{ 26887765, 1940688, 10308941, 42037682, 1716669, 38995955, 17690888, 23227010, 4054166, 22275630, 6863237, 15140164, 38703696, 19044355, 43996569, 12255978, 28516070 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2003-02-26"), "end-date": date("2010-08-05") } ] }
-{ "id": 11588467, "id-copy": 11588467, "alias": "Soon", "name": "SoonHays", "user-since": datetime("2011-12-21T05:33:54.000Z"), "user-since-copy": datetime("2011-12-21T05:33:54.000Z"), "friend-ids": {{ 659930 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2009-05-20"), "end-date": date("2009-07-16") } ] }
-{ "id": 11591713, "id-copy": 11591713, "alias": "Nannie", "name": "NannieDiller", "user-since": datetime("2008-11-27T08:31:02.000Z"), "user-since-copy": datetime("2008-11-27T08:31:02.000Z"), "friend-ids": {{ 26059738, 32515289, 13702345, 16949001, 10188160, 30251286 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2000-11-27") } ] }
-{ "id": 11592799, "id-copy": 11592799, "alias": "Booker", "name": "BookerBurkett", "user-since": datetime("2008-07-19T14:13:28.000Z"), "user-since-copy": datetime("2008-07-19T14:13:28.000Z"), "friend-ids": {{ 8693431, 28970363, 8276536, 42506445, 20113337, 40761495 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2010-10-26"), "end-date": date("2010-11-15") } ] }
-{ "id": 11598403, "id-copy": 11598403, "alias": "Jo", "name": "JoCattley", "user-since": datetime("2008-01-04T03:33:03.000Z"), "user-since-copy": datetime("2008-01-04T03:33:03.000Z"), "friend-ids": {{ 28948698, 9851844, 31708351, 28418023, 33052184, 24995451, 2840550, 19426008, 3790086 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2006-09-15") } ] }
-{ "id": 11616502, "id-copy": 11616502, "alias": "Bernetta", "name": "BernettaMackendoerfer", "user-since": datetime("2005-04-22T03:41:17.000Z"), "user-since-copy": datetime("2005-04-22T03:41:17.000Z"), "friend-ids": {{ 18804036, 29570084, 43932411, 41492349, 46505981, 32524166, 5307968 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2004-08-14"), "end-date": date("2009-08-03") } ] }
-{ "id": 11616628, "id-copy": 11616628, "alias": "Jessamine", "name": "JessamineWolff", "user-since": datetime("2008-05-03T17:05:35.000Z"), "user-since-copy": datetime("2008-05-03T17:05:35.000Z"), "friend-ids": {{ 38285911, 42183685, 11422759, 25927239, 22771435, 47814309, 43146385, 39761181, 1670925, 15764683, 8068597, 3561105 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2007-07-26"), "end-date": date("2010-03-16") } ] }
-{ "id": 11625859, "id-copy": 11625859, "alias": "Zacharias", "name": "ZachariasSanner", "user-since": datetime("2007-06-12T21:21:21.000Z"), "user-since-copy": datetime("2007-06-12T21:21:21.000Z"), "friend-ids": {{ 13379571, 45822651, 39352555, 11549959, 24329960, 2142134, 15486962, 43011509, 46074449, 9322703 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2007-05-15") } ] }
-{ "id": 11626156, "id-copy": 11626156, "alias": "Laurine", "name": "LaurineBastion", "user-since": datetime("2012-05-14T21:34:43.000Z"), "user-since-copy": datetime("2012-05-14T21:34:43.000Z"), "friend-ids": {{ 13978691, 24432513, 41105156, 4981880 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-03-09") } ] }
-{ "id": 11633284, "id-copy": 11633284, "alias": "Quinn", "name": "QuinnMillhouse", "user-since": datetime("2006-08-06T07:42:49.000Z"), "user-since-copy": datetime("2006-08-06T07:42:49.000Z"), "friend-ids": {{ 15791690, 46827169, 41678324, 25101779, 24496106, 29442447, 29240215, 23819212, 11076551, 27248100, 1506119, 37415860 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2008-01-06") } ] }
-{ "id": 11633326, "id-copy": 11633326, "alias": "Jodi", "name": "JodiBrindle", "user-since": datetime("2009-01-02T19:57:58.000Z"), "user-since-copy": datetime("2009-01-02T19:57:58.000Z"), "friend-ids": {{ 5287281, 24414393, 31942570, 45025515, 35679462, 45244705, 4931287, 11590610, 39846242, 14999029, 38735562, 6275771, 33435194 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2009-11-25") } ] }
-{ "id": 11659237, "id-copy": 11659237, "alias": "Orlando", "name": "OrlandoMcloskey", "user-since": datetime("2006-09-15T00:02:58.000Z"), "user-since-copy": datetime("2006-09-15T00:02:58.000Z"), "friend-ids": {{ 18927260, 17411696, 20569511, 5242025, 18974872, 24923117, 42416784, 37339853, 42886763, 12241986, 40609114, 8814896, 30383771, 23631329, 41937811, 13354366, 40113344, 11968348, 23416173, 1546554, 46467044, 5542363, 32084191, 3049632 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2006-04-20") } ] }
-{ "id": 11681410, "id-copy": 11681410, "alias": "Wendell", "name": "WendellGarneys", "user-since": datetime("2007-07-23T13:10:29.000Z"), "user-since-copy": datetime("2007-07-23T13:10:29.000Z"), "friend-ids": {{ 11124106, 3438927, 28547601, 18074764, 35037765, 25438231, 8196141, 26000844, 6063826, 22981069, 31549929, 33158093, 40748728, 12245244, 2442169, 7879517, 877005, 24286984 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2008-02-10"), "end-date": date("2008-05-15") } ] }
-{ "id": 11721010, "id-copy": 11721010, "alias": "Eliot", "name": "EliotTennant", "user-since": datetime("2009-07-25T22:16:20.000Z"), "user-since-copy": datetime("2009-07-25T22:16:20.000Z"), "friend-ids": {{ 41972338, 13293762, 47012929, 13695904, 25235210, 39246961, 36832468, 26854695, 3046764, 17117110, 10902219, 36959080, 32665222 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-11-26") } ] }
-{ "id": 11723506, "id-copy": 11723506, "alias": "Odelia", "name": "OdeliaPaul", "user-since": datetime("2006-03-14T15:49:03.000Z"), "user-since-copy": datetime("2006-03-14T15:49:03.000Z"), "friend-ids": {{ 874326, 37021972, 27293893, 40453006, 44728117, 338941, 22832206, 11391929, 46420525 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2012-05-05") } ] }
-{ "id": 11729626, "id-copy": 11729626, "alias": "Kassandra", "name": "KassandraBaker", "user-since": datetime("2010-12-26T12:18:49.000Z"), "user-since-copy": datetime("2010-12-26T12:18:49.000Z"), "friend-ids": {{ 2336026, 15350108, 46098823, 35193308, 34644345, 45989141, 31179029, 15991657, 12863616, 18297246, 26571280, 16935684, 31339122, 10623785, 24666322, 23094237, 28117245, 40096052, 37538843, 8085609, 2437482, 8885815, 42016898, 4654048 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2007-07-10") } ] }
-{ "id": 11741821, "id-copy": 11741821, "alias": "Cal", "name": "CalHowe", "user-since": datetime("2005-12-27T20:26:31.000Z"), "user-since-copy": datetime("2005-12-27T20:26:31.000Z"), "friend-ids": {{ 45052138 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2006-12-10"), "end-date": date("2006-02-25") } ] }
-{ "id": 11762239, "id-copy": 11762239, "alias": "Guillermo", "name": "GuillermoCallison", "user-since": datetime("2009-02-12T13:46:40.000Z"), "user-since-copy": datetime("2009-02-12T13:46:40.000Z"), "friend-ids": {{ 3494924, 650832, 22099424, 11629223, 45581083, 206762, 27794516, 7639789, 31794781, 22985617, 17273963, 9120417, 9496942, 47474589, 47872578, 34639130, 37695869, 41346670, 7789418, 24870369, 31562430, 2414862, 41928569 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2008-07-26") } ] }
-{ "id": 11763463, "id-copy": 11763463, "alias": "Haven", "name": "HavenRaub", "user-since": datetime("2012-03-01T12:41:53.000Z"), "user-since-copy": datetime("2012-03-01T12:41:53.000Z"), "friend-ids": {{ 19981286 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2001-04-26") } ] }
-{ "id": 11788834, "id-copy": 11788834, "alias": "Benny", "name": "BennyAgg", "user-since": datetime("2011-12-19T14:28:16.000Z"), "user-since-copy": datetime("2011-12-19T14:28:16.000Z"), "friend-ids": {{ 6023130, 41817759, 15338300, 40598251, 38750529, 43646078, 9057658 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2006-09-16") } ] }
-{ "id": 11811196, "id-copy": 11811196, "alias": "Levi", "name": "LeviVeith", "user-since": datetime("2010-04-28T03:02:38.000Z"), "user-since-copy": datetime("2010-04-28T03:02:38.000Z"), "friend-ids": {{ 24907725, 35390929, 34837809, 5881290, 28179492, 44686412, 32544180, 20478414, 15685375, 8767940, 7295427 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2004-09-01") } ] }
-{ "id": 11840218, "id-copy": 11840218, "alias": "Deandre", "name": "DeandreMackendrick", "user-since": datetime("2012-07-03T08:22:13.000Z"), "user-since-copy": datetime("2012-07-03T08:22:13.000Z"), "friend-ids": {{ 36310775, 13455844, 1133499, 44183463, 28002311, 40758157, 33299342, 47526543, 9613784, 5698202, 1492720, 5663846 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2006-03-12"), "end-date": date("2009-08-08") } ] }
-{ "id": 11874358, "id-copy": 11874358, "alias": "Rachyl", "name": "RachylOmara", "user-since": datetime("2008-05-19T19:05:44.000Z"), "user-since-copy": datetime("2008-05-19T19:05:44.000Z"), "friend-ids": {{ 17070163, 39951748, 9940832, 6714785, 4963198, 17121038, 29997771, 21420071, 3672434, 37974288 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2001-11-11"), "end-date": date("2008-07-25") } ] }
-{ "id": 11919640, "id-copy": 11919640, "alias": "Blanch", "name": "BlanchHawkins", "user-since": datetime("2007-09-24T10:11:40.000Z"), "user-since-copy": datetime("2007-09-24T10:11:40.000Z"), "friend-ids": {{ 28731986, 7289796, 42121816, 33230171 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2007-09-17") } ] }
-{ "id": 11932807, "id-copy": 11932807, "alias": "Sheridan", "name": "SheridanCarr", "user-since": datetime("2009-05-17T01:39:53.000Z"), "user-since-copy": datetime("2009-05-17T01:39:53.000Z"), "friend-ids": {{ 12836351, 10066178, 40881248, 3744364, 18904729, 10238846, 27947251, 23407801, 39613208, 34468026, 20801656, 46114253, 26807188, 13084266, 27104805, 27016320, 25825154, 16782132, 29528918 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-09-04"), "end-date": date("2005-01-15") } ] }
-{ "id": 11954992, "id-copy": 11954992, "alias": "Caitlin", "name": "CaitlinLangston", "user-since": datetime("2007-01-02T01:50:34.000Z"), "user-since-copy": datetime("2007-01-02T01:50:34.000Z"), "friend-ids": {{ 23355687, 22474136, 28513847, 32515387, 44041844, 33706721, 10874992, 36341753, 34431157, 16146113, 15462591, 18188151, 29554174, 44940738, 25888018, 42795884, 14382632, 12734889, 11724519, 15830341, 25725320, 37580394, 24124411, 47984339 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2010-05-26"), "end-date": date("2010-03-28") } ] }
-{ "id": 11981266, "id-copy": 11981266, "alias": "Meghann", "name": "MeghannBatten", "user-since": datetime("2008-06-04T14:25:11.000Z"), "user-since-copy": datetime("2008-06-04T14:25:11.000Z"), "friend-ids": {{ 39206334, 28999157, 22813777 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2012-06-26") } ] }
-{ "id": 11988241, "id-copy": 11988241, "alias": "Cyrilla", "name": "CyrillaRohtin", "user-since": datetime("2005-02-10T08:24:14.000Z"), "user-since-copy": datetime("2005-02-10T08:24:14.000Z"), "friend-ids": {{ 32725541, 26677413, 29278988, 218049, 19833496, 20655804, 27991386, 5326490, 28583388, 41013948, 35541276, 41552165, 8526660 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2004-07-20"), "end-date": date("2004-08-19") } ] }
-{ "id": 9020338, "id-copy": 9020338, "alias": "Shenika", "name": "ShenikaColdsmith", "user-since": datetime("2011-02-22T08:03:05.000Z"), "user-since-copy": datetime("2011-02-22T08:03:05.000Z"), "friend-ids": {{ 28029790, 45719398, 12088661, 4134025, 27354070, 46504723, 23155578, 3370020, 26477155, 27314367, 7672726, 41117417 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2011-04-18") } ] }
-{ "id": 9035089, "id-copy": 9035089, "alias": "Marylyn", "name": "MarylynSteele", "user-since": datetime("2005-04-24T04:55:25.000Z"), "user-since-copy": datetime("2005-04-24T04:55:25.000Z"), "friend-ids": {{ 4250473, 16568038, 10872744, 32513859, 37267973, 2225211, 45148996, 1080441, 13013464, 10394988, 3316854, 8183563, 228753, 6849521, 8786964, 21073526 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2011-02-11"), "end-date": date("2011-10-08") } ] }
-{ "id": 9041443, "id-copy": 9041443, "alias": "Maria", "name": "MariaWard", "user-since": datetime("2006-12-25T01:24:40.000Z"), "user-since-copy": datetime("2006-12-25T01:24:40.000Z"), "friend-ids": {{ 10660010, 19103672, 11300656, 44383404, 36523093, 11434370, 34405687, 30889551, 4843181, 22025114, 26395363, 8607483, 25294309 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2005-06-13") } ] }
-{ "id": 9041689, "id-copy": 9041689, "alias": "Freeman", "name": "FreemanDriggers", "user-since": datetime("2011-05-23T03:51:13.000Z"), "user-since-copy": datetime("2011-05-23T03:51:13.000Z"), "friend-ids": {{ 29448942, 29196543, 22725448, 15145190, 11938396, 44028947, 18379392, 21813464, 7448397, 43717728, 10728731, 24177517, 29069798, 37056934, 27601399, 26867839, 16593922, 22247111 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2007-01-14") } ] }
-{ "id": 9043201, "id-copy": 9043201, "alias": "Eliseo", "name": "EliseoBagley", "user-since": datetime("2007-05-17T10:44:18.000Z"), "user-since-copy": datetime("2007-05-17T10:44:18.000Z"), "friend-ids": {{ 41250222, 28415639, 40825493, 11902499, 39161617, 16612650, 39102228, 46013732, 42664763, 20165539, 40891614, 2887877, 27999503, 5059039, 9617378, 16378780, 21987749 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2003-05-26") } ] }
-{ "id": 9045535, "id-copy": 9045535, "alias": "Ebenezer", "name": "EbenezerPery", "user-since": datetime("2008-06-05T17:48:45.000Z"), "user-since-copy": datetime("2008-06-05T17:48:45.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2012-04-07"), "end-date": date("2012-06-10") } ] }
-{ "id": 9050164, "id-copy": 9050164, "alias": "Haydee", "name": "HaydeeCook", "user-since": datetime("2005-08-28T12:13:59.000Z"), "user-since-copy": datetime("2005-08-28T12:13:59.000Z"), "friend-ids": {{ 26484166, 27686644, 42277018, 5893537, 34617524, 12158738, 41566344, 30653024, 23636324, 24072660, 1784294, 38620941, 40846838, 30303402, 27004887, 35907658, 42893556, 10118575, 47861482 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2007-11-23") } ] }
-{ "id": 9081124, "id-copy": 9081124, "alias": "Aureole", "name": "AureoleChappel", "user-since": datetime("2005-03-24T18:14:35.000Z"), "user-since-copy": datetime("2005-03-24T18:14:35.000Z"), "friend-ids": {{ 16199402, 2970920 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2005-07-19"), "end-date": date("2011-04-02") } ] }
-{ "id": 9107137, "id-copy": 9107137, "alias": "Woodrow", "name": "WoodrowMueller", "user-since": datetime("2012-06-15T04:53:52.000Z"), "user-since-copy": datetime("2012-06-15T04:53:52.000Z"), "friend-ids": {{ 39459662, 1343459, 16606290, 21443457, 29053037, 28244658, 27954195, 9411908, 2059678, 24579828, 40955404 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2002-02-11") } ] }
-{ "id": 9125827, "id-copy": 9125827, "alias": "Kary", "name": "KaryHildyard", "user-since": datetime("2006-03-17T23:21:33.000Z"), "user-since-copy": datetime("2006-03-17T23:21:33.000Z"), "friend-ids": {{ 5570026 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2007-09-27") } ] }
-{ "id": 9136882, "id-copy": 9136882, "alias": "Cassie", "name": "CassieGarratt", "user-since": datetime("2005-08-07T05:09:11.000Z"), "user-since-copy": datetime("2005-08-07T05:09:11.000Z"), "friend-ids": {{ 40916371, 42882703, 37748113, 45347468, 37653228, 15540626, 29276950, 31566687, 14600173, 12909057, 39561446, 41035377, 45987458, 43649639, 24488758, 25625568, 15566464, 584815, 35900688, 1079087, 46148561, 46404398 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2010-11-13"), "end-date": date("2010-09-04") } ] }
-{ "id": 9146107, "id-copy": 9146107, "alias": "Femie", "name": "FemieBurns", "user-since": datetime("2007-05-05T03:23:12.000Z"), "user-since-copy": datetime("2007-05-05T03:23:12.000Z"), "friend-ids": {{ 38688633, 2489245, 43502175, 34373436, 11854240, 23544813, 44263720, 20953878, 37021620, 16028559, 20673451, 46975172, 47409532, 44524395 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-07-26") } ] }
-{ "id": 9155080, "id-copy": 9155080, "alias": "Errol", "name": "ErrolLittle", "user-since": datetime("2011-12-20T07:09:25.000Z"), "user-since-copy": datetime("2011-12-20T07:09:25.000Z"), "friend-ids": {{ 17400275, 40794627, 12632163, 45365986, 7980045, 7368579, 40357205, 29279590, 258707, 38447445, 27048261, 19911849, 10768265, 24278809, 11940146, 33555290, 23286799, 40641141, 33877442 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-03-05") } ] }
-{ "id": 9168649, "id-copy": 9168649, "alias": "Harmony", "name": "HarmonyMackendoerfer", "user-since": datetime("2006-06-25T21:01:50.000Z"), "user-since-copy": datetime("2006-06-25T21:01:50.000Z"), "friend-ids": {{ 197057, 11973988, 2042364, 21282964, 25761405, 10180346, 39780287, 39243722, 2984620, 7756400, 21311572, 21013939, 16998045, 39135533, 47720897, 20316953 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2005-12-17"), "end-date": date("2009-07-11") } ] }
-{ "id": 9170767, "id-copy": 9170767, "alias": "Noble", "name": "NobleByers", "user-since": datetime("2012-04-19T03:21:33.000Z"), "user-since-copy": datetime("2012-04-19T03:21:33.000Z"), "friend-ids": {{ 17464807, 11911237, 31984348, 14323306, 21828766, 24212960, 3269277, 24648466, 30032203, 15837021, 12033801, 3899014, 6105665, 4416812, 33902540, 9640452, 3739829, 14414940, 36838129, 7327467, 35420130, 24031049 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2007-08-11") } ] }
-{ "id": 9179413, "id-copy": 9179413, "alias": "Benton", "name": "BentonMorland", "user-since": datetime("2006-02-08T13:43:03.000Z"), "user-since-copy": datetime("2006-02-08T13:43:03.000Z"), "friend-ids": {{ 25229017, 22411534, 46862190, 17238544, 10875646, 19572187, 9889710, 23940269, 24489112, 7997331, 8866147, 29705622, 35336434, 14756488, 40059408, 32606759, 37546068, 24168033, 20761302, 45465986, 27519909, 23920570, 3984052, 38799668 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-12-05") } ] }
-{ "id": 9205615, "id-copy": 9205615, "alias": "Eddie", "name": "EddieRosensteel", "user-since": datetime("2007-01-03T07:17:37.000Z"), "user-since-copy": datetime("2007-01-03T07:17:37.000Z"), "friend-ids": {{ 4208455, 19941893 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2005-11-15") } ] }
-{ "id": 9209866, "id-copy": 9209866, "alias": "Timothy", "name": "TimothyBuck", "user-since": datetime("2009-11-07T14:19:12.000Z"), "user-since-copy": datetime("2009-11-07T14:19:12.000Z"), "friend-ids": {{ 43082021, 25019103, 26061770, 7134151, 17663441, 35230064, 731481, 6719229, 23303796, 40777269 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2000-04-03"), "end-date": date("2000-04-20") } ] }
-{ "id": 9221836, "id-copy": 9221836, "alias": "Claud", "name": "ClaudPratt", "user-since": datetime("2008-01-01T04:10:02.000Z"), "user-since-copy": datetime("2008-01-01T04:10:02.000Z"), "friend-ids": {{ 35586361, 40548794, 7169299, 24675214, 21079165, 37323851, 16881366, 24433012, 38047831, 34495409, 33711705, 8957126, 38345318 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2002-12-05") } ] }
-{ "id": 9259234, "id-copy": 9259234, "alias": "Abigail", "name": "AbigailNicola", "user-since": datetime("2009-08-11T09:18:47.000Z"), "user-since-copy": datetime("2009-08-11T09:18:47.000Z"), "friend-ids": {{ 5465164, 47505082 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2006-02-22"), "end-date": date("2007-10-02") } ] }
-{ "id": 9267007, "id-copy": 9267007, "alias": "Perla", "name": "PerlaCox", "user-since": datetime("2009-04-14T20:56:37.000Z"), "user-since-copy": datetime("2009-04-14T20:56:37.000Z"), "friend-ids": {{ 8937408, 4640163, 41404266, 15668694, 21004833, 12635405, 40379208, 18641131, 14014264, 39008348, 36559306, 26261953, 3593955, 13559713, 34525259 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2003-07-02") } ] }
-{ "id": 9267397, "id-copy": 9267397, "alias": "Corbin", "name": "CorbinWhite", "user-since": datetime("2006-01-07T07:43:27.000Z"), "user-since-copy": datetime("2006-01-07T07:43:27.000Z"), "friend-ids": {{ 11772390, 16826538, 16103166, 3256508, 40044263, 44187580, 29521314, 46200384, 40192445, 1239869, 14257012, 21632509, 6292478, 38738535, 18136574, 8369661, 45672754 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2000-09-16"), "end-date": date("2003-07-12") } ] }
-{ "id": 9274378, "id-copy": 9274378, "alias": "Callista", "name": "CallistaCatleay", "user-since": datetime("2012-01-11T05:02:51.000Z"), "user-since-copy": datetime("2012-01-11T05:02:51.000Z"), "friend-ids": {{ 35709258, 45469345, 7683235, 10959232, 44123341, 35853639, 11693773, 39944820, 47667622, 42781782, 4756825, 23566535 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2002-04-15"), "end-date": date("2003-04-03") } ] }
-{ "id": 9286279, "id-copy": 9286279, "alias": "Barnaby", "name": "BarnabyAckerley", "user-since": datetime("2006-09-15T01:56:34.000Z"), "user-since-copy": datetime("2006-09-15T01:56:34.000Z"), "friend-ids": {{ 21236050, 22647474, 18898492, 22530993, 4332450, 38947319, 25882415, 47187086, 5810354, 18396369, 44918707, 9732196, 14821426, 148735 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2011-02-10") } ] }
-{ "id": 9291964, "id-copy": 9291964, "alias": "Ned", "name": "NedPullman", "user-since": datetime("2011-02-02T07:25:43.000Z"), "user-since-copy": datetime("2011-02-02T07:25:43.000Z"), "friend-ids": {{ 3168566, 3349059, 43400084, 26187570, 11222713, 9924690, 7250860, 9801843, 18856900, 3558502, 17237369, 20047877, 28454433, 12279948, 19319514, 36151797 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-08-11") } ] }
-{ "id": 9295696, "id-copy": 9295696, "alias": "Margaux", "name": "MargauxPerkins", "user-since": datetime("2012-05-23T04:28:13.000Z"), "user-since-copy": datetime("2012-05-23T04:28:13.000Z"), "friend-ids": {{ 23713491, 4271158, 27340057, 7815427, 14232017, 22868851, 2293397, 24147381, 11816307, 16597552, 47120663, 40746124, 9777479, 18134957, 39193317, 19755909, 42252346 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2006-02-17"), "end-date": date("2007-05-06") } ] }
-{ "id": 9313492, "id-copy": 9313492, "alias": "Tera", "name": "TeraWolfe", "user-since": datetime("2010-12-20T12:47:25.000Z"), "user-since-copy": datetime("2010-12-20T12:47:25.000Z"), "friend-ids": {{ 45424983, 18345704, 14849759, 31638064, 38670515, 48015953, 36114769 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2001-04-26"), "end-date": date("2004-12-06") } ] }
-{ "id": 9329272, "id-copy": 9329272, "alias": "Nonie", "name": "NonieStafford", "user-since": datetime("2005-10-01T21:12:24.000Z"), "user-since-copy": datetime("2005-10-01T21:12:24.000Z"), "friend-ids": {{ 42745071, 14744035, 37742648, 31179205, 28520118, 32828516, 2726599, 1667680 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2004-06-21") } ] }
-{ "id": 9341965, "id-copy": 9341965, "alias": "Stephania", "name": "StephaniaBriner", "user-since": datetime("2007-06-15T18:17:32.000Z"), "user-since-copy": datetime("2007-06-15T18:17:32.000Z"), "friend-ids": {{ 9361850, 12128362, 42864061, 6323327, 34867192, 32746507, 17493376, 17276666, 33869929, 20708786 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2004-03-23"), "end-date": date("2009-01-07") } ] }
-{ "id": 9345424, "id-copy": 9345424, "alias": "Jasmin", "name": "JasminGaskins", "user-since": datetime("2012-06-15T19:40:07.000Z"), "user-since-copy": datetime("2012-06-15T19:40:07.000Z"), "friend-ids": {{ 20837477, 42339634, 41136248, 24571549, 41060055, 18621328, 2057295, 41313707 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2012-05-27"), "end-date": date("2012-07-28") } ] }
-{ "id": 9361930, "id-copy": 9361930, "alias": "Leonard", "name": "LeonardAshbaugh", "user-since": datetime("2008-06-13T07:49:33.000Z"), "user-since-copy": datetime("2008-06-13T07:49:33.000Z"), "friend-ids": {{ 33929562, 22722370, 18562061, 44346144, 38834006, 1660309, 17690686, 8299074, 13219630, 42802095, 2203402, 47180979, 43715995, 24339545, 42132653, 32010945, 18200992, 5115504 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2012-04-15") } ] }
-{ "id": 9366253, "id-copy": 9366253, "alias": "Emma", "name": "EmmaKnisely", "user-since": datetime("2012-07-08T20:39:00.000Z"), "user-since-copy": datetime("2012-07-08T20:39:00.000Z"), "friend-ids": {{ 40874500, 35049897, 29559982, 42737582, 11405173, 38919458, 26268603, 38582942, 13758558, 16949073 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2009-11-18") } ] }
-{ "id": 9372871, "id-copy": 9372871, "alias": "Emerson", "name": "EmersonSell", "user-since": datetime("2010-01-25T11:12:56.000Z"), "user-since-copy": datetime("2010-01-25T11:12:56.000Z"), "friend-ids": {{ 13800934, 24493814 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2004-02-14"), "end-date": date("2005-11-07") } ] }
-{ "id": 9373819, "id-copy": 9373819, "alias": "Man", "name": "ManHarding", "user-since": datetime("2005-03-19T02:36:47.000Z"), "user-since-copy": datetime("2005-03-19T02:36:47.000Z"), "friend-ids": {{ 10687886, 6212430, 40098775, 8554409, 18917793, 9329327, 38361031, 27404932, 29083756, 28482636, 38832020, 7859160, 14175144, 3316105, 16742847, 8143105, 13049385, 22288103, 36693926, 26571195, 6536981, 32281681, 41798492, 36467563 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2009-02-08") } ] }
-{ "id": 9386794, "id-copy": 9386794, "alias": "Issac", "name": "IssacNickolson", "user-since": datetime("2009-12-11T08:40:10.000Z"), "user-since-copy": datetime("2009-12-11T08:40:10.000Z"), "friend-ids": {{ 4077760, 26197904, 22088648 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2002-05-17") } ] }
-{ "id": 9395638, "id-copy": 9395638, "alias": "Toby", "name": "TobyThomlinson", "user-since": datetime("2012-02-02T02:11:31.000Z"), "user-since-copy": datetime("2012-02-02T02:11:31.000Z"), "friend-ids": {{ 39086825, 14218540, 37526829, 46631432, 24407673, 19484977, 3657630 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2012-02-26") } ] }
-{ "id": 9403096, "id-copy": 9403096, "alias": "Clarita", "name": "ClaritaRitter", "user-since": datetime("2007-11-18T14:11:04.000Z"), "user-since-copy": datetime("2007-11-18T14:11:04.000Z"), "friend-ids": {{ 11967380, 17558867 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2011-01-28"), "end-date": date("2011-05-05") } ] }
-{ "id": 9408688, "id-copy": 9408688, "alias": "Goddard", "name": "GoddardWeisgarber", "user-since": datetime("2011-05-21T13:18:54.000Z"), "user-since-copy": datetime("2011-05-21T13:18:54.000Z"), "friend-ids": {{ 2820008, 31637633, 35026624, 544628, 2552858 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2007-11-26") } ] }
-{ "id": 9415921, "id-copy": 9415921, "alias": "Shad", "name": "ShadHaynes", "user-since": datetime("2010-01-19T22:19:28.000Z"), "user-since-copy": datetime("2010-01-19T22:19:28.000Z"), "friend-ids": {{ 4608515, 39839555, 31370710, 43278478, 731705, 26523982, 15560444, 10605444, 20229128, 41477079, 47960417, 1744587, 35477897, 10362849, 38394199, 24090076, 14390416 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2010-06-23") } ] }
-{ "id": 9420304, "id-copy": 9420304, "alias": "Alwyn", "name": "AlwynAkers", "user-since": datetime("2009-11-08T08:30:46.000Z"), "user-since-copy": datetime("2009-11-08T08:30:46.000Z"), "friend-ids": {{ 40384671, 13399303, 2163402 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-06-14"), "end-date": date("2012-07-17") } ] }
-{ "id": 9426244, "id-copy": 9426244, "alias": "Lamar", "name": "LamarMaugham", "user-since": datetime("2005-03-08T17:00:15.000Z"), "user-since-copy": datetime("2005-03-08T17:00:15.000Z"), "friend-ids": {{ 36168436, 20740167, 21922111, 32892152, 34608833, 28621520, 40818313, 23842558, 41275216, 36331147, 40737858, 45983619, 14033949, 23132425, 33634408 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2011-12-24") } ] }
-{ "id": 9434542, "id-copy": 9434542, "alias": "Alice", "name": "AliceRopes", "user-since": datetime("2011-09-10T10:32:17.000Z"), "user-since-copy": datetime("2011-09-10T10:32:17.000Z"), "friend-ids": {{ 30233815, 23593045, 243865, 46494768, 15852416, 2627657, 12253908, 11415849, 36381160, 25773586, 9952015, 20363967, 45499740, 15573031, 2939342, 24137982, 34026341, 34111551, 30963526, 7116453 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2008-07-09") } ] }
-{ "id": 9440818, "id-copy": 9440818, "alias": "Poppy", "name": "PoppyBoyer", "user-since": datetime("2007-06-09T08:15:05.000Z"), "user-since-copy": datetime("2007-06-09T08:15:05.000Z"), "friend-ids": {{ 10721272, 26882431, 45774996, 44725231, 34694934, 28877797, 12922671, 16078039, 43902220, 27311426, 34146150, 39285332, 7343219, 17482231, 15496713, 12439079, 18097780, 30046636, 16951144, 27968612 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2006-11-17") } ] }
-{ "id": 9446506, "id-copy": 9446506, "alias": "Deshawn", "name": "DeshawnBashline", "user-since": datetime("2009-03-11T18:09:19.000Z"), "user-since-copy": datetime("2009-03-11T18:09:19.000Z"), "friend-ids": {{ 22236205, 44669386, 5098679, 17631352, 40353783, 17155709 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-11-12"), "end-date": date("2003-04-22") } ] }
-{ "id": 9477919, "id-copy": 9477919, "alias": "Lilly", "name": "LillyLinton", "user-since": datetime("2005-01-09T12:24:01.000Z"), "user-since-copy": datetime("2005-01-09T12:24:01.000Z"), "friend-ids": {{ 19117935, 45208482, 36019625, 39146688, 15911832 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2011-03-03"), "end-date": date("2011-10-03") } ] }
-{ "id": 9480964, "id-copy": 9480964, "alias": "Ava", "name": "AvaCross", "user-since": datetime("2005-11-03T14:59:13.000Z"), "user-since-copy": datetime("2005-11-03T14:59:13.000Z"), "friend-ids": {{ 9693959, 3138151, 20631444, 8672727, 33701530, 14630539, 38539482, 3066915, 30934733, 38630163, 25673376 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2000-06-05"), "end-date": date("2000-10-06") } ] }
-{ "id": 9505936, "id-copy": 9505936, "alias": "Kerrie", "name": "KerrieGadow", "user-since": datetime("2005-06-26T08:47:14.000Z"), "user-since-copy": datetime("2005-06-26T08:47:14.000Z"), "friend-ids": {{ 46457424, 17421010, 11336465, 19785227 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-12-08"), "end-date": date("2010-04-11") } ] }
-{ "id": 9512989, "id-copy": 9512989, "alias": "Lilliana", "name": "LillianaAdams", "user-since": datetime("2007-06-01T16:54:29.000Z"), "user-since-copy": datetime("2007-06-01T16:54:29.000Z"), "friend-ids": {{ 14085316, 47471900, 24950195, 44416851, 6677091, 34188319, 1783776, 35860593, 29193624, 11999697, 13365419, 39452732, 14401842, 9087264, 15679216, 39424118, 45063958, 11967959, 29634503, 15763396 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2001-02-08"), "end-date": date("2008-03-23") } ] }
-{ "id": 9521401, "id-copy": 9521401, "alias": "Donnette", "name": "DonnetteFaust", "user-since": datetime("2012-03-22T09:38:14.000Z"), "user-since-copy": datetime("2012-03-22T09:38:14.000Z"), "friend-ids": {{ 25050925 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2008-08-20"), "end-date": date("2009-07-09") } ] }
-{ "id": 9545461, "id-copy": 9545461, "alias": "Sandra", "name": "SandraFea", "user-since": datetime("2005-12-09T14:40:28.000Z"), "user-since-copy": datetime("2005-12-09T14:40:28.000Z"), "friend-ids": {{ 28976045 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2012-02-02") } ] }
-{ "id": 9555157, "id-copy": 9555157, "alias": "Alea", "name": "AleaWallick", "user-since": datetime("2009-11-12T19:32:16.000Z"), "user-since-copy": datetime("2009-11-12T19:32:16.000Z"), "friend-ids": {{ 9936033, 18972695, 22198051, 44425768, 37636218, 25373418, 17204473, 6543589, 23627204, 40204583, 18664982, 27647616, 43332268, 41812682 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2009-02-17") } ] }
-{ "id": 9563056, "id-copy": 9563056, "alias": "Iantha", "name": "IanthaHoward", "user-since": datetime("2009-03-09T10:16:12.000Z"), "user-since-copy": datetime("2009-03-09T10:16:12.000Z"), "friend-ids": {{ 31445918, 39207727, 45365035, 7861010, 28533268, 29009652, 40156013, 40416479, 42741676, 30221879, 30189614, 46450645, 30914117, 33681301, 19457868, 23309378, 15126664, 32913981, 5396205 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2000-03-18"), "end-date": date("2009-01-05") } ] }
-{ "id": 9574393, "id-copy": 9574393, "alias": "Ghislaine", "name": "GhislaineTaylor", "user-since": datetime("2005-01-23T07:49:26.000Z"), "user-since-copy": datetime("2005-01-23T07:49:26.000Z"), "friend-ids": {{ 23799181, 25411427, 3758740, 47542325, 41070945, 45261892, 23309481 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2003-04-15") } ] }
-{ "id": 9588427, "id-copy": 9588427, "alias": "Tiffany", "name": "TiffanyGeyer", "user-since": datetime("2007-09-10T11:20:53.000Z"), "user-since-copy": datetime("2007-09-10T11:20:53.000Z"), "friend-ids": {{ 31357437, 16305152, 39281885, 25249419, 434661, 13634747, 39812462, 25218908, 22362649, 41696008, 4523776, 40340358, 45330588, 299997, 11538141, 20972409, 25152923, 8627592, 33381524, 6226232 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2005-02-20") } ] }
-{ "id": 9594523, "id-copy": 9594523, "alias": "Tam", "name": "TamWillcox", "user-since": datetime("2011-12-23T11:41:58.000Z"), "user-since-copy": datetime("2011-12-23T11:41:58.000Z"), "friend-ids": {{ 27383896, 20745988, 10063024, 8241427, 40299998, 32408463, 25171835, 22380586, 15344194, 25951348, 28733234, 45421004, 2273747, 2229862, 6241144, 6704115, 8659430, 47431991, 47929530, 24393021 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2001-07-27") } ] }
-{ "id": 9629923, "id-copy": 9629923, "alias": "Adria", "name": "AdriaBoyer", "user-since": datetime("2005-08-12T16:31:38.000Z"), "user-since-copy": datetime("2005-08-12T16:31:38.000Z"), "friend-ids": {{ 43812176, 1271309, 1412045, 18793840, 40264072, 41525831, 25536841, 46110606, 40440782, 37228709, 37745315, 19025404, 13458371, 32475836, 30506186, 6860193, 44650222, 5924034 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2012-03-08") } ] }
-{ "id": 9664990, "id-copy": 9664990, "alias": "Travis", "name": "TravisJube", "user-since": datetime("2010-02-12T13:42:04.000Z"), "user-since-copy": datetime("2010-02-12T13:42:04.000Z"), "friend-ids": {{ 22627931, 5992593, 8208547, 37326819, 14939087, 18366709, 29043862, 45062025, 21360937, 19730114, 26779317, 46856921, 28406774, 40580511, 8062361, 2179206, 47765870, 14039643, 28857662, 42600706 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2007-01-14") } ] }
-{ "id": 9676201, "id-copy": 9676201, "alias": "Jessica", "name": "JessicaBeals", "user-since": datetime("2006-12-02T17:13:07.000Z"), "user-since-copy": datetime("2006-12-02T17:13:07.000Z"), "friend-ids": {{ 40180348, 5499689, 43937013, 12294744, 47607871, 15173594, 19403387, 30591667, 1488569, 11862843, 26230465, 15334606, 4397778, 8140277, 39859715, 25854759, 7216524, 41695061, 43036500, 15618315, 4503056, 23790965, 14510949, 34347866 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2011-05-15"), "end-date": date("2011-10-27") } ] }
-{ "id": 9682723, "id-copy": 9682723, "alias": "Rick", "name": "RickEisaman", "user-since": datetime("2011-01-04T04:42:13.000Z"), "user-since-copy": datetime("2011-01-04T04:42:13.000Z"), "friend-ids": {{ 843458, 40779817, 24515616, 9016765, 37332064, 2164822, 45832315, 27168757, 43771964, 46638388, 43667809 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2004-08-13"), "end-date": date("2011-04-11") } ] }
-{ "id": 9740476, "id-copy": 9740476, "alias": "Tucker", "name": "TuckerRogers", "user-since": datetime("2005-05-22T22:00:09.000Z"), "user-since-copy": datetime("2005-05-22T22:00:09.000Z"), "friend-ids": {{ 13095635, 36113924, 11767777, 15169454, 1692699, 19622409, 17110214 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2009-03-24"), "end-date": date("2011-02-13") } ] }
-{ "id": 9746482, "id-copy": 9746482, "alias": "Ava", "name": "AvaEndsley", "user-since": datetime("2005-07-05T11:34:59.000Z"), "user-since-copy": datetime("2005-07-05T11:34:59.000Z"), "friend-ids": {{ 38589612, 37168849, 27697487, 47869699, 7140447, 1195276, 25105593, 46071, 5222989, 39550451, 45838187, 8513498, 44093597, 25194162, 11534580, 37101502, 6417166, 23315276, 9854625 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2011-06-15") } ] }
-{ "id": 9767755, "id-copy": 9767755, "alias": "Joel", "name": "JoelHoopengarner", "user-since": datetime("2012-01-19T13:22:46.000Z"), "user-since-copy": datetime("2012-01-19T13:22:46.000Z"), "friend-ids": {{ 41934568, 20874721, 33807743 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2012-01-21"), "end-date": date("2012-06-09") } ] }
-{ "id": 9783310, "id-copy": 9783310, "alias": "Basil", "name": "BasilLangston", "user-since": datetime("2005-06-10T11:35:51.000Z"), "user-since-copy": datetime("2005-06-10T11:35:51.000Z"), "friend-ids": {{ 21087606, 17287729, 8132136, 17055542, 5795845, 41180261, 10977404, 29700430, 47047119, 358942, 29290990, 19557422, 35447157, 33135473, 36720866, 39510564 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2000-05-11"), "end-date": date("2000-03-09") } ] }
-{ "id": 9795463, "id-copy": 9795463, "alias": "Brunilda", "name": "BrunildaPheleps", "user-since": datetime("2007-04-21T01:56:02.000Z"), "user-since-copy": datetime("2007-04-21T01:56:02.000Z"), "friend-ids": {{ 39507879, 43296507, 45019669, 39481546, 16657717, 8707249, 47148318, 46560087, 42473978, 11974026, 40145543, 2127794, 19537942, 28159963, 21439105, 32578039, 24112998, 47853039, 6406099, 30697429 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2001-07-13") } ] }
-{ "id": 9805759, "id-copy": 9805759, "alias": "Emmie", "name": "EmmieJohns", "user-since": datetime("2008-11-01T15:15:13.000Z"), "user-since-copy": datetime("2008-11-01T15:15:13.000Z"), "friend-ids": {{ 47090234, 24484835, 11048702 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2008-02-26") } ] }
-{ "id": 9811513, "id-copy": 9811513, "alias": "Casie", "name": "CasieRose", "user-since": datetime("2011-11-25T11:32:36.000Z"), "user-since-copy": datetime("2011-11-25T11:32:36.000Z"), "friend-ids": {{ 8913855, 26924028, 19426899, 38037518, 39689117, 32691982, 6561788, 36463261, 31724455, 18356325, 23130893, 35227626, 13738524, 4700460, 6963740, 13255939, 12215189, 33593825, 34229322 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2003-11-22") } ] }
-{ "id": 9818617, "id-copy": 9818617, "alias": "Elwyn", "name": "ElwynEndsley", "user-since": datetime("2012-04-12T18:14:54.000Z"), "user-since-copy": datetime("2012-04-12T18:14:54.000Z"), "friend-ids": {{ 44007613, 15744997, 9366576, 44776374, 19082361, 9967101, 25247773, 20407697 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-09-09") } ] }
-{ "id": 9866572, "id-copy": 9866572, "alias": "Evelina", "name": "EvelinaBerry", "user-since": datetime("2006-12-16T03:56:00.000Z"), "user-since-copy": datetime("2006-12-16T03:56:00.000Z"), "friend-ids": {{ 13883615, 43198063, 30615747, 3228427, 23840450, 43443245, 17107485, 34691909, 44890462, 47992198, 46475465, 28790498, 7693182, 41338502, 6694688, 17592193, 9966336, 40899188, 16363000, 43996364 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2004-03-01"), "end-date": date("2008-08-21") } ] }
-{ "id": 9867190, "id-copy": 9867190, "alias": "Elvis", "name": "ElvisBasinger", "user-since": datetime("2009-01-16T11:48:43.000Z"), "user-since-copy": datetime("2009-01-16T11:48:43.000Z"), "friend-ids": {{ 31562017, 45465097, 29858836, 21720764, 37465930, 20639296, 7168709 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2011-01-11"), "end-date": date("2011-01-26") } ] }
-{ "id": 9879709, "id-copy": 9879709, "alias": "Winfred", "name": "WinfredCraig", "user-since": datetime("2005-08-03T19:34:00.000Z"), "user-since-copy": datetime("2005-08-03T19:34:00.000Z"), "friend-ids": {{ 22314477, 25116324, 22136373, 35942614, 21324680, 17967388, 29463891, 36125380, 20673052, 27353154, 25107580, 24689990, 17672337, 16922511, 26158336, 35966438, 26619840, 29808016, 12075922, 33292381, 17902188 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2010-02-04") } ] }
-{ "id": 9880696, "id-copy": 9880696, "alias": "Cynthia", "name": "CynthiaSeidner", "user-since": datetime("2006-03-17T01:36:33.000Z"), "user-since-copy": datetime("2006-03-17T01:36:33.000Z"), "friend-ids": {{ 47318799, 28282167 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2008-07-02"), "end-date": date("2010-11-25") } ] }
-{ "id": 9896473, "id-copy": 9896473, "alias": "Harlan", "name": "HarlanAnderson", "user-since": datetime("2012-06-03T22:40:33.000Z"), "user-since-copy": datetime("2012-06-03T22:40:33.000Z"), "friend-ids": {{ 28073049, 32365932, 23795268, 7563960, 47274822, 4907078, 8659018, 33480175, 3984139, 20631025, 26879093, 27168884, 20063035, 22192716, 18259756, 28904415, 28492528, 4140983, 12014021, 10959797, 38881978, 45835171, 6556552, 26372018 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2000-08-18") } ] }
-{ "id": 9950824, "id-copy": 9950824, "alias": "Maryann", "name": "MaryannCressman", "user-since": datetime("2011-02-25T17:51:21.000Z"), "user-since-copy": datetime("2011-02-25T17:51:21.000Z"), "friend-ids": {{ 30203965, 23348792, 19093409, 21079475 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2006-10-11"), "end-date": date("2006-10-09") } ] }
-{ "id": 9978190, "id-copy": 9978190, "alias": "Tatianna", "name": "TatiannaSchmidt", "user-since": datetime("2012-07-05T14:37:56.000Z"), "user-since-copy": datetime("2012-07-05T14:37:56.000Z"), "friend-ids": {{ 15128198 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2008-11-17") } ] }
-{ "id": 9997456, "id-copy": 9997456, "alias": "Micah", "name": "MicahRogers", "user-since": datetime("2008-03-01T05:53:42.000Z"), "user-since-copy": datetime("2008-03-01T05:53:42.000Z"), "friend-ids": {{ 17761154, 33509079, 36866187, 24618619, 7048673, 18747407, 31947241, 33710255, 40699565, 22334622, 24425777, 19450074, 39309621, 4464803, 15881946, 35888289, 10539684, 17175942, 20754578, 27045156, 14301629, 19478576 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2011-02-16") } ] }
-{ "id": 10001047, "id-copy": 10001047, "alias": "Rodger", "name": "RodgerRifler", "user-since": datetime("2009-12-08T18:34:21.000Z"), "user-since-copy": datetime("2009-12-08T18:34:21.000Z"), "friend-ids": {{ 41832587, 41015556, 17486735, 38428485, 29774516, 38574837, 2061546, 46972940, 25654449, 776023, 1164809, 34242171, 9752352, 1088591, 26406961, 7270316, 36371574, 24413303, 36287374, 43343719, 6830709, 2919772, 41313339 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2003-02-08") } ] }
-{ "id": 10059343, "id-copy": 10059343, "alias": "Randy", "name": "RandyQueer", "user-since": datetime("2005-06-01T02:30:35.000Z"), "user-since-copy": datetime("2005-06-01T02:30:35.000Z"), "friend-ids": {{ 8688755, 7077909, 41009273, 26932559, 29488059, 6408736, 6374592, 5042147, 21880854, 12704496, 28046022, 2384964, 20867794, 3990470, 7132171 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2006-07-07"), "end-date": date("2007-04-08") } ] }
-{ "id": 10065595, "id-copy": 10065595, "alias": "Zenobia", "name": "ZenobiaHiggens", "user-since": datetime("2009-11-06T11:19:47.000Z"), "user-since-copy": datetime("2009-11-06T11:19:47.000Z"), "friend-ids": {{ 19623415, 12770212, 30381171, 20436392, 33497094, 39556081, 22592010, 44832685, 35801007, 39682093, 26870566, 8667589, 43790411, 24760722, 8286108, 20709133 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2001-07-28"), "end-date": date("2004-12-26") } ] }
-{ "id": 10071475, "id-copy": 10071475, "alias": "Kyra", "name": "KyraWile", "user-since": datetime("2010-08-21T20:27:23.000Z"), "user-since-copy": datetime("2010-08-21T20:27:23.000Z"), "friend-ids": {{ 24326501, 3159228, 33973593, 47221189, 17474184, 17812891 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2010-04-14") } ] }
-{ "id": 10073632, "id-copy": 10073632, "alias": "Hadley", "name": "HadleyPainter", "user-since": datetime("2010-08-18T16:57:45.000Z"), "user-since-copy": datetime("2010-08-18T16:57:45.000Z"), "friend-ids": {{ 35310707, 40074121, 28614727, 29388510, 29966750, 45475518, 5989395, 9892960, 7137969, 5530675, 2278234, 9571067, 29644726, 30689189, 41083149 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2004-06-13"), "end-date": date("2004-11-28") } ] }
-{ "id": 10083103, "id-copy": 10083103, "alias": "Albertine", "name": "AlbertineShick", "user-since": datetime("2006-11-10T03:24:02.000Z"), "user-since-copy": datetime("2006-11-10T03:24:02.000Z"), "friend-ids": {{ 22979883, 41779991, 30340160, 44852777, 43786950, 33382165, 898482, 16427018, 1264379, 19925419, 10166319, 12658187, 38802346 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2010-03-05") } ] }
-{ "id": 10085446, "id-copy": 10085446, "alias": "Merla", "name": "MerlaWhitehead", "user-since": datetime("2006-12-08T11:13:30.000Z"), "user-since-copy": datetime("2006-12-08T11:13:30.000Z"), "friend-ids": {{ 44039547 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-03-16"), "end-date": date("2009-04-16") } ] }
-{ "id": 10086913, "id-copy": 10086913, "alias": "Margaretta", "name": "MargarettaPfeifer", "user-since": datetime("2012-03-04T14:47:18.000Z"), "user-since-copy": datetime("2012-03-04T14:47:18.000Z"), "friend-ids": {{ 9800482, 3761286, 34428154, 18082184, 14845214, 33053674, 46786785, 22235473, 23677556, 24819784, 47587008, 36939436, 14987278 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2003-07-08"), "end-date": date("2010-03-01") } ] }
-{ "id": 10090042, "id-copy": 10090042, "alias": "Gaye", "name": "GayeHayhurst", "user-since": datetime("2006-09-23T14:26:31.000Z"), "user-since-copy": datetime("2006-09-23T14:26:31.000Z"), "friend-ids": {{ 41099035, 16443590, 9899624, 2459064, 25428448, 1420220, 1487058, 13700561, 11008052, 36459693, 45632468, 30351729, 33053870, 26372759, 10801940, 37166367 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2005-07-15"), "end-date": date("2010-05-04") } ] }
-{ "id": 10116496, "id-copy": 10116496, "alias": "Gena", "name": "GenaJerome", "user-since": datetime("2005-03-04T21:38:41.000Z"), "user-since-copy": datetime("2005-03-04T21:38:41.000Z"), "friend-ids": {{ 11698908, 11838778, 10546816, 13504928, 25681727, 20198355, 28316946, 13835662, 16328293, 39540292, 43990464, 31393679, 34806990, 19167324, 8558031, 37794176, 14389975 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2003-10-01"), "end-date": date("2006-06-13") } ] }
-{ "id": 10122346, "id-copy": 10122346, "alias": "Salal", "name": "SalalPearson", "user-since": datetime("2011-11-14T10:42:11.000Z"), "user-since-copy": datetime("2011-11-14T10:42:11.000Z"), "friend-ids": {{ 44003884, 37124809, 7600567, 5158911, 31009406, 10708460 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2001-02-17"), "end-date": date("2010-06-23") } ] }
-{ "id": 10123051, "id-copy": 10123051, "alias": "Rowland", "name": "RowlandWaldron", "user-since": datetime("2011-08-01T17:20:14.000Z"), "user-since-copy": datetime("2011-08-01T17:20:14.000Z"), "friend-ids": {{ 7693849, 5416143, 10885197, 39771258, 41278769, 16236783, 18739058, 2293485, 32013369, 34882536, 14339467, 3680575, 4461977, 33715303, 26345760, 45729149, 17585375, 39496021 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2001-12-10"), "end-date": date("2006-04-07") } ] }
-{ "id": 10128076, "id-copy": 10128076, "alias": "Parker", "name": "ParkerHutton", "user-since": datetime("2011-06-05T03:46:01.000Z"), "user-since-copy": datetime("2011-06-05T03:46:01.000Z"), "friend-ids": {{ 24818185, 42512828, 22798434, 38901116, 12147430, 47942796, 34742031, 7142883, 11882526, 16055416, 3892909, 12824325, 13378363, 34281637, 15457426, 24092146, 27678834, 15804956 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2007-04-12"), "end-date": date("2009-05-09") } ] }
-{ "id": 10131352, "id-copy": 10131352, "alias": "Brett", "name": "BrettBullard", "user-since": datetime("2011-03-20T00:21:15.000Z"), "user-since-copy": datetime("2011-03-20T00:21:15.000Z"), "friend-ids": {{ 42102691, 34313392, 19476509, 40509353, 40764048, 32856149, 20306336, 18276288, 34284082, 32265145, 23912229, 7426729, 26377621, 43687843, 6140857, 4573908, 6840657, 18335864, 19868141, 6051525 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2005-11-09"), "end-date": date("2008-12-05") } ] }
-{ "id": 10132771, "id-copy": 10132771, "alias": "Gaenor", "name": "GaenorEvans", "user-since": datetime("2006-01-23T20:07:34.000Z"), "user-since-copy": datetime("2006-01-23T20:07:34.000Z"), "friend-ids": {{ 20344517, 47988409, 39449785, 16775663, 20200468 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-03-17") } ] }
-{ "id": 10138039, "id-copy": 10138039, "alias": "Farah", "name": "FarahAnn", "user-since": datetime("2008-05-10T19:04:28.000Z"), "user-since-copy": datetime("2008-05-10T19:04:28.000Z"), "friend-ids": {{ 32501277, 13715476, 10452566, 2652600, 16449577, 12508457, 30925424, 21595197, 26030962, 31683678 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2003-10-02") } ] }
-{ "id": 10150873, "id-copy": 10150873, "alias": "Shanice", "name": "ShaniceReiss", "user-since": datetime("2005-07-07T09:46:00.000Z"), "user-since-copy": datetime("2005-07-07T09:46:00.000Z"), "friend-ids": {{ 29208488, 6994033, 13074568, 31547206, 2547580, 15915539, 37448883, 38739687, 33246865, 28231547, 33861348, 44929557, 13977747, 44297013, 22367804 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2002-09-07"), "end-date": date("2006-04-23") } ] }
-{ "id": 10186180, "id-copy": 10186180, "alias": "Mina", "name": "MinaGist", "user-since": datetime("2012-07-05T21:56:14.000Z"), "user-since-copy": datetime("2012-07-05T21:56:14.000Z"), "friend-ids": {{ 12424234, 41863508, 44607839, 36984124, 3839840, 38458170, 41721653, 4785194, 20595881, 13515001 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-07-19") } ] }
-{ "id": 10188805, "id-copy": 10188805, "alias": "Margarita", "name": "MargaritaBrinigh", "user-since": datetime("2011-06-26T06:22:38.000Z"), "user-since-copy": datetime("2011-06-26T06:22:38.000Z"), "friend-ids": {{ 39275311, 42262790, 35041935, 12137373, 8507536 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2004-03-17") } ] }
-{ "id": 10193368, "id-copy": 10193368, "alias": "Oneida", "name": "OneidaEve", "user-since": datetime("2005-01-16T07:26:07.000Z"), "user-since-copy": datetime("2005-01-16T07:26:07.000Z"), "friend-ids": {{ 46396755, 39763353, 13661339, 5992749, 293256, 15572483, 16775625, 21543680 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2003-01-12"), "end-date": date("2008-03-22") } ] }
-{ "id": 10211827, "id-copy": 10211827, "alias": "Fanny", "name": "FannyHarrold", "user-since": datetime("2010-08-28T09:57:52.000Z"), "user-since-copy": datetime("2010-08-28T09:57:52.000Z"), "friend-ids": {{ 4061493, 30492642, 8550070, 34805906, 5798646, 39169853, 45190690, 34218456, 3758565, 18038216 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2008-10-14"), "end-date": date("2008-05-18") } ] }
-{ "id": 10215280, "id-copy": 10215280, "alias": "Barbara", "name": "BarbaraEve", "user-since": datetime("2012-03-09T01:36:52.000Z"), "user-since-copy": datetime("2012-03-09T01:36:52.000Z"), "friend-ids": {{ 32562793, 33679771, 10306498, 37847497, 30180151, 3504698 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2011-12-14") } ] }
-{ "id": 10238749, "id-copy": 10238749, "alias": "Elspeth", "name": "ElspethFilby", "user-since": datetime("2010-02-08T22:55:13.000Z"), "user-since-copy": datetime("2010-02-08T22:55:13.000Z"), "friend-ids": {{ 307224, 16533888 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2011-12-10") } ] }
-{ "id": 10269739, "id-copy": 10269739, "alias": "Shantel", "name": "ShantelEve", "user-since": datetime("2012-06-06T00:37:05.000Z"), "user-since-copy": datetime("2012-06-06T00:37:05.000Z"), "friend-ids": {{ 39436396, 20382971, 47821933, 28867521, 23217564, 40672635, 34693766, 4383592, 42534606, 23535312, 9112260, 4828073, 37429286, 27965200, 30257544, 47609429, 18527025, 33339218, 898986, 2817270, 6040099, 47802547 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2000-05-24") } ] }
-{ "id": 10283503, "id-copy": 10283503, "alias": "Terrilyn", "name": "TerrilynZadovsky", "user-since": datetime("2007-06-17T05:40:01.000Z"), "user-since-copy": datetime("2007-06-17T05:40:01.000Z"), "friend-ids": {{ 30185148, 22395650, 3212998, 41571861, 21336440, 41050091 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2009-12-14") } ] }
-{ "id": 10283941, "id-copy": 10283941, "alias": "Jeffie", "name": "JeffieChappel", "user-since": datetime("2012-06-17T10:07:53.000Z"), "user-since-copy": datetime("2012-06-17T10:07:53.000Z"), "friend-ids": {{ 37665650, 44995551, 8518132, 25975224, 22980129, 41720034, 42152946, 26671472, 25698917, 24270208, 36866555, 6728174, 46967331, 31563323, 1382901, 6764335, 35373496 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2005-01-26") } ] }
-{ "id": 10295389, "id-copy": 10295389, "alias": "Major", "name": "MajorDrabble", "user-since": datetime("2009-05-23T12:56:48.000Z"), "user-since-copy": datetime("2009-05-23T12:56:48.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2000-10-26") } ] }
-{ "id": 10307032, "id-copy": 10307032, "alias": "Quentin", "name": "QuentinSauter", "user-since": datetime("2012-07-11T07:16:43.000Z"), "user-since-copy": datetime("2012-07-11T07:16:43.000Z"), "friend-ids": {{ 1926278, 42211794, 1508832, 14973540, 6721046, 28872485, 5047722, 7805271, 31508326, 20891455, 38735410, 13190567, 18209753, 44468536, 34640135, 47290587, 25576626 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-02-13") } ] }
-{ "id": 10322023, "id-copy": 10322023, "alias": "Shanita", "name": "ShanitaBeedell", "user-since": datetime("2011-06-09T23:50:09.000Z"), "user-since-copy": datetime("2011-06-09T23:50:09.000Z"), "friend-ids": {{ 22628842, 2169935, 20656034, 9086684, 17234788, 11936164, 12465122, 2543006, 40067557, 36767662, 633930, 41805132, 13246529, 43801547, 44953975, 36902947, 34935791, 22923033, 28190533, 18230134, 9484458, 21184932 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2011-10-10") } ] }
-{ "id": 10322398, "id-copy": 10322398, "alias": "Alanna", "name": "AlannaBollinger", "user-since": datetime("2008-09-01T20:05:18.000Z"), "user-since-copy": datetime("2008-09-01T20:05:18.000Z"), "friend-ids": {{ 4294902, 42664964 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2009-10-08"), "end-date": date("2011-09-26") } ] }
-{ "id": 10346116, "id-copy": 10346116, "alias": "Breana", "name": "BreanaPainter", "user-since": datetime("2012-04-05T12:15:17.000Z"), "user-since-copy": datetime("2012-04-05T12:15:17.000Z"), "friend-ids": {{ 39999376, 5382299, 36254541, 16829210, 7084172, 13545656, 24681698, 34171417, 28514693, 8090159, 35046661, 44544921, 47754565, 28732689, 19680056, 21398367, 39260450 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2012-08-24"), "end-date": date("2012-08-24") } ] }
-{ "id": 10346338, "id-copy": 10346338, "alias": "Caelie", "name": "CaelieYates", "user-since": datetime("2011-11-10T19:17:38.000Z"), "user-since-copy": datetime("2011-11-10T19:17:38.000Z"), "friend-ids": {{ 3910270, 7940512, 32351319, 27966615, 33829964, 34529061, 19420019, 7423616, 22246488, 7284253, 8419860, 43330144 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2005-02-07"), "end-date": date("2011-09-05") } ] }
-{ "id": 10350421, "id-copy": 10350421, "alias": "Diane", "name": "DianeFisher", "user-since": datetime("2010-10-19T11:08:52.000Z"), "user-since-copy": datetime("2010-10-19T11:08:52.000Z"), "friend-ids": {{ 22455675, 20415125, 21917591, 44414352, 39158851, 3446534, 6627839, 28358200, 1176552, 37914774 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2008-06-04"), "end-date": date("2009-09-11") } ] }
-{ "id": 10353946, "id-copy": 10353946, "alias": "Cass", "name": "CassPirl", "user-since": datetime("2010-10-25T21:08:28.000Z"), "user-since-copy": datetime("2010-10-25T21:08:28.000Z"), "friend-ids": {{ 43117144, 29185875, 28524977, 4904289, 37353728, 30484159, 40114905, 18108320, 46098949, 30207639 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2006-06-10") } ] }
-{ "id": 10361965, "id-copy": 10361965, "alias": "Arlen", "name": "ArlenFlick", "user-since": datetime("2011-07-14T18:38:37.000Z"), "user-since-copy": datetime("2011-07-14T18:38:37.000Z"), "friend-ids": {{ 34249140, 2887282, 47622716, 3897801, 33692288, 14374380, 14183995, 41311739, 6378075, 17721901, 20807501, 8908974, 41080464, 26497672 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2008-05-18"), "end-date": date("2011-09-18") } ] }
-{ "id": 10364356, "id-copy": 10364356, "alias": "Katharine", "name": "KatharineHoward", "user-since": datetime("2012-03-04T04:40:32.000Z"), "user-since-copy": datetime("2012-03-04T04:40:32.000Z"), "friend-ids": {{ 38784, 9497194, 38432548, 30160971, 16843331, 36942612, 32507064, 41108421, 31761239, 20202472, 37170299, 39217222, 14201294, 46319310 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2011-11-09"), "end-date": date("2011-07-18") } ] }
-{ "id": 10367416, "id-copy": 10367416, "alias": "Damion", "name": "DamionDean", "user-since": datetime("2008-01-06T05:55:09.000Z"), "user-since-copy": datetime("2008-01-06T05:55:09.000Z"), "friend-ids": {{ 45804001, 13077962, 28346489, 25877214, 10164033, 42903493, 66753, 27961850, 41137249, 20490506 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2007-12-18") } ] }
-{ "id": 10391044, "id-copy": 10391044, "alias": "Kendrick", "name": "KendrickNabholz", "user-since": datetime("2007-10-11T19:49:13.000Z"), "user-since-copy": datetime("2007-10-11T19:49:13.000Z"), "friend-ids": {{ 39264696, 35794708, 222108, 29542536, 34470710, 16736694, 36282306, 12411530, 12507843, 30193842, 45764599, 32250152, 16472135, 26507230, 17443301, 16787960, 17651924, 37659951, 28610616, 12928071 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2007-05-07") } ] }
-{ "id": 10394488, "id-copy": 10394488, "alias": "Oswald", "name": "OswaldRay", "user-since": datetime("2006-02-12T17:39:23.000Z"), "user-since-copy": datetime("2006-02-12T17:39:23.000Z"), "friend-ids": {{ 14370372, 14174983, 7749259, 39375970, 1755409, 9056913 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2011-12-04"), "end-date": date("2011-06-08") } ] }
-{ "id": 10396831, "id-copy": 10396831, "alias": "Carman", "name": "CarmanElder", "user-since": datetime("2011-12-27T21:50:41.000Z"), "user-since-copy": datetime("2011-12-27T21:50:41.000Z"), "friend-ids": {{ 41782166, 39862540, 39100006, 45023958, 29253172, 31208143, 12637805, 5844876, 37296616, 20896053, 18358082, 11068853, 5350064, 14456765, 15758928 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2012-05-18"), "end-date": date("2012-07-26") } ] }
-{ "id": 10404706, "id-copy": 10404706, "alias": "Rylan", "name": "RylanEmrick", "user-since": datetime("2008-11-23T00:55:36.000Z"), "user-since-copy": datetime("2008-11-23T00:55:36.000Z"), "friend-ids": {{ 17936230, 20908773, 34834317, 26134774, 3534090, 7699389, 11743997, 37809096, 23228338, 19069026, 662582, 40839640, 26706968, 42711557, 28658968, 39161015, 29201879, 7516443, 21802464, 16456657, 32689464 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2012-02-16") } ] }
-{ "id": 10444585, "id-copy": 10444585, "alias": "Harrietta", "name": "HarriettaDunkle", "user-since": datetime("2012-01-26T16:14:19.000Z"), "user-since-copy": datetime("2012-01-26T16:14:19.000Z"), "friend-ids": {{ 9013750, 39577621, 40067238, 24177261, 41169182, 5939218, 13820152, 47741655 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2004-06-13") } ] }
-{ "id": 10453144, "id-copy": 10453144, "alias": "Jason", "name": "JasonSachse", "user-since": datetime("2009-01-25T10:27:17.000Z"), "user-since-copy": datetime("2009-01-25T10:27:17.000Z"), "friend-ids": {{ 12949882, 32048809, 23087453, 3994051, 20775019, 22184704, 38106058, 34520240, 13724092, 16309751, 25955640, 4812195, 40546554, 12695295, 16574455, 38615670, 43405164, 7997407, 12239790 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2005-08-01"), "end-date": date("2008-02-08") } ] }
-{ "id": 10469071, "id-copy": 10469071, "alias": "Apryl", "name": "AprylWatson", "user-since": datetime("2006-10-03T08:37:12.000Z"), "user-since-copy": datetime("2006-10-03T08:37:12.000Z"), "friend-ids": {{ 4517575, 34635569, 1199146 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2001-02-01"), "end-date": date("2007-09-01") } ] }
-{ "id": 10469980, "id-copy": 10469980, "alias": "Rosalynne", "name": "RosalynneZalack", "user-since": datetime("2012-03-07T10:12:20.000Z"), "user-since-copy": datetime("2012-03-07T10:12:20.000Z"), "friend-ids": {{ 46118617, 27264184, 8045697, 30832992, 47861079, 24266748, 10689886, 14799850, 1178687, 39540720, 17568852, 24394222, 10078451, 4748570, 47808632, 35277954, 8802885, 13747535, 22203533, 42065169, 19096770, 14087466, 45753492 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2006-04-15"), "end-date": date("2010-07-14") } ] }
-{ "id": 10486213, "id-copy": 10486213, "alias": "Modesto", "name": "ModestoCox", "user-since": datetime("2006-02-07T05:43:24.000Z"), "user-since-copy": datetime("2006-02-07T05:43:24.000Z"), "friend-ids": {{ 42665859, 12929499, 5618502, 24287766, 38722882, 5162913, 2978226, 37521984, 43144325, 3313029, 17680751, 726799 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2007-08-12") } ] }
-{ "id": 10487029, "id-copy": 10487029, "alias": "Fredericka", "name": "FrederickaShea", "user-since": datetime("2011-04-07T06:12:40.000Z"), "user-since-copy": datetime("2011-04-07T06:12:40.000Z"), "friend-ids": {{ 45223639, 1019151, 30626857, 10247171, 36952244, 36646177, 2396690, 26604216, 19215860, 20900949, 14160764 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-01-08") } ] }
-{ "id": 10495420, "id-copy": 10495420, "alias": "Wendy", "name": "WendyMcloskey", "user-since": datetime("2011-04-26T23:38:24.000Z"), "user-since-copy": datetime("2011-04-26T23:38:24.000Z"), "friend-ids": {{ 16762653, 46262691, 12313140, 20481262, 347993, 23105127, 1680519, 20880265, 45611347, 21907223, 46615281, 17188244, 44019800, 46943250, 28647738, 16792673, 29406270, 42714079 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2008-08-27") } ] }
-{ "id": 10498285, "id-copy": 10498285, "alias": "Kiley", "name": "KileyBridger", "user-since": datetime("2006-05-14T21:55:34.000Z"), "user-since-copy": datetime("2006-05-14T21:55:34.000Z"), "friend-ids": {{ 38780484, 46190003, 905670, 35609390, 46621151, 5099226, 24328595, 16340411, 13326485, 13872400, 35896828, 9196151, 8525875, 7461206, 28379538, 46461267, 45270205, 35718577, 5310596, 7080391 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2009-11-11"), "end-date": date("2009-06-23") } ] }
-{ "id": 10504084, "id-copy": 10504084, "alias": "Etsuko", "name": "EtsukoDealtry", "user-since": datetime("2012-05-11T00:35:22.000Z"), "user-since-copy": datetime("2012-05-11T00:35:22.000Z"), "friend-ids": {{ 27578969, 40308832, 15379566, 8664135, 21276773, 43659426, 28027401, 23264043, 23981731, 19124540, 36281456, 38766688, 37886842, 20522702, 28559857, 9838362, 30409517, 14237008, 41013610, 41586760, 37285778, 29427060, 45678692, 32255048 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-10-12"), "end-date": date("2011-12-04") } ] }
-{ "id": 10513507, "id-copy": 10513507, "alias": "Jasmin", "name": "JasminHatfield", "user-since": datetime("2009-06-25T22:45:16.000Z"), "user-since-copy": datetime("2009-06-25T22:45:16.000Z"), "friend-ids": {{ 31323261 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-05-12"), "end-date": date("2003-05-07") } ] }
-{ "id": 10515721, "id-copy": 10515721, "alias": "Mariano", "name": "MarianoTrout", "user-since": datetime("2007-08-27T09:33:28.000Z"), "user-since-copy": datetime("2007-08-27T09:33:28.000Z"), "friend-ids": {{ 18516004, 4847094, 31548989, 28302698, 18308169, 15068883, 33358074, 19739053, 34017693 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2004-10-18") } ] }
-{ "id": 10533343, "id-copy": 10533343, "alias": "Gwendolen", "name": "GwendolenHanseu", "user-since": datetime("2007-02-04T19:56:51.000Z"), "user-since-copy": datetime("2007-02-04T19:56:51.000Z"), "friend-ids": {{ 25281794, 21814505, 11684475, 5599252, 17261378, 11061422, 27392332, 47872606, 39198697, 17314413, 4034634, 42776559, 43885593, 24835625, 18150148, 4946129, 9288372, 5675162, 34976580 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2010-07-10") } ] }
-{ "id": 10540441, "id-copy": 10540441, "alias": "Albert", "name": "AlbertBasinger", "user-since": datetime("2007-05-12T06:03:38.000Z"), "user-since-copy": datetime("2007-05-12T06:03:38.000Z"), "friend-ids": {{ 36392592, 35815177, 22050314, 45279196, 15405747, 33802667, 44081359, 2027267, 47159697, 20007080 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2002-10-04"), "end-date": date("2005-08-17") } ] }
-{ "id": 10552405, "id-copy": 10552405, "alias": "Les", "name": "LesBarth", "user-since": datetime("2008-04-02T11:02:37.000Z"), "user-since-copy": datetime("2008-04-02T11:02:37.000Z"), "friend-ids": {{ 33645432, 43039707 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2000-09-18") } ] }
-{ "id": 10563310, "id-copy": 10563310, "alias": "Justina", "name": "JustinaHall", "user-since": datetime("2010-08-24T08:57:45.000Z"), "user-since-copy": datetime("2010-08-24T08:57:45.000Z"), "friend-ids": {{ 42796179, 25994871, 35439919, 28722419, 7189994, 41505357, 35095639, 14693797, 36519323, 32598167, 6323551, 14565174, 35997662, 9705559, 3996730 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2011-02-20"), "end-date": date("2011-05-05") } ] }
-{ "id": 10573795, "id-copy": 10573795, "alias": "Neil", "name": "NeilMilne", "user-since": datetime("2005-11-15T02:57:46.000Z"), "user-since-copy": datetime("2005-11-15T02:57:46.000Z"), "friend-ids": {{ 33469327, 4261514, 43412669, 17289131, 27535421, 15267017, 14005060 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2001-11-13"), "end-date": date("2001-10-28") } ] }
-{ "id": 10636498, "id-copy": 10636498, "alias": "Grahame", "name": "GrahameLeslie", "user-since": datetime("2006-01-17T16:17:07.000Z"), "user-since-copy": datetime("2006-01-17T16:17:07.000Z"), "friend-ids": {{ 3924169, 14543253, 19830425, 34696361, 26630699, 47664771 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2004-03-25") } ] }
-{ "id": 10642153, "id-copy": 10642153, "alias": "Wally", "name": "WallyRiggle", "user-since": datetime("2011-10-10T21:43:33.000Z"), "user-since-copy": datetime("2011-10-10T21:43:33.000Z"), "friend-ids": {{ 32910135, 45556839, 6526394, 13177451, 10588491, 40270322, 17438379, 21204776, 46036116, 44249789, 7375979, 43487252, 24858016, 3947997 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-10-10") } ] }
-{ "id": 10650265, "id-copy": 10650265, "alias": "Kristia", "name": "KristiaCowart", "user-since": datetime("2005-09-27T20:13:12.000Z"), "user-since-copy": datetime("2005-09-27T20:13:12.000Z"), "friend-ids": {{ 41553475, 45442923, 20846576, 6432869, 40830841 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2006-05-05") } ] }
-{ "id": 10659022, "id-copy": 10659022, "alias": "Cecelia", "name": "CeceliaHandyside", "user-since": datetime("2007-02-22T12:42:30.000Z"), "user-since-copy": datetime("2007-02-22T12:42:30.000Z"), "friend-ids": {{ 9051, 38746030, 6178049, 22068473, 25755202, 11577837, 28994476 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2004-07-09"), "end-date": date("2009-10-14") } ] }
-{ "id": 10668283, "id-copy": 10668283, "alias": "Dorian", "name": "DorianTomlinson", "user-since": datetime("2008-06-22T00:01:46.000Z"), "user-since-copy": datetime("2008-06-22T00:01:46.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2004-12-17") } ] }
-{ "id": 10674199, "id-copy": 10674199, "alias": "Dorothy", "name": "DorothyPritchard", "user-since": datetime("2007-09-19T04:32:05.000Z"), "user-since-copy": datetime("2007-09-19T04:32:05.000Z"), "friend-ids": {{ 11239155, 14468542, 8244419, 30563447, 2235193, 33015958, 11941749, 22198664, 41531114, 11614864, 43486312, 11394784, 46038310, 8248070, 12346192 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2000-10-03") } ] }
-{ "id": 10686646, "id-copy": 10686646, "alias": "Deborah", "name": "DeborahRosenstiehl", "user-since": datetime("2012-06-18T16:51:32.000Z"), "user-since-copy": datetime("2012-06-18T16:51:32.000Z"), "friend-ids": {{ 34005621, 6910583, 11226890, 1333457, 13615971, 15332838, 30484423, 38261521, 39526604, 12093262, 15397660, 29644860, 36715060, 16753181 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2005-07-01"), "end-date": date("2007-10-22") } ] }
-{ "id": 10703185, "id-copy": 10703185, "alias": "Sabina", "name": "SabinaHall", "user-since": datetime("2012-05-18T20:37:33.000Z"), "user-since-copy": datetime("2012-05-18T20:37:33.000Z"), "friend-ids": {{ 432154, 6472603, 35649237, 46598578, 35486135, 44354453 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2002-11-04"), "end-date": date("2011-10-12") } ] }
-{ "id": 10710526, "id-copy": 10710526, "alias": "Heike", "name": "HeikeReed", "user-since": datetime("2009-08-15T19:20:30.000Z"), "user-since-copy": datetime("2009-08-15T19:20:30.000Z"), "friend-ids": {{ 36253853, 35694929, 43324582, 24829816 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2011-02-12"), "end-date": date("2011-01-22") } ] }
-{ "id": 10734148, "id-copy": 10734148, "alias": "Allannah", "name": "AllannahHoffhants", "user-since": datetime("2005-11-18T00:54:25.000Z"), "user-since-copy": datetime("2005-11-18T00:54:25.000Z"), "friend-ids": {{ 26897353, 13343289, 1991130, 39024681, 21839148, 38693973, 19132058, 17589948, 13367008, 30389658, 21757614, 45618415, 23559236, 35669455, 22088928, 2531202, 120534, 867017, 8590987, 25956219, 21819960, 41918122, 31042839, 15019901 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2004-10-25") } ] }
-{ "id": 10738096, "id-copy": 10738096, "alias": "Dori", "name": "DoriAlcocke", "user-since": datetime("2010-05-21T04:59:08.000Z"), "user-since-copy": datetime("2010-05-21T04:59:08.000Z"), "friend-ids": {{ 44039507, 40951102, 39132038, 31982600, 46848423, 43375356, 6188106, 3044041, 38421537, 18640387, 21639042, 11192576, 15659477, 360828, 26875197, 19433881 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2008-02-19"), "end-date": date("2011-03-24") } ] }
-{ "id": 10765090, "id-copy": 10765090, "alias": "Louiza", "name": "LouizaMcelroy", "user-since": datetime("2012-08-14T02:46:00.000Z"), "user-since-copy": datetime("2012-08-14T02:46:00.000Z"), "friend-ids": {{ 14365973, 9091111, 44279279, 45125689, 29955385, 23874606, 18142514, 24878700, 13928633, 47391704, 29729670, 35422059, 987030, 3200788, 7640346, 32947024, 32550247, 25746061, 34112521, 41193622, 2620213, 30090329, 5531715 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2002-06-16"), "end-date": date("2003-05-13") } ] }
-{ "id": 10768810, "id-copy": 10768810, "alias": "Gaston", "name": "GastonBender", "user-since": datetime("2008-05-24T17:27:14.000Z"), "user-since-copy": datetime("2008-05-24T17:27:14.000Z"), "friend-ids": {{ 29652235, 40180625, 34608178, 43814186, 9682855, 24692412, 33119254, 20480079, 35147289, 24629496, 1449575 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-04-06") } ] }
-{ "id": 10771030, "id-copy": 10771030, "alias": "Jen", "name": "JenZaun", "user-since": datetime("2006-12-02T14:42:43.000Z"), "user-since-copy": datetime("2006-12-02T14:42:43.000Z"), "friend-ids": {{ 38166077 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2003-12-01"), "end-date": date("2010-04-12") } ] }
-{ "id": 10772929, "id-copy": 10772929, "alias": "Hugh", "name": "HughTrout", "user-since": datetime("2008-01-24T03:16:55.000Z"), "user-since-copy": datetime("2008-01-24T03:16:55.000Z"), "friend-ids": {{ 39704817, 19656412, 37084896, 5219803, 23455492, 14248249, 26973609, 4607440, 25844255, 3032226, 45432192, 47011338, 41460367, 28779211, 31780563, 31808543, 29732190, 1264228, 7989711, 38397890, 7638694, 3002993, 8960147, 46258407 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2010-08-02"), "end-date": date("2010-05-08") } ] }
-{ "id": 10779373, "id-copy": 10779373, "alias": "Donya", "name": "DonyaWegley", "user-since": datetime("2012-03-28T01:26:06.000Z"), "user-since-copy": datetime("2012-03-28T01:26:06.000Z"), "friend-ids": {{ 24977052, 19856115, 36795249, 7875698, 23317261, 5916235, 17789989, 41932923 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2008-10-18") } ] }
-{ "id": 10786438, "id-copy": 10786438, "alias": "Sherika", "name": "SherikaShick", "user-since": datetime("2005-05-18T21:46:18.000Z"), "user-since-copy": datetime("2005-05-18T21:46:18.000Z"), "friend-ids": {{ 11188876, 12936787, 43459190, 40396919, 7166644, 20299758 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2002-06-09") } ] }
-{ "id": 10794448, "id-copy": 10794448, "alias": "Delmar", "name": "DelmarDowning", "user-since": datetime("2012-03-10T23:41:49.000Z"), "user-since-copy": datetime("2012-03-10T23:41:49.000Z"), "friend-ids": {{ 34002211, 41487, 45067426, 9754093, 23041928, 41378740, 4013550, 11584362, 46202858, 43273004, 35465505 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2005-09-12") } ] }
-{ "id": 10795960, "id-copy": 10795960, "alias": "Hallam", "name": "HallamBousum", "user-since": datetime("2010-04-23T14:02:10.000Z"), "user-since-copy": datetime("2010-04-23T14:02:10.000Z"), "friend-ids": {{ 23447883, 39605256, 41998325 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2007-04-18") } ] }
-{ "id": 10797166, "id-copy": 10797166, "alias": "Alethea", "name": "AletheaMills", "user-since": datetime("2011-01-10T03:06:16.000Z"), "user-since-copy": datetime("2011-01-10T03:06:16.000Z"), "friend-ids": {{ 25077851, 2396037, 25762626, 31358162, 41492027, 31211140, 38478662, 9688210, 16865534, 4209161, 19863828, 23760993, 36041139, 46184667 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2004-05-04") } ] }
-{ "id": 10803184, "id-copy": 10803184, "alias": "Daria", "name": "DariaPyle", "user-since": datetime("2010-11-22T05:29:27.000Z"), "user-since-copy": datetime("2010-11-22T05:29:27.000Z"), "friend-ids": {{ 26747755, 39431389, 24370112, 37832812, 20626868, 30614988, 38041392, 31908762, 47561829, 45121087, 24496373, 32944554, 16470795, 11915899, 29900938, 4003497, 38829225, 36390033, 36474051 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2009-10-03") } ] }
-{ "id": 10808284, "id-copy": 10808284, "alias": "Natalie", "name": "NatalieJewell", "user-since": datetime("2007-04-15T14:17:38.000Z"), "user-since-copy": datetime("2007-04-15T14:17:38.000Z"), "friend-ids": {{ 20839191, 18422391, 2571767, 39525211, 38867255, 13491856 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2005-09-10"), "end-date": date("2011-01-20") } ] }
-{ "id": 10809730, "id-copy": 10809730, "alias": "Algar", "name": "AlgarZaun", "user-since": datetime("2008-08-14T06:37:59.000Z"), "user-since-copy": datetime("2008-08-14T06:37:59.000Z"), "friend-ids": {{ 12676185, 26087426, 42241358, 47854149, 22179884, 34701736, 35541344, 46257087, 35091522, 10779069 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2010-09-13") } ] }
-{ "id": 10827610, "id-copy": 10827610, "alias": "Madelina", "name": "MadelinaCamp", "user-since": datetime("2010-06-08T13:22:59.000Z"), "user-since-copy": datetime("2010-06-08T13:22:59.000Z"), "friend-ids": {{ 35445385, 15924939, 7897517, 15573537, 7234891, 46098859, 877311, 40923818, 45519215, 27332107, 1693386, 21101894, 35225 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2002-10-04") } ] }
-{ "id": 10834579, "id-copy": 10834579, "alias": "Penni", "name": "PenniBlunt", "user-since": datetime("2010-05-20T20:29:16.000Z"), "user-since-copy": datetime("2010-05-20T20:29:16.000Z"), "friend-ids": {{ 25926886, 10263270, 4098530, 40765625, 16591278 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2003-03-22") } ] }
-{ "id": 10856059, "id-copy": 10856059, "alias": "Leland", "name": "LelandMcdonald", "user-since": datetime("2006-09-26T03:35:07.000Z"), "user-since-copy": datetime("2006-09-26T03:35:07.000Z"), "friend-ids": {{ 29735881, 7080599, 14172811, 24274797, 5773081, 2653240, 18151967, 34988676, 6599030, 46463015, 23254278, 37618443, 32396573 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2004-01-17") } ] }
-{ "id": 10858339, "id-copy": 10858339, "alias": "Eugenio", "name": "EugenioLangston", "user-since": datetime("2006-06-14T22:24:18.000Z"), "user-since-copy": datetime("2006-06-14T22:24:18.000Z"), "friend-ids": {{ 18107191, 19162062, 26048227, 16199255, 32644324, 3917262, 38994370, 36221435, 34919041 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2011-02-03") } ] }
-{ "id": 10858909, "id-copy": 10858909, "alias": "Kiley", "name": "KileyCoates", "user-since": datetime("2011-02-03T03:12:41.000Z"), "user-since-copy": datetime("2011-02-03T03:12:41.000Z"), "friend-ids": {{ 47990206, 29775839, 33872749, 38952297, 38802567, 38822660, 12420330, 18852873, 30468156, 29085185, 2660660, 28283210, 6711584, 35851765, 31124383, 39930865, 18329720 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2006-09-22") } ] }
-{ "id": 10860286, "id-copy": 10860286, "alias": "Albert", "name": "AlbertMills", "user-since": datetime("2005-01-04T04:39:49.000Z"), "user-since-copy": datetime("2005-01-04T04:39:49.000Z"), "friend-ids": {{ 45171802, 36246654, 30029601, 40155304, 4876814, 275363, 46427463, 5698619, 34383185, 47844520, 45026162, 33852471, 36744791, 40565586, 47142152, 42828565 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2012-02-20"), "end-date": date("2012-03-21") } ] }
-{ "id": 10865788, "id-copy": 10865788, "alias": "Ebba", "name": "EbbaSwartzbaugh", "user-since": datetime("2007-08-18T11:38:20.000Z"), "user-since-copy": datetime("2007-08-18T11:38:20.000Z"), "friend-ids": {{ 12850265, 19824056, 2754383, 43333892, 9287993, 14972999, 3729396, 20735424 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2001-10-07"), "end-date": date("2004-07-17") } ] }
-{ "id": 10888777, "id-copy": 10888777, "alias": "Bevis", "name": "BevisStall", "user-since": datetime("2007-04-05T02:35:27.000Z"), "user-since-copy": datetime("2007-04-05T02:35:27.000Z"), "friend-ids": {{ 1924847, 33036971, 5163765, 37816368, 15975671, 11388174, 38485519, 43186487, 30402693, 34350975, 24348537, 34349089, 22680019, 30625064, 23751465, 9072515, 15915109 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2005-03-11") } ] }
-{ "id": 10899544, "id-copy": 10899544, "alias": "Valentine", "name": "ValentineFisher", "user-since": datetime("2008-07-04T14:36:11.000Z"), "user-since-copy": datetime("2008-07-04T14:36:11.000Z"), "friend-ids": {{ 26471524, 781270, 17136010, 12943313, 42125653, 40372131 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2008-09-02"), "end-date": date("2008-01-21") } ] }
-{ "id": 10901332, "id-copy": 10901332, "alias": "Caelie", "name": "CaelieShafer", "user-since": datetime("2011-09-24T05:08:05.000Z"), "user-since-copy": datetime("2011-09-24T05:08:05.000Z"), "friend-ids": {{ 40761096, 31796928, 1066172, 21271172, 41179382, 46260705, 9287042, 37605846, 18083603, 23469027, 45497916, 10102434, 724885, 31794816, 44125905, 46373183, 28321712 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2012-07-04") } ] }
-{ "id": 10905802, "id-copy": 10905802, "alias": "Jamika", "name": "JamikaJowers", "user-since": datetime("2007-05-24T01:31:04.000Z"), "user-since-copy": datetime("2007-05-24T01:31:04.000Z"), "friend-ids": {{ 16476991, 9041491, 10867973, 18057276, 13716912, 184635, 47717267, 37995364 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-08-20") } ] }
-{ "id": 10930153, "id-copy": 10930153, "alias": "Liliana", "name": "LilianaGoodman", "user-since": datetime("2009-06-22T20:57:17.000Z"), "user-since-copy": datetime("2009-06-22T20:57:17.000Z"), "friend-ids": {{ 4302195, 1569986, 5108357, 40772631, 30372008, 36454077, 26878227, 10958250, 46069776, 4779188, 46627230, 47074148, 25489453, 24956443, 31679399, 21835639, 42097220, 35662047, 6354581, 34282348, 13473927 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2008-10-25") } ] }
-{ "id": 10937395, "id-copy": 10937395, "alias": "Madlyn", "name": "MadlynRader", "user-since": datetime("2010-11-11T02:19:12.000Z"), "user-since-copy": datetime("2010-11-11T02:19:12.000Z"), "friend-ids": {{ 8750346, 40237703, 11127018, 23810876, 33862918, 8179642 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2011-03-12"), "end-date": date("2011-12-06") } ] }
-{ "id": 10938328, "id-copy": 10938328, "alias": "Tyrese", "name": "TyreseStainforth", "user-since": datetime("2011-03-03T04:21:04.000Z"), "user-since-copy": datetime("2011-03-03T04:21:04.000Z"), "friend-ids": {{ 33557445, 27981614, 25595450, 31820772, 42028444, 31389097, 16332592, 3555278, 45113070, 5198333 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2000-06-04") } ] }
-{ "id": 10953628, "id-copy": 10953628, "alias": "Clement", "name": "ClementHoenshell", "user-since": datetime("2009-01-24T03:52:54.000Z"), "user-since-copy": datetime("2009-01-24T03:52:54.000Z"), "friend-ids": {{ 24684431, 16961296, 13566818 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2011-05-07") } ] }
-{ "id": 10957867, "id-copy": 10957867, "alias": "Zach", "name": "ZachOppenheimer", "user-since": datetime("2012-01-01T14:40:11.000Z"), "user-since-copy": datetime("2012-01-01T14:40:11.000Z"), "friend-ids": {{ 27759480, 2112389, 8560433, 10052851, 37714587, 16717012, 36648956, 44803993, 36030695, 5359496, 32302980, 27143894, 19287706 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-05-14"), "end-date": date("2004-02-23") } ] }
-{ "id": 10967254, "id-copy": 10967254, "alias": "Andre", "name": "AndreCowper", "user-since": datetime("2011-12-21T20:22:47.000Z"), "user-since-copy": datetime("2011-12-21T20:22:47.000Z"), "friend-ids": {{ 23645341, 16267661, 7660549, 24716202, 20945538, 10125828, 1712260, 5309070, 16802418, 18273953, 12670834 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2003-12-25"), "end-date": date("2004-04-09") } ] }
-{ "id": 10970950, "id-copy": 10970950, "alias": "Shana", "name": "ShanaRose", "user-since": datetime("2008-09-17T10:03:01.000Z"), "user-since-copy": datetime("2008-09-17T10:03:01.000Z"), "friend-ids": {{ 21025589, 17977659, 39920039, 44311386, 2634251 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2006-09-17") } ] }
-{ "id": 10985830, "id-copy": 10985830, "alias": "Spencer", "name": "SpencerWilo", "user-since": datetime("2010-03-02T07:41:59.000Z"), "user-since-copy": datetime("2010-03-02T07:41:59.000Z"), "friend-ids": {{ 5766878, 20551454, 27297902, 44757901, 7660518, 28072828, 6387548, 6276027, 40692560, 36168648, 24514885, 40791549, 15536640, 23757967, 19875372 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2004-04-14"), "end-date": date("2009-02-17") } ] }
-{ "id": 11012734, "id-copy": 11012734, "alias": "Jordan", "name": "JordanSadley", "user-since": datetime("2011-02-26T18:40:19.000Z"), "user-since-copy": datetime("2011-02-26T18:40:19.000Z"), "friend-ids": {{ 37319587, 37212468, 3023956, 43125609 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2007-07-03"), "end-date": date("2011-01-25") } ] }
-{ "id": 11049274, "id-copy": 11049274, "alias": "Fitz", "name": "FitzBeail", "user-since": datetime("2012-08-10T03:25:57.000Z"), "user-since-copy": datetime("2012-08-10T03:25:57.000Z"), "friend-ids": {{ 39403330, 13441324, 723509, 34025727, 23266816, 33898717, 11053310, 14582395, 38435153, 45855468, 45712821 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2004-02-16"), "end-date": date("2007-01-07") } ] }
-{ "id": 11061631, "id-copy": 11061631, "alias": "Maxene", "name": "MaxeneKellogg", "user-since": datetime("2005-11-13T01:09:31.000Z"), "user-since-copy": datetime("2005-11-13T01:09:31.000Z"), "friend-ids": {{ 31578394, 39466620, 35741359, 14244925, 3000582, 39031643, 5008430, 18315325, 30440631, 37868108, 12014032, 32314102, 42887702, 1853960, 28022174, 2024670, 38864358, 42073112, 16259942, 34693959, 25315399, 37475597, 33599283 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2008-05-13") } ] }
-{ "id": 11064301, "id-copy": 11064301, "alias": "Dave", "name": "DaveNicholas", "user-since": datetime("2007-01-09T09:19:57.000Z"), "user-since-copy": datetime("2007-01-09T09:19:57.000Z"), "friend-ids": {{ 19136340, 40809808, 18774928, 405329, 27436466, 35586548, 16671212, 44582715, 47932437, 22599645, 26281489, 39246487, 39088455, 43696576, 28175190 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2005-02-04") } ] }
-{ "id": 11109553, "id-copy": 11109553, "alias": "Walker", "name": "WalkerDrennan", "user-since": datetime("2007-05-03T02:10:46.000Z"), "user-since-copy": datetime("2007-05-03T02:10:46.000Z"), "friend-ids": {{ 38288636, 35385410, 24803705, 31461936, 34309407 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2010-05-20") } ] }
-{ "id": 11129635, "id-copy": 11129635, "alias": "Porter", "name": "PorterRohtin", "user-since": datetime("2005-08-07T05:18:16.000Z"), "user-since-copy": datetime("2005-08-07T05:18:16.000Z"), "friend-ids": {{ 15192554, 37509296, 35638203, 5517199, 3781940, 43497242, 28477558, 4325184, 34919156, 18037278, 36486191, 13966437, 16629611, 40623060 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2005-07-13") } ] }
-{ "id": 11140213, "id-copy": 11140213, "alias": "Montgomery", "name": "MontgomeryWhittier", "user-since": datetime("2007-06-19T17:46:13.000Z"), "user-since-copy": datetime("2007-06-19T17:46:13.000Z"), "friend-ids": {{ 32831460, 6030454, 30437362, 21866470, 17388602, 40815157, 20000967, 47555494, 5818137, 40634742, 21692148, 2365521, 33290069, 46471164, 9192561, 35768343, 7552168, 3577338, 5346012, 31129868 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2008-02-24") } ] }
-{ "id": 11162920, "id-copy": 11162920, "alias": "Michael", "name": "MichaelJohns", "user-since": datetime("2007-12-21T06:52:31.000Z"), "user-since-copy": datetime("2007-12-21T06:52:31.000Z"), "friend-ids": {{ 47587192, 5639113, 24042062, 26141562, 4128346, 25702038, 16421361, 44444678, 30940270, 16928219, 27816662, 37884076, 40854508, 21061894, 42850960, 42453718, 2763269, 16035171, 47650572, 26811622 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-02-24") } ] }
-{ "id": 11195221, "id-copy": 11195221, "alias": "Clement", "name": "ClementBriner", "user-since": datetime("2006-12-27T02:29:02.000Z"), "user-since-copy": datetime("2006-12-27T02:29:02.000Z"), "friend-ids": {{ 33023290 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2000-06-05") } ] }
-{ "id": 11203174, "id-copy": 11203174, "alias": "Lise", "name": "LiseRockwell", "user-since": datetime("2005-04-21T02:17:33.000Z"), "user-since-copy": datetime("2005-04-21T02:17:33.000Z"), "friend-ids": {{ 25322984, 687106, 15193641, 24397137, 34772763, 24725595, 30853266, 14933558, 36895249, 39451299, 2620397, 44594032, 3455415, 39921033, 21621070, 800967 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2006-01-13"), "end-date": date("2008-07-23") } ] }
-{ "id": 11221033, "id-copy": 11221033, "alias": "Vernon", "name": "VernonLear", "user-since": datetime("2006-04-19T13:02:26.000Z"), "user-since-copy": datetime("2006-04-19T13:02:26.000Z"), "friend-ids": {{ 45628776, 31762296, 22963223, 10079920, 20931037, 41768759, 25910794, 41882156, 36691498, 1652094, 25804751, 35757270, 40057670, 37961622, 7430384, 1498630, 7636920, 17109852, 12569850, 47366298, 22902730, 5889994, 21003934, 1929823 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2000-04-18") } ] }
-{ "id": 11223157, "id-copy": 11223157, "alias": "Lavina", "name": "LavinaPeters", "user-since": datetime("2007-11-08T11:13:48.000Z"), "user-since-copy": datetime("2007-11-08T11:13:48.000Z"), "friend-ids": {{ 45286302 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-01-13") } ] }
-{ "id": 11259028, "id-copy": 11259028, "alias": "Linsay", "name": "LinsayBranson", "user-since": datetime("2011-04-28T08:49:14.000Z"), "user-since-copy": datetime("2011-04-28T08:49:14.000Z"), "friend-ids": {{ 24222662, 814967, 16722114, 24161306, 31611, 2964110, 4912379 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2006-05-18"), "end-date": date("2006-12-16") } ] }
-{ "id": 11273239, "id-copy": 11273239, "alias": "Alanis", "name": "AlanisNeely", "user-since": datetime("2009-04-11T16:49:56.000Z"), "user-since-copy": datetime("2009-04-11T16:49:56.000Z"), "friend-ids": {{ 16788046, 3222185, 46272663, 16782006, 29597609, 9709951, 37694695, 39662749, 18430270, 38598018, 40033174, 34984089, 8435528, 2669100, 18469173, 25201258, 29975180, 16379939, 24603, 2573554, 16344157, 16880724, 2437581 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-10-01"), "end-date": date("2006-08-24") } ] }
-{ "id": 11280553, "id-copy": 11280553, "alias": "Wendy", "name": "WendyClarke", "user-since": datetime("2009-08-28T16:53:37.000Z"), "user-since-copy": datetime("2009-08-28T16:53:37.000Z"), "friend-ids": {{ 10802559, 42649709, 8824750, 19241403, 43339000, 23865070, 9842110, 7051904, 39440876, 16961992 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2004-11-15"), "end-date": date("2005-01-15") } ] }
-{ "id": 11287327, "id-copy": 11287327, "alias": "Vito", "name": "VitoMoffat", "user-since": datetime("2008-02-08T03:16:42.000Z"), "user-since-copy": datetime("2008-02-08T03:16:42.000Z"), "friend-ids": {{ 36850894, 16346016, 4072987, 36112362, 13277841, 24976604, 20216096, 36253616, 13624540, 39256929, 8411929, 13545093, 27563972, 4306316, 9819682, 21998450, 16647991, 1987261 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2001-07-08"), "end-date": date("2005-04-23") } ] }
-{ "id": 11290987, "id-copy": 11290987, "alias": "Ilana", "name": "IlanaTedrow", "user-since": datetime("2009-03-03T00:10:34.000Z"), "user-since-copy": datetime("2009-03-03T00:10:34.000Z"), "friend-ids": {{ 20902982, 27972021, 22354642, 32382609, 18711912, 17070293 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2005-11-28"), "end-date": date("2009-09-17") } ] }
-{ "id": 11293477, "id-copy": 11293477, "alias": "Tamzen", "name": "TamzenWheeler", "user-since": datetime("2006-02-25T23:55:58.000Z"), "user-since-copy": datetime("2006-02-25T23:55:58.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-05-19"), "end-date": date("2011-03-06") } ] }
-{ "id": 11297359, "id-copy": 11297359, "alias": "Perry", "name": "PerryLowe", "user-since": datetime("2005-12-28T02:16:57.000Z"), "user-since-copy": datetime("2005-12-28T02:16:57.000Z"), "friend-ids": {{ 33439767 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2007-08-11"), "end-date": date("2009-05-16") } ] }
-{ "id": 11307946, "id-copy": 11307946, "alias": "Helga", "name": "HelgaStough", "user-since": datetime("2007-01-12T21:50:11.000Z"), "user-since-copy": datetime("2007-01-12T21:50:11.000Z"), "friend-ids": {{ 22768365 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2007-01-04"), "end-date": date("2009-06-25") } ] }
-{ "id": 11318098, "id-copy": 11318098, "alias": "Lucilla", "name": "LucillaSteele", "user-since": datetime("2006-05-02T12:10:51.000Z"), "user-since-copy": datetime("2006-05-02T12:10:51.000Z"), "friend-ids": {{ 43202249, 11116520, 19404968, 23494384, 41664359, 2459832, 21895811, 29849475, 32963400, 24381723, 46790616, 10343240, 43849340, 16769526, 26104853 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2009-10-09") } ] }
-{ "id": 11318329, "id-copy": 11318329, "alias": "April", "name": "AprilSurrency", "user-since": datetime("2008-09-02T21:07:03.000Z"), "user-since-copy": datetime("2008-09-02T21:07:03.000Z"), "friend-ids": {{ 8646916, 27873471, 41336682, 42549624, 39851926, 29548550, 31209458, 40169445, 27695329, 20395537, 10311481, 47078664, 32368262, 6850643, 26890752 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2009-12-11") } ] }
-{ "id": 11327731, "id-copy": 11327731, "alias": "Duncan", "name": "DuncanPennington", "user-since": datetime("2007-09-08T05:38:28.000Z"), "user-since-copy": datetime("2007-09-08T05:38:28.000Z"), "friend-ids": {{ 7591038, 8046115, 16606742, 39494564, 32760725, 39036737, 9937167, 38968828, 32536611 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2003-12-06") } ] }
-{ "id": 11330215, "id-copy": 11330215, "alias": "Tilly", "name": "TillyMckinnon", "user-since": datetime("2011-04-13T10:13:13.000Z"), "user-since-copy": datetime("2011-04-13T10:13:13.000Z"), "friend-ids": {{ 5559510, 31907101, 45791333, 35002065, 1302921, 37193818, 32812039, 41322357, 20631502 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2001-03-22"), "end-date": date("2008-08-22") } ] }
-{ "id": 11333794, "id-copy": 11333794, "alias": "Yung", "name": "YungNash", "user-since": datetime("2010-06-08T17:32:35.000Z"), "user-since-copy": datetime("2010-06-08T17:32:35.000Z"), "friend-ids": {{ 11329358, 14452899, 15459758, 31785934, 15405998, 17431717, 36883854, 1230831, 17690420, 45243495, 31580409, 15264731, 10067263, 20381783, 41240146, 2883831, 29492394, 38409147, 35853447, 26151247 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-06-24"), "end-date": date("2010-03-23") } ] }
-{ "id": 11348449, "id-copy": 11348449, "alias": "Domitila", "name": "DomitilaPolson", "user-since": datetime("2009-09-24T21:31:17.000Z"), "user-since-copy": datetime("2009-09-24T21:31:17.000Z"), "friend-ids": {{ 46755392, 24913792, 47792230, 2451253, 10548653, 3083052, 20700516, 15133622, 17284439, 40871072, 6444103, 44749243, 45289097, 19631062, 8873017, 6262067, 4742977, 672148, 19303779 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2005-06-03") } ] }
-{ "id": 11355979, "id-copy": 11355979, "alias": "Sal", "name": "SalChapman", "user-since": datetime("2012-07-23T17:03:04.000Z"), "user-since-copy": datetime("2012-07-23T17:03:04.000Z"), "friend-ids": {{ 4959799, 33919735, 33624568, 9885012, 16788595, 39510500, 34856818, 22167281, 44317359, 45181449, 43901851, 42402339, 9573000, 16655168 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2006-12-10") } ] }
-{ "id": 11364871, "id-copy": 11364871, "alias": "Darrell", "name": "DarrellTaggart", "user-since": datetime("2007-02-14T07:06:21.000Z"), "user-since-copy": datetime("2007-02-14T07:06:21.000Z"), "friend-ids": {{ 42942141, 33727432, 32050372, 39330410, 38031970, 18321427, 4533038, 45054607, 34474798, 29859123, 17215101, 24811589, 12250229, 4712867, 23411515, 10287620, 37707941 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2006-11-26"), "end-date": date("2007-02-18") } ] }
-{ "id": 11366056, "id-copy": 11366056, "alias": "Devin", "name": "DevinUlery", "user-since": datetime("2011-05-03T13:27:51.000Z"), "user-since-copy": datetime("2011-05-03T13:27:51.000Z"), "friend-ids": {{ 25443767, 42385070, 31515075, 31340661, 25371541, 34378389, 40381786, 23698797, 40141450, 12814851, 41414503, 39733660, 27910438, 44106204, 18806338, 37909692, 12502759, 4270087, 5110443, 14347603, 19313129, 8826229 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2001-12-15") } ] }
-{ "id": 11370337, "id-copy": 11370337, "alias": "Devin", "name": "DevinWatson", "user-since": datetime("2009-07-19T11:47:07.000Z"), "user-since-copy": datetime("2009-07-19T11:47:07.000Z"), "friend-ids": {{ 25117468, 31957773, 46217915, 26169035, 34203342, 32134285, 10572760, 10974016, 33771064, 4177645, 4910095, 18301833, 15264956, 5806057, 37899843, 35459189, 4391801, 34940818 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2008-06-19") } ] }
-{ "id": 11390830, "id-copy": 11390830, "alias": "Luciano", "name": "LucianoHooker", "user-since": datetime("2006-08-16T08:17:56.000Z"), "user-since-copy": datetime("2006-08-16T08:17:56.000Z"), "friend-ids": {{ 42206490, 5533465, 32480435, 18058343 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2004-02-19") } ] }
-{ "id": 11400016, "id-copy": 11400016, "alias": "Beaumont", "name": "BeaumontMiller", "user-since": datetime("2008-05-12T07:13:22.000Z"), "user-since-copy": datetime("2008-05-12T07:13:22.000Z"), "friend-ids": {{ 41935126, 36767417, 10582797, 47501456, 43527117, 2821865, 27905409, 13531461, 16278289, 9565333, 15686197, 15195167, 29350985, 8804024, 31606110, 44124513, 15106563, 26509959, 47480296, 13623445, 17378703, 33568332, 19922072, 12746355 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2002-03-04") } ] }
-{ "id": 11416066, "id-copy": 11416066, "alias": "Janna", "name": "JannaBowchiew", "user-since": datetime("2010-12-06T10:53:56.000Z"), "user-since-copy": datetime("2010-12-06T10:53:56.000Z"), "friend-ids": {{ 43816151, 22032304, 27239988, 23813127, 34936097, 8817657, 39872787, 27628236, 38333824, 40879066 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-04-19"), "end-date": date("2008-01-09") } ] }
-{ "id": 11417764, "id-copy": 11417764, "alias": "Maren", "name": "MarenDickson", "user-since": datetime("2006-07-20T06:36:52.000Z"), "user-since-copy": datetime("2006-07-20T06:36:52.000Z"), "friend-ids": {{ 14573904, 11946003, 35291176, 25103717, 30010131, 886854, 46625000, 28533752, 46506784, 15300620, 40647607, 10249516, 27751123, 3883546, 41772148, 26655932, 39026036, 4416966, 15070564, 7052224, 10264392, 13650303, 30752174 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2012-08-26"), "end-date": date("2012-08-29") } ] }
-{ "id": 11425216, "id-copy": 11425216, "alias": "Levi", "name": "LeviEiford", "user-since": datetime("2010-04-10T23:37:26.000Z"), "user-since-copy": datetime("2010-04-10T23:37:26.000Z"), "friend-ids": {{ 39348801, 15029457, 33995161, 27782571, 16712478, 28987111 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-08-12") } ] }
-{ "id": 11426248, "id-copy": 11426248, "alias": "Chryssa", "name": "ChryssaHincken", "user-since": datetime("2005-06-16T01:11:36.000Z"), "user-since-copy": datetime("2005-06-16T01:11:36.000Z"), "friend-ids": {{ 47119545 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2003-11-20"), "end-date": date("2003-10-07") } ] }
-{ "id": 11441509, "id-copy": 11441509, "alias": "Franklyn", "name": "FranklynZimmer", "user-since": datetime("2012-03-22T13:12:29.000Z"), "user-since-copy": datetime("2012-03-22T13:12:29.000Z"), "friend-ids": {{ 12883110, 5637339, 42139368, 25533619, 19998291, 4231212, 40792266, 9689761, 7591603, 29088602, 40962884, 9432997, 29850101, 47563888, 10384431, 30557751, 9141240, 45176888, 40987369, 42808497, 37891546, 8520042, 12875368, 39706341 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2008-06-09") } ] }
-{ "id": 11447332, "id-copy": 11447332, "alias": "Sherisse", "name": "SherisseMaugham", "user-since": datetime("2012-02-09T14:21:08.000Z"), "user-since-copy": datetime("2012-02-09T14:21:08.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-09-16") } ] }
-{ "id": 11452525, "id-copy": 11452525, "alias": "Suzanna", "name": "SuzannaOlphert", "user-since": datetime("2005-10-22T04:41:20.000Z"), "user-since-copy": datetime("2005-10-22T04:41:20.000Z"), "friend-ids": {{ 44250347, 21517625, 10831891, 23365285, 2000581, 43387385, 40167252, 25288275, 6768341, 36116792, 10670805 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2001-10-21"), "end-date": date("2005-03-11") } ] }
-{ "id": 11456404, "id-copy": 11456404, "alias": "Lonny", "name": "LonnyUllman", "user-since": datetime("2008-10-19T03:05:07.000Z"), "user-since-copy": datetime("2008-10-19T03:05:07.000Z"), "friend-ids": {{ 30675414, 44654756, 8273748, 12998719, 20082930 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2009-11-02"), "end-date": date("2011-05-11") } ] }
-{ "id": 11458594, "id-copy": 11458594, "alias": "Rosaline", "name": "RosalineHawker", "user-since": datetime("2006-06-07T01:36:07.000Z"), "user-since-copy": datetime("2006-06-07T01:36:07.000Z"), "friend-ids": {{ 13674953, 43755185, 20151836, 40023637, 35564429, 45196295, 33392303, 2080473, 6786170, 42815553, 10811200, 5050190, 20987923, 32613675 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2002-06-05") } ] }
-{ "id": 11476339, "id-copy": 11476339, "alias": "Hopkin", "name": "HopkinNicholas", "user-since": datetime("2008-09-23T20:48:07.000Z"), "user-since-copy": datetime("2008-09-23T20:48:07.000Z"), "friend-ids": {{ 30021024, 29046949, 8412580, 10700657, 15739611, 36768609 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2004-01-02") } ] }
-{ "id": 11494930, "id-copy": 11494930, "alias": "Eleanor", "name": "EleanorAnderson", "user-since": datetime("2008-09-01T04:27:31.000Z"), "user-since-copy": datetime("2008-09-01T04:27:31.000Z"), "friend-ids": {{ 46834294, 32081711 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2008-01-19") } ] }
-{ "id": 11506045, "id-copy": 11506045, "alias": "Marci", "name": "MarciSaltser", "user-since": datetime("2011-08-05T00:36:14.000Z"), "user-since-copy": datetime("2011-08-05T00:36:14.000Z"), "friend-ids": {{ 44810951, 11599851, 4960763, 13454104, 22872317, 44594135, 15792938 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2002-06-22"), "end-date": date("2009-08-20") } ] }
-{ "id": 11507149, "id-copy": 11507149, "alias": "Kendal", "name": "KendalCourtney", "user-since": datetime("2006-06-22T04:28:09.000Z"), "user-since-copy": datetime("2006-06-22T04:28:09.000Z"), "friend-ids": {{ 9084267, 26163683, 15271756, 4229254, 5439809, 23992890, 23144677, 26584955, 29430424, 15196312, 19993838, 3665259, 15861241, 15197583, 15693177 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2010-08-06"), "end-date": date("2011-04-21") } ] }
-{ "id": 11515915, "id-copy": 11515915, "alias": "Hunter", "name": "HunterBash", "user-since": datetime("2011-03-05T16:16:17.000Z"), "user-since-copy": datetime("2011-03-05T16:16:17.000Z"), "friend-ids": {{ 14847122, 46314922, 14414318, 46374290, 45050391, 22617753 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2004-01-20") } ] }
-{ "id": 11525575, "id-copy": 11525575, "alias": "Zack", "name": "ZackMills", "user-since": datetime("2007-10-15T20:53:30.000Z"), "user-since-copy": datetime("2007-10-15T20:53:30.000Z"), "friend-ids": {{ 11119738, 47490530, 18951399, 24413247, 4019030, 39064308, 43279140, 11316225, 15383674, 40613636, 4793869, 21591307, 23561981, 3763992, 32892218, 34334911, 40693733 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2012-05-25"), "end-date": date("2012-07-09") } ] }
-{ "id": 11529364, "id-copy": 11529364, "alias": "Rufus", "name": "RufusGreen", "user-since": datetime("2009-04-14T15:51:24.000Z"), "user-since-copy": datetime("2009-04-14T15:51:24.000Z"), "friend-ids": {{ 5011595 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2000-09-25"), "end-date": date("2004-08-22") } ] }
-{ "id": 11536582, "id-copy": 11536582, "alias": "Deon", "name": "DeonBickerson", "user-since": datetime("2007-05-18T18:12:00.000Z"), "user-since-copy": datetime("2007-05-18T18:12:00.000Z"), "friend-ids": {{ 2848304, 6359671, 29695732, 42414044, 3277185, 17642866, 47064497, 32240400, 43486181, 5049864, 22831246, 9259974, 17502793, 29955647, 6928887, 19609966 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2009-01-20"), "end-date": date("2009-03-12") } ] }
-{ "id": 11547586, "id-copy": 11547586, "alias": "Rosanne", "name": "RosanneWatkins", "user-since": datetime("2008-03-02T16:07:45.000Z"), "user-since-copy": datetime("2008-03-02T16:07:45.000Z"), "friend-ids": {{ 47389452, 44553302, 30722503, 3892313, 9603884, 12058710, 18459884, 23971280, 39791340, 25400946, 25149383, 8391991, 6548649, 20662585, 34505551, 8352025 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2005-05-17") } ] }
-{ "id": 11551078, "id-copy": 11551078, "alias": "Percy", "name": "PercyStocker", "user-since": datetime("2012-01-12T15:14:02.000Z"), "user-since-copy": datetime("2012-01-12T15:14:02.000Z"), "friend-ids": {{ 8927010, 25565873, 1309019, 9736505, 27953053, 6619625, 45562540, 32022492, 1535156, 11343220, 40057278, 5452463, 36005348, 35072612, 31954888 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2004-06-01"), "end-date": date("2010-03-09") } ] }
-{ "id": 11562148, "id-copy": 11562148, "alias": "Rexana", "name": "RexanaStange", "user-since": datetime("2012-08-13T20:11:05.000Z"), "user-since-copy": datetime("2012-08-13T20:11:05.000Z"), "friend-ids": {{ 22418981, 44892347, 43890424, 38530948, 33178064 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2004-11-21"), "end-date": date("2007-11-01") } ] }
-{ "id": 11582299, "id-copy": 11582299, "alias": "Seward", "name": "SewardReddish", "user-since": datetime("2007-11-07T11:10:00.000Z"), "user-since-copy": datetime("2007-11-07T11:10:00.000Z"), "friend-ids": {{ 14793773, 24447668, 30727802, 4757816, 26139324, 4433524, 15974482 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2012-02-10") } ] }
-{ "id": 11596522, "id-copy": 11596522, "alias": "Gena", "name": "GenaTurzanski", "user-since": datetime("2012-06-22T18:42:25.000Z"), "user-since-copy": datetime("2012-06-22T18:42:25.000Z"), "friend-ids": {{ 22525625, 22327219, 18520174, 38679685, 16561552, 1999972, 8066310, 24245231, 11682156, 31330371, 38780021, 46833789, 6710024, 38963740, 38984150, 33451484, 19022059, 36880540, 40003274 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2002-09-10") } ] }
-{ "id": 11619817, "id-copy": 11619817, "alias": "Conor", "name": "ConorIsaman", "user-since": datetime("2007-07-19T03:08:58.000Z"), "user-since-copy": datetime("2007-07-19T03:08:58.000Z"), "friend-ids": {{ 3118516, 11993690, 44936801, 20826732, 45978958, 5214526, 29651996, 39212065, 47935248, 13306157, 33084407, 537249, 42089040, 7553609, 42024531, 23482433, 45497814, 26865252, 42135224, 41353574, 28567135, 7898064 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2002-04-26") } ] }
-{ "id": 11626564, "id-copy": 11626564, "alias": "Gia", "name": "GiaNehling", "user-since": datetime("2007-05-04T02:40:35.000Z"), "user-since-copy": datetime("2007-05-04T02:40:35.000Z"), "friend-ids": {{ 14435544, 22982758, 14548448, 20359010, 43749230, 6484290, 43513351, 3652065, 1851524, 15523948, 1941233, 47031188, 12649571, 42789428, 10950757, 18325469, 24986924, 39948729, 29738829, 268135, 32952373, 29859037 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2007-06-13"), "end-date": date("2008-07-06") } ] }
-{ "id": 11626678, "id-copy": 11626678, "alias": "Reed", "name": "ReedHaile", "user-since": datetime("2011-05-28T09:52:04.000Z"), "user-since-copy": datetime("2011-05-28T09:52:04.000Z"), "friend-ids": {{ 38955792, 36648350, 7510300, 36168809, 41493759, 45265187, 1653351, 44881482, 44038304 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2012-03-08"), "end-date": date("2012-05-08") } ] }
-{ "id": 11627800, "id-copy": 11627800, "alias": "Andrina", "name": "AndrinaOrbell", "user-since": datetime("2005-01-07T13:18:15.000Z"), "user-since-copy": datetime("2005-01-07T13:18:15.000Z"), "friend-ids": {{ 14378125 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2001-07-27"), "end-date": date("2009-01-26") } ] }
-{ "id": 11630158, "id-copy": 11630158, "alias": "Jewel", "name": "JewelPrechtl", "user-since": datetime("2008-09-24T10:05:42.000Z"), "user-since-copy": datetime("2008-09-24T10:05:42.000Z"), "friend-ids": {{ 17110258, 26859370, 7070027, 19698792, 10087924, 31999744, 35694569, 10315290, 15006946, 25258889, 8036893, 20721778, 31250890, 31525573 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2002-10-09") } ] }
-{ "id": 11638618, "id-copy": 11638618, "alias": "Garfield", "name": "GarfieldHardie", "user-since": datetime("2007-07-05T04:44:27.000Z"), "user-since-copy": datetime("2007-07-05T04:44:27.000Z"), "friend-ids": {{ 47307628, 3109848, 30936899, 7173119, 33551634, 24239136, 11619168, 633835, 34791947, 12052833, 19798108, 3426648, 395456, 18555868, 18509839, 8340275, 14943912, 42330581, 313099, 25632353, 27912788, 20281899, 8961605, 13625222 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2001-02-24") } ] }
-{ "id": 11666128, "id-copy": 11666128, "alias": "Mathilda", "name": "MathildaBurris", "user-since": datetime("2006-01-04T14:30:09.000Z"), "user-since-copy": datetime("2006-01-04T14:30:09.000Z"), "friend-ids": {{ 21229678, 40152290, 2867638, 27694777, 34054129, 47727334, 39805693, 9084777, 37744206, 47011794, 2190990, 19109454 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2007-09-14"), "end-date": date("2007-03-17") } ] }
-{ "id": 11675221, "id-copy": 11675221, "alias": "Calanthe", "name": "CalantheGearhart", "user-since": datetime("2007-06-08T02:44:20.000Z"), "user-since-copy": datetime("2007-06-08T02:44:20.000Z"), "friend-ids": {{ 19185575 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2010-05-21") } ] }
-{ "id": 11693350, "id-copy": 11693350, "alias": "Crystal", "name": "CrystalDickinson", "user-since": datetime("2007-02-08T08:05:12.000Z"), "user-since-copy": datetime("2007-02-08T08:05:12.000Z"), "friend-ids": {{ 32246301, 35277320, 38987334, 3391139, 30437594, 35314588, 32659406, 19055708, 5245289, 1155014, 9266846, 20085529, 27878886, 25128707, 46223557, 16459237, 41315912, 26681594 }}, "employment": [ { "organization-name": "Strongtone", "start-date": date("2011-07-03"), "end-date": date("2011-08-05") } ] }
-{ "id": 11694928, "id-copy": 11694928, "alias": "Anne", "name": "AnnePritchard", "user-since": datetime("2005-05-25T23:02:45.000Z"), "user-since-copy": datetime("2005-05-25T23:02:45.000Z"), "friend-ids": {{ 4000537, 32410978, 2682612, 1214946, 38250943, 36272447, 14182545, 27782322, 2714608, 38315875 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2011-02-22"), "end-date": date("2011-11-07") } ] }
-{ "id": 11695309, "id-copy": 11695309, "alias": "Petula", "name": "PetulaTanner", "user-since": datetime("2011-12-23T13:29:44.000Z"), "user-since-copy": datetime("2011-12-23T13:29:44.000Z"), "friend-ids": {{ 39411346, 33118908, 44553603 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2003-02-26"), "end-date": date("2007-11-12") } ] }
-{ "id": 11748019, "id-copy": 11748019, "alias": "Malinda", "name": "MalindaMoberly", "user-since": datetime("2005-06-21T22:34:38.000Z"), "user-since-copy": datetime("2005-06-21T22:34:38.000Z"), "friend-ids": {{ 46792750, 47197275, 45940765, 43931611, 33201251, 32508732, 23681521, 35069089, 43652710, 22676488, 5098654, 29592897, 18671070, 40200423 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2004-08-18") } ] }
-{ "id": 11779591, "id-copy": 11779591, "alias": "Galina", "name": "GalinaRoberts", "user-since": datetime("2007-03-18T12:09:38.000Z"), "user-since-copy": datetime("2007-03-18T12:09:38.000Z"), "friend-ids": {{ 16134690, 41543844 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2010-04-17") } ] }
-{ "id": 11781745, "id-copy": 11781745, "alias": "Merv", "name": "MervStocker", "user-since": datetime("2008-10-15T03:41:54.000Z"), "user-since-copy": datetime("2008-10-15T03:41:54.000Z"), "friend-ids": {{ 26394519, 2599602, 40237077, 43817129, 30392481, 43051494, 36128635, 35974184, 37237292, 7775912, 11569464, 9112021, 26837692, 11548106, 29331601, 11126182, 18076463, 33866145, 22408972, 42318835, 47199541, 26807788 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2005-01-15"), "end-date": date("2008-02-18") } ] }
-{ "id": 11782354, "id-copy": 11782354, "alias": "Glynda", "name": "GlyndaEnderly", "user-since": datetime("2007-11-25T06:01:45.000Z"), "user-since-copy": datetime("2007-11-25T06:01:45.000Z"), "friend-ids": {{ 16202981, 24035766, 10175614, 27353200, 26183740, 6084065, 31664832, 22446721, 2792685, 37521374, 1999182, 12494503, 18087992, 44433851 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2004-06-10") } ] }
-{ "id": 11788345, "id-copy": 11788345, "alias": "Mindy", "name": "MindyRockwell", "user-since": datetime("2011-02-20T23:55:16.000Z"), "user-since-copy": datetime("2011-02-20T23:55:16.000Z"), "friend-ids": {{ 7821092, 24614722, 27718237, 19686343, 43916267, 7882804, 34422272, 46273261, 658009, 42620170, 36177155, 3340224, 27157340, 20438623, 19694381, 15643415, 43465380, 17719224, 37073374, 42060457, 29532671, 3781069, 26121650 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2011-05-11") } ] }
-{ "id": 11793622, "id-copy": 11793622, "alias": "Leonard", "name": "LeonardAlice", "user-since": datetime("2011-03-02T21:42:07.000Z"), "user-since-copy": datetime("2011-03-02T21:42:07.000Z"), "friend-ids": {{ 38648452, 2302677, 713863, 2484976, 20706899, 6649310, 9952945, 1293945, 23188221, 43521816, 2398744, 28382427, 45933146, 27717079, 12894240, 8077643, 38945982, 12658937, 36047491, 42431984, 43626155 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2001-02-12"), "end-date": date("2001-06-02") } ] }
-{ "id": 11811079, "id-copy": 11811079, "alias": "Kenelm", "name": "KenelmKellogg", "user-since": datetime("2006-05-14T04:13:36.000Z"), "user-since-copy": datetime("2006-05-14T04:13:36.000Z"), "friend-ids": {{ 28287762, 45591894, 12026636, 34381293, 17018521, 37239852, 5735876, 8145944, 34171842, 32986088, 16537938, 20530369, 35161854, 1076550, 26081966, 35666231 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-02-03") } ] }
-{ "id": 11822506, "id-copy": 11822506, "alias": "Jerrold", "name": "JerroldEwing", "user-since": datetime("2010-08-27T22:34:36.000Z"), "user-since-copy": datetime("2010-08-27T22:34:36.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2007-03-21"), "end-date": date("2008-04-26") } ] }
-{ "id": 11839117, "id-copy": 11839117, "alias": "Kyra", "name": "KyraMcdonald", "user-since": datetime("2010-07-08T20:46:49.000Z"), "user-since-copy": datetime("2010-07-08T20:46:49.000Z"), "friend-ids": {{ 42933043, 41665211, 13075886, 36147059, 20127919, 31449381, 47427643, 24399833, 16541120, 38909218, 15609877, 46802599, 31772232, 46743670 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2005-06-08"), "end-date": date("2007-11-11") } ] }
-{ "id": 11862502, "id-copy": 11862502, "alias": "Innocent", "name": "InnocentWilliamson", "user-since": datetime("2005-06-09T18:44:51.000Z"), "user-since-copy": datetime("2005-06-09T18:44:51.000Z"), "friend-ids": {{ 14750408, 36287814, 21197416, 34246775, 18776860, 32777856, 46956112, 18578056, 13053407, 3282278, 29812571, 25299530, 47168979, 6027296, 10540009 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2010-05-20"), "end-date": date("2010-01-24") } ] }
-{ "id": 11872177, "id-copy": 11872177, "alias": "Lillie", "name": "LillieLineman", "user-since": datetime("2009-09-28T02:48:03.000Z"), "user-since-copy": datetime("2009-09-28T02:48:03.000Z"), "friend-ids": {{ 16078664, 22307944, 21464886, 40255882, 39090292, 32823112, 5748916, 46831442, 25498280, 268782, 22829744, 17001614 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2006-02-18") } ] }
-{ "id": 11878948, "id-copy": 11878948, "alias": "Corey", "name": "CoreyWarrick", "user-since": datetime("2005-05-28T15:18:23.000Z"), "user-since-copy": datetime("2005-05-28T15:18:23.000Z"), "friend-ids": {{ 17192577, 19646534, 44755348, 28653064, 30539369, 15001411, 11921646, 44450607, 33599896, 41984600, 2187246, 8785209, 28099595 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2010-12-07") } ] }
-{ "id": 11886709, "id-copy": 11886709, "alias": "Leigh", "name": "LeighBatten", "user-since": datetime("2005-06-18T21:25:13.000Z"), "user-since-copy": datetime("2005-06-18T21:25:13.000Z"), "friend-ids": {{ 161610, 3498914, 24173074, 33102324, 42213688, 44551300, 36373040, 30704767, 24224319, 5784194, 13092764, 38315503, 13246046, 2836280, 672136, 37021775 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2001-05-26"), "end-date": date("2001-05-11") } ] }
-{ "id": 11886856, "id-copy": 11886856, "alias": "Eldred", "name": "EldredArmstrong", "user-since": datetime("2012-02-20T10:08:40.000Z"), "user-since-copy": datetime("2012-02-20T10:08:40.000Z"), "friend-ids": {{ 5146204, 10549788, 40744824, 38277859 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-09-18") } ] }
-{ "id": 11888530, "id-copy": 11888530, "alias": "Louis", "name": "LouisRichards", "user-since": datetime("2011-10-26T02:27:49.000Z"), "user-since-copy": datetime("2011-10-26T02:27:49.000Z"), "friend-ids": {{ 40512993, 46289399 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2000-04-18"), "end-date": date("2002-08-03") } ] }
-{ "id": 11893462, "id-copy": 11893462, "alias": "Shonna", "name": "ShonnaDickson", "user-since": datetime("2007-06-12T09:36:50.000Z"), "user-since-copy": datetime("2007-06-12T09:36:50.000Z"), "friend-ids": {{ 30462288, 43630666, 35884473, 25217438, 3196051, 41844836, 8922622, 15388786, 33486563, 22739607, 42411271, 47936046, 8921955, 11314832, 13138669, 1057389, 45874085 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2010-06-18") } ] }
-{ "id": 11914129, "id-copy": 11914129, "alias": "Ebenezer", "name": "EbenezerMonahan", "user-since": datetime("2006-01-08T08:17:51.000Z"), "user-since-copy": datetime("2006-01-08T08:17:51.000Z"), "friend-ids": {{ 9692770 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2002-10-22"), "end-date": date("2005-07-17") } ] }
-{ "id": 11921524, "id-copy": 11921524, "alias": "Mickey", "name": "MickeySybilla", "user-since": datetime("2012-03-28T17:05:25.000Z"), "user-since-copy": datetime("2012-03-28T17:05:25.000Z"), "friend-ids": {{ 40813978, 14172552, 40702786, 929262, 2220334, 33077762, 20716547, 11400385, 21916926, 38422356, 13378381, 32362984, 8162369, 8965084, 37823302, 3542211, 29294304, 37672739, 28359647 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2007-09-27") } ] }
-{ "id": 11937787, "id-copy": 11937787, "alias": "Addison", "name": "AddisonEckert", "user-since": datetime("2007-04-26T01:06:38.000Z"), "user-since-copy": datetime("2007-04-26T01:06:38.000Z"), "friend-ids": {{ 6446414, 23134374, 38952228, 25368200, 47868440, 29231397, 15672064, 2482344, 22824732, 13563448, 43826877 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2009-10-09") } ] }
-{ "id": 11951800, "id-copy": 11951800, "alias": "Camron", "name": "CamronBrooks", "user-since": datetime("2006-03-05T19:32:03.000Z"), "user-since-copy": datetime("2006-03-05T19:32:03.000Z"), "friend-ids": {{ 39430755, 45789857, 5352132, 34490450, 39117503, 2233039, 16387184 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2006-12-26"), "end-date": date("2007-11-16") } ] }
-{ "id": 11953306, "id-copy": 11953306, "alias": "Teale", "name": "TealeHoltzer", "user-since": datetime("2007-02-14T21:50:54.000Z"), "user-since-copy": datetime("2007-02-14T21:50:54.000Z"), "friend-ids": {{ 30902622, 26223630, 46832466, 32585590, 34005386, 23371032, 25984545, 7502619 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2010-02-14"), "end-date": date("2011-07-08") } ] }
-{ "id": 11969527, "id-copy": 11969527, "alias": "Adrian", "name": "AdrianTedrow", "user-since": datetime("2012-02-13T21:27:48.000Z"), "user-since-copy": datetime("2012-02-13T21:27:48.000Z"), "friend-ids": {{ 36940614, 29564878 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2002-01-16") } ] }
-{ "id": 11978782, "id-copy": 11978782, "alias": "Louiza", "name": "LouizaLlora", "user-since": datetime("2012-06-24T06:19:05.000Z"), "user-since-copy": datetime("2012-06-24T06:19:05.000Z"), "friend-ids": {{ 36495107, 35125435, 30347420, 17703387, 40909002 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2008-05-25") } ] }
-{ "id": 11987626, "id-copy": 11987626, "alias": "Chassidy", "name": "ChassidyHector", "user-since": datetime("2008-07-23T16:16:55.000Z"), "user-since-copy": datetime("2008-07-23T16:16:55.000Z"), "friend-ids": {{ 29831103, 12411598, 20670552, 42569662 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2010-08-22") } ] }
-{ "id": 11989645, "id-copy": 11989645, "alias": "Weston", "name": "WestonPershing", "user-since": datetime("2010-04-02T17:25:31.000Z"), "user-since-copy": datetime("2010-04-02T17:25:31.000Z"), "friend-ids": {{ 11689127 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2006-03-27") } ] }
-{ "id": 11989660, "id-copy": 11989660, "alias": "Rolland", "name": "RollandGarneis", "user-since": datetime("2008-09-16T19:54:32.000Z"), "user-since-copy": datetime("2008-09-16T19:54:32.000Z"), "friend-ids": {{ 30959592, 6160903, 27316367, 6518756, 23008668, 36942525, 39489068, 8710310, 17726852, 72593, 15440937, 4901728, 28916846, 38257093, 28414859, 8857050 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-02-11") } ] }
-{ "id": 9001816, "id-copy": 9001816, "alias": "Concordia", "name": "ConcordiaThomlinson", "user-since": datetime("2006-04-13T03:30:17.000Z"), "user-since-copy": datetime("2006-04-13T03:30:17.000Z"), "friend-ids": {{ 31001079, 10620343, 29160614, 8991085, 45471665, 865015, 11592391, 33106281, 15448665, 29325047, 47814022, 4562661, 11895808, 41974900 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2002-03-25") } ] }
-{ "id": 9005248, "id-copy": 9005248, "alias": "Jervis", "name": "JervisWarrick", "user-since": datetime("2007-02-06T17:54:17.000Z"), "user-since-copy": datetime("2007-02-06T17:54:17.000Z"), "friend-ids": {{ 5038062, 15101135, 28136073, 10706469, 8706391, 10623870, 1759405, 37020186, 17173998, 14985805, 19308437, 43696985, 46650868, 25621415, 14252531, 44491166, 42536769, 33614525, 34665072, 640793 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2010-08-21") } ] }
-{ "id": 9039973, "id-copy": 9039973, "alias": "Desmond", "name": "DesmondRice", "user-since": datetime("2008-04-17T12:00:38.000Z"), "user-since-copy": datetime("2008-04-17T12:00:38.000Z"), "friend-ids": {{ 16128090, 28937536, 30905098, 25666304, 23272582, 29438991, 42040849, 42396891, 9345677, 9260055, 17415621, 31581557, 1249365, 20734436, 2341357, 36307325, 20347771, 23723655 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2002-10-24"), "end-date": date("2008-02-24") } ] }
-{ "id": 9050866, "id-copy": 9050866, "alias": "Jimmie", "name": "JimmieBicknell", "user-since": datetime("2007-02-15T16:39:19.000Z"), "user-since-copy": datetime("2007-02-15T16:39:19.000Z"), "friend-ids": {{ 17248854, 13830961, 10571254, 637235, 18219702, 4541511, 42876025, 19679892, 14009802, 15312402, 20914286, 41969971, 39807443, 5990836, 1594551, 25853135, 25021671, 21604624, 47574478 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2001-04-09") } ] }
-{ "id": 9056494, "id-copy": 9056494, "alias": "Alvena", "name": "AlvenaPearsall", "user-since": datetime("2005-08-09T08:50:25.000Z"), "user-since-copy": datetime("2005-08-09T08:50:25.000Z"), "friend-ids": {{ 26263956, 80589, 37669623, 32875186, 42026139, 22169384, 47224581, 25632957, 28392334, 42393204, 15028714, 554526 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2005-02-19") } ] }
-{ "id": 9074290, "id-copy": 9074290, "alias": "Riley", "name": "RileyBode", "user-since": datetime("2010-11-20T01:12:36.000Z"), "user-since-copy": datetime("2010-11-20T01:12:36.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2004-10-12") } ] }
-{ "id": 9099376, "id-copy": 9099376, "alias": "Tena", "name": "TenaKline", "user-since": datetime("2011-10-20T14:46:29.000Z"), "user-since-copy": datetime("2011-10-20T14:46:29.000Z"), "friend-ids": {{ 28615752, 16589994, 24896126, 32768352, 40921310, 22643822, 39206554, 45652466, 17237997, 44705249, 30599864, 17750741, 14758376, 4842744 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2000-03-18") } ] }
-{ "id": 9139057, "id-copy": 9139057, "alias": "Esther", "name": "EstherUllman", "user-since": datetime("2010-01-05T19:25:44.000Z"), "user-since-copy": datetime("2010-01-05T19:25:44.000Z"), "friend-ids": {{ 25401186, 25915246, 33727208, 17431690, 24541706, 19998503, 42399029, 30405906, 20023918, 9788811, 32513474, 14919034, 10073867, 9309154, 1423378, 37386209, 16346279, 45167618, 34716280, 29023237, 20639001, 332097, 28344544 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2003-09-05"), "end-date": date("2009-10-17") } ] }
-{ "id": 9158293, "id-copy": 9158293, "alias": "Cortney", "name": "CortneyPainter", "user-since": datetime("2006-03-15T09:03:09.000Z"), "user-since-copy": datetime("2006-03-15T09:03:09.000Z"), "friend-ids": {{ 42832801, 24287760, 37934712, 43376751, 24673433, 14168792, 46862345, 46736573, 21181723, 2094484, 30254710, 45439521, 26589024, 45746175, 13898656, 13470143, 9669892 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2011-06-13") } ] }
-{ "id": 9190501, "id-copy": 9190501, "alias": "Leonardo", "name": "LeonardoBarr", "user-since": datetime("2008-02-23T14:20:45.000Z"), "user-since-copy": datetime("2008-02-23T14:20:45.000Z"), "friend-ids": {{ 24193096, 44367993, 10307197, 20420512, 36000544, 45069724, 42621729, 10863302, 21701700, 7110735, 6226449, 3269792, 12797617, 19460642, 7357145, 27051982, 31847212, 28691920, 382743, 11602175, 1787538, 42283089, 19610964 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2001-06-25") } ] }
-{ "id": 9201610, "id-copy": 9201610, "alias": "Elaine", "name": "ElaineMcclymonds", "user-since": datetime("2008-04-13T17:06:35.000Z"), "user-since-copy": datetime("2008-04-13T17:06:35.000Z"), "friend-ids": {{ 18934024, 5114594, 25593808 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-08-28") } ] }
-{ "id": 9205834, "id-copy": 9205834, "alias": "Tristin", "name": "TristinWalker", "user-since": datetime("2012-04-25T01:08:05.000Z"), "user-since-copy": datetime("2012-04-25T01:08:05.000Z"), "friend-ids": {{ 2222398, 15073251, 16222879, 24405969, 32651599, 44500557, 31699173, 41724026, 1745441, 9674348, 29594086, 26580583, 42258300, 36027050, 3204087, 2147469, 36519580 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2007-02-09") } ] }
-{ "id": 9210847, "id-copy": 9210847, "alias": "Kristeen", "name": "KristeenShaffer", "user-since": datetime("2008-01-04T12:31:50.000Z"), "user-since-copy": datetime("2008-01-04T12:31:50.000Z"), "friend-ids": {{ 662954, 18313322, 10737685, 5498351, 24795605, 4497605, 45729062, 31007969, 16211490, 19408104, 5882137, 12084923, 14143383, 31263672, 32404691, 8973685, 32756191, 3822704 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2009-12-07"), "end-date": date("2010-02-08") } ] }
-{ "id": 9223375, "id-copy": 9223375, "alias": "Anne", "name": "AnneMoore", "user-since": datetime("2010-07-16T22:06:20.000Z"), "user-since-copy": datetime("2010-07-16T22:06:20.000Z"), "friend-ids": {{ 45553359, 40589681, 9461257, 39253068, 14447226, 37656564, 37047377, 34855985 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2011-04-25") } ] }
-{ "id": 9226960, "id-copy": 9226960, "alias": "Irish", "name": "IrishJohnson", "user-since": datetime("2009-09-07T21:02:01.000Z"), "user-since-copy": datetime("2009-09-07T21:02:01.000Z"), "friend-ids": {{ 4920892, 15681759, 19110917, 26620361, 34712468, 40890326, 20312413 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2009-11-11") } ] }
-{ "id": 9262768, "id-copy": 9262768, "alias": "Graham", "name": "GrahamHunt", "user-since": datetime("2009-03-19T13:15:02.000Z"), "user-since-copy": datetime("2009-03-19T13:15:02.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2012-04-23"), "end-date": date("2012-04-15") } ] }
-{ "id": 9271291, "id-copy": 9271291, "alias": "Kaitlynn", "name": "KaitlynnPycroft", "user-since": datetime("2010-10-09T11:30:12.000Z"), "user-since-copy": datetime("2010-10-09T11:30:12.000Z"), "friend-ids": {{ 38067939, 25732262, 17076819, 19477302, 29794559 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2006-09-04") } ] }
-{ "id": 9275620, "id-copy": 9275620, "alias": "Jackie", "name": "JackieRumbaugh", "user-since": datetime("2011-10-11T07:30:25.000Z"), "user-since-copy": datetime("2011-10-11T07:30:25.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2004-01-14") } ] }
-{ "id": 9288154, "id-copy": 9288154, "alias": "Lauren", "name": "LaurenGraff", "user-since": datetime("2005-12-28T07:21:17.000Z"), "user-since-copy": datetime("2005-12-28T07:21:17.000Z"), "friend-ids": {{ 38658043, 4029859, 43671010, 20184796, 23429992, 3744331, 39377881, 1336305, 33712064, 36443 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2009-04-06") } ] }
-{ "id": 9297361, "id-copy": 9297361, "alias": "Yasmine", "name": "YasmineBullard", "user-since": datetime("2006-07-11T23:54:23.000Z"), "user-since-copy": datetime("2006-07-11T23:54:23.000Z"), "friend-ids": {{ 27580636, 11448774, 32271178, 9627095, 11487349, 46595708 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2001-10-06"), "end-date": date("2003-03-05") } ] }
-{ "id": 9317395, "id-copy": 9317395, "alias": "Timothy", "name": "TimothyMays", "user-since": datetime("2007-05-23T15:42:26.000Z"), "user-since-copy": datetime("2007-05-23T15:42:26.000Z"), "friend-ids": {{ 38066468, 16126194, 20685050, 8542551, 36810930, 36333903, 31522960, 44908120, 45171970, 9212095, 16986466, 41689196, 22300874, 45983009, 30918582, 5896299, 2682406, 6649020, 33199300, 14523848 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2006-04-16"), "end-date": date("2008-02-21") } ] }
-{ "id": 9320062, "id-copy": 9320062, "alias": "Samantha", "name": "SamanthaTanner", "user-since": datetime("2010-06-25T14:13:49.000Z"), "user-since-copy": datetime("2010-06-25T14:13:49.000Z"), "friend-ids": {{ 19538026 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2009-02-24") } ] }
-{ "id": 9329746, "id-copy": 9329746, "alias": "Albert", "name": "AlbertZundel", "user-since": datetime("2005-11-01T23:41:02.000Z"), "user-since-copy": datetime("2005-11-01T23:41:02.000Z"), "friend-ids": {{ 44252308, 14483702, 27233282, 24263669, 35409140, 38591765, 42901786, 24502313, 6384822, 36359249, 36816246, 16578182, 530819, 29481837, 12698700, 6101521, 11990316, 35327955, 10435272 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2003-08-06"), "end-date": date("2010-09-22") } ] }
-{ "id": 9331075, "id-copy": 9331075, "alias": "Monday", "name": "MondayWarrick", "user-since": datetime("2012-01-13T06:13:30.000Z"), "user-since-copy": datetime("2012-01-13T06:13:30.000Z"), "friend-ids": {{ 27699724, 39094128, 11014820, 44605243, 20177679, 37579779, 35875781, 13713739, 8882475, 37427927, 28595578, 3788567, 31200715, 40590973, 7630783, 36856789, 22013865 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2000-04-08") } ] }
-{ "id": 9367306, "id-copy": 9367306, "alias": "Jacinth", "name": "JacinthBynum", "user-since": datetime("2012-03-08T11:26:04.000Z"), "user-since-copy": datetime("2012-03-08T11:26:04.000Z"), "friend-ids": {{ 35048012, 42620612, 39526901, 12673410, 16363143, 45509270, 47714729, 47902094, 12551745, 45510597, 31513255, 2848992, 16088751, 1953590, 32956014, 38607548, 15982103, 31161780, 7331812, 44977526, 15022020, 19905573 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2003-03-24") } ] }
-{ "id": 9373726, "id-copy": 9373726, "alias": "Joe", "name": "JoeRoche", "user-since": datetime("2005-07-09T16:42:53.000Z"), "user-since-copy": datetime("2005-07-09T16:42:53.000Z"), "friend-ids": {{ 16433644, 5532847, 743901, 2134179, 43053028, 36961668, 9731766, 45686582, 17084459, 27026683, 1687547, 6582422, 38798685, 9871595, 2677099, 42280963, 32191501, 4347234 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2009-09-16") } ] }
-{ "id": 9389254, "id-copy": 9389254, "alias": "Jon", "name": "JonShaw", "user-since": datetime("2006-12-10T11:28:23.000Z"), "user-since-copy": datetime("2006-12-10T11:28:23.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2010-07-24") } ] }
-{ "id": 9396193, "id-copy": 9396193, "alias": "Franklyn", "name": "FranklynVorrasi", "user-since": datetime("2007-06-27T09:38:03.000Z"), "user-since-copy": datetime("2007-06-27T09:38:03.000Z"), "friend-ids": {{ 12870114, 28811462, 19219273, 38745339, 22310708, 11419733, 21583164, 42276545, 1177024, 43617748, 11702666, 19332437, 1523883, 40265275, 41227772 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2001-03-13"), "end-date": date("2009-02-07") } ] }
-{ "id": 9408427, "id-copy": 9408427, "alias": "Matt", "name": "MattPritchard", "user-since": datetime("2008-10-02T15:31:39.000Z"), "user-since-copy": datetime("2008-10-02T15:31:39.000Z"), "friend-ids": {{ 3596345, 15476624, 33857894, 13004846, 29332890, 23638145, 43402648, 14337754, 3290802, 10537283, 9989868, 33400736, 43952799, 34128983, 3090230, 12591428, 15051691, 7239629, 10295253, 23448932, 30507945 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2012-02-05") } ] }
-{ "id": 9453925, "id-copy": 9453925, "alias": "Ritchie", "name": "RitchieJube", "user-since": datetime("2008-04-28T12:33:34.000Z"), "user-since-copy": datetime("2008-04-28T12:33:34.000Z"), "friend-ids": {{ 44327769, 45189889, 11098478, 41612069, 40647950, 638474, 21614810, 22273745, 6230791, 15120137, 18477729, 16895919, 5907839, 43993812, 31639138, 7966991, 11024409 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2012-07-22") } ] }
-{ "id": 9461770, "id-copy": 9461770, "alias": "Georgina", "name": "GeorginaPearson", "user-since": datetime("2005-02-04T09:47:21.000Z"), "user-since-copy": datetime("2005-02-04T09:47:21.000Z"), "friend-ids": {{ 26615251, 5874803, 5189465, 29564778, 1778424, 38706542, 38915757, 16819394, 3318129, 2166806, 30570432, 15192853, 4857015, 41673300, 23510020 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2003-06-06") } ] }
-{ "id": 9467614, "id-copy": 9467614, "alias": "Eloisa", "name": "EloisaEvans", "user-since": datetime("2012-01-20T01:00:51.000Z"), "user-since-copy": datetime("2012-01-20T01:00:51.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2000-11-03"), "end-date": date("2003-01-14") } ] }
-{ "id": 9478720, "id-copy": 9478720, "alias": "Angelia", "name": "AngeliaKettlewell", "user-since": datetime("2005-05-27T06:29:30.000Z"), "user-since-copy": datetime("2005-05-27T06:29:30.000Z"), "friend-ids": {{ 42556433, 20033025, 38112512, 19420757, 31822717, 7116081, 39544900, 19203395, 46787205, 32303456, 4509345, 45558040, 42616291, 6929369, 9272653, 37459048, 37113569, 38942369, 47741031, 46761451, 14163845 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2012-03-28"), "end-date": date("2012-03-04") } ] }
-{ "id": 9482569, "id-copy": 9482569, "alias": "Marty", "name": "MartyBurnett", "user-since": datetime("2006-03-21T10:10:40.000Z"), "user-since-copy": datetime("2006-03-21T10:10:40.000Z"), "friend-ids": {{ 5791578, 3884688, 7686005 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2009-02-01") } ] }
-{ "id": 9497698, "id-copy": 9497698, "alias": "Jenny", "name": "JennyBiery", "user-since": datetime("2007-07-24T17:20:06.000Z"), "user-since-copy": datetime("2007-07-24T17:20:06.000Z"), "friend-ids": {{ 37832227, 17148339, 38184683, 45775690, 17511050, 1866913, 30631091, 5996302, 3796747, 33135567, 5930972, 9509054, 44003369, 34299276, 16135297, 15435466, 42464299, 34961792, 47264306, 30734198, 26192613 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2007-10-02"), "end-date": date("2011-09-20") } ] }
-{ "id": 9502096, "id-copy": 9502096, "alias": "Hebe", "name": "HebeEndsley", "user-since": datetime("2012-08-08T18:55:28.000Z"), "user-since-copy": datetime("2012-08-08T18:55:28.000Z"), "friend-ids": {{ 34917916, 5530270, 12994124, 25113086, 28819142, 44228082 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2007-04-11") } ] }
-{ "id": 9503443, "id-copy": 9503443, "alias": "Ebenezer", "name": "EbenezerFulton", "user-since": datetime("2012-07-03T20:14:05.000Z"), "user-since-copy": datetime("2012-07-03T20:14:05.000Z"), "friend-ids": {{ 11155403, 7932344, 24822329, 19823943, 37496284 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2011-08-22") } ] }
-{ "id": 9510451, "id-copy": 9510451, "alias": "Chuck", "name": "ChuckFinck", "user-since": datetime("2011-09-10T08:27:31.000Z"), "user-since-copy": datetime("2011-09-10T08:27:31.000Z"), "friend-ids": {{ 5559039, 8997599, 8311284, 20478562, 13734713, 21511695, 30393493 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2001-10-12") } ] }
-{ "id": 9521683, "id-copy": 9521683, "alias": "Tennille", "name": "TennilleHamilton", "user-since": datetime("2009-04-21T20:56:25.000Z"), "user-since-copy": datetime("2009-04-21T20:56:25.000Z"), "friend-ids": {{ 32048407, 3619952, 41652292, 45570368, 31678290, 11241324 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2010-10-15") } ] }
-{ "id": 9549610, "id-copy": 9549610, "alias": "Blossom", "name": "BlossomGreif", "user-since": datetime("2010-05-03T21:08:56.000Z"), "user-since-copy": datetime("2010-05-03T21:08:56.000Z"), "friend-ids": {{ 47791115, 42952282 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2011-12-25"), "end-date": date("2011-11-27") } ] }
-{ "id": 9556570, "id-copy": 9556570, "alias": "Kassandra", "name": "KassandraKern", "user-since": datetime("2010-12-03T15:29:12.000Z"), "user-since-copy": datetime("2010-12-03T15:29:12.000Z"), "friend-ids": {{ 35944118, 3024691, 43927521, 44121317, 29834404, 18626717, 47095811, 38438153, 30557309, 37143411, 41634172, 23338449, 30455300, 12009022, 26366377, 36381324, 25084236, 36521163, 20063914, 11419154, 40243010, 9336807, 3544397, 20455720 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2005-02-12") } ] }
-{ "id": 9574261, "id-copy": 9574261, "alias": "Kalysta", "name": "KalystaBeedell", "user-since": datetime("2010-01-27T14:57:31.000Z"), "user-since-copy": datetime("2010-01-27T14:57:31.000Z"), "friend-ids": {{ 5811189, 22155580, 41736564, 27399656, 40013573, 28340467, 45690668, 16097604, 9655169, 44870593 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2009-12-16"), "end-date": date("2010-10-22") } ] }
-{ "id": 9575338, "id-copy": 9575338, "alias": "Isabell", "name": "IsabellWain", "user-since": datetime("2011-07-05T12:26:43.000Z"), "user-since-copy": datetime("2011-07-05T12:26:43.000Z"), "friend-ids": {{ 42651024, 15652966, 27390748, 19369775, 44130969, 45269514, 210916, 36228917, 31857984, 11676544, 42752689, 14021599, 31749945, 9405328, 37567152, 17083209, 32654328, 39607403, 18699149, 37082017, 6059914, 881724 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2003-06-04") } ] }
-{ "id": 9577867, "id-copy": 9577867, "alias": "Lavette", "name": "LavetteSnyder", "user-since": datetime("2007-02-22T10:01:04.000Z"), "user-since-copy": datetime("2007-02-22T10:01:04.000Z"), "friend-ids": {{ 25749553, 31379974, 15118772, 38725424, 26760226, 8908746, 20299291, 20288328, 19659485, 22400738, 477700, 20253845, 12753420, 46016251, 29518581, 21898853, 19015599, 3455762, 19350275, 2630122 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2011-04-22") } ] }
-{ "id": 9598486, "id-copy": 9598486, "alias": "Grover", "name": "GroverNewbern", "user-since": datetime("2012-01-06T20:50:38.000Z"), "user-since-copy": datetime("2012-01-06T20:50:38.000Z"), "friend-ids": {{ 8389292, 25521744, 23387036, 38008541, 43673600, 23656679, 1401712, 39164079, 1810015, 20625744, 15651316, 23441546, 24572830, 19077921 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2009-07-28"), "end-date": date("2010-06-09") } ] }
-{ "id": 9599647, "id-copy": 9599647, "alias": "Alexandria", "name": "AlexandriaWade", "user-since": datetime("2012-06-25T06:48:48.000Z"), "user-since-copy": datetime("2012-06-25T06:48:48.000Z"), "friend-ids": {{ 20910866, 20843338, 8182424, 21070448, 43548111, 39370893, 26760127, 11135506 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2011-06-02") } ] }
-{ "id": 9635563, "id-copy": 9635563, "alias": "Tamsen", "name": "TamsenCowart", "user-since": datetime("2010-10-07T05:11:20.000Z"), "user-since-copy": datetime("2010-10-07T05:11:20.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2007-01-07") } ] }
-{ "id": 9636802, "id-copy": 9636802, "alias": "Gage", "name": "GageHair", "user-since": datetime("2011-01-23T22:31:49.000Z"), "user-since-copy": datetime("2011-01-23T22:31:49.000Z"), "friend-ids": {{ 46795684, 38195763, 25882078, 28871879, 5178144, 17683475, 43441471, 5427133, 13936915, 2608474, 9513798, 31041524, 557454, 22452168, 12948004, 16835098, 1151241, 37188687 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2000-05-02"), "end-date": date("2010-02-13") } ] }
-{ "id": 9638626, "id-copy": 9638626, "alias": "Hisako", "name": "HisakoEisaman", "user-since": datetime("2008-05-26T23:34:43.000Z"), "user-since-copy": datetime("2008-05-26T23:34:43.000Z"), "friend-ids": {{ 17773563, 18434504, 1082020, 40557107, 43294701, 1982610, 8259201, 47490886, 20044705, 35882471, 7297053, 17276976, 38660830, 36435103, 29511457, 3474864, 17100964, 23978369, 6260698, 17616437, 1617227, 18325960, 42613056 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2009-07-12") } ] }
-{ "id": 9690049, "id-copy": 9690049, "alias": "Ahmed", "name": "AhmedVinsant", "user-since": datetime("2009-12-24T23:10:10.000Z"), "user-since-copy": datetime("2009-12-24T23:10:10.000Z"), "friend-ids": {{ 9425379, 24773026, 47645199, 12718095, 32145472, 30931581, 11512330, 46898742, 26190870, 38985851, 40692118, 34327720, 47432207 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2002-05-26") } ] }
-{ "id": 9693988, "id-copy": 9693988, "alias": "Geordie", "name": "GeordieBunten", "user-since": datetime("2006-08-03T15:00:25.000Z"), "user-since-copy": datetime("2006-08-03T15:00:25.000Z"), "friend-ids": {{ 31987089, 15556815, 3656365, 35713356, 9573642, 38459850, 44400137, 44882118, 44921684, 47393814, 7869122, 35085016, 43725704, 17602789, 9966406, 20936803, 26425879, 41666932 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2007-01-20") } ] }
-{ "id": 9696160, "id-copy": 9696160, "alias": "Lawerence", "name": "LawerenceLudwig", "user-since": datetime("2005-09-04T07:08:01.000Z"), "user-since-copy": datetime("2005-09-04T07:08:01.000Z"), "friend-ids": {{ 33125788, 14719007, 35434564 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-02-02") } ] }
-{ "id": 9699673, "id-copy": 9699673, "alias": "Jim", "name": "JimPycroft", "user-since": datetime("2012-07-25T20:20:38.000Z"), "user-since-copy": datetime("2012-07-25T20:20:38.000Z"), "friend-ids": {{ 14858146, 47543880, 3186927, 38198580, 2365336, 5255886, 11178580, 41188272, 17623582, 6422949, 4405751, 12128017, 32409443, 38861849, 16511892, 24515731, 46665640, 40644816, 19341995, 44288533, 26148671 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2007-01-24"), "end-date": date("2009-12-16") } ] }
-{ "id": 9709663, "id-copy": 9709663, "alias": "Trevor", "name": "TrevorSell", "user-since": datetime("2008-08-28T18:18:54.000Z"), "user-since-copy": datetime("2008-08-28T18:18:54.000Z"), "friend-ids": {{ 13788189, 27667188, 588943, 1574745, 5763893, 19661124, 45630528, 47078471, 42976078, 32943975 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2007-07-04") } ] }
-{ "id": 9719995, "id-copy": 9719995, "alias": "Hazel", "name": "HazelKnopsnider", "user-since": datetime("2007-04-05T01:11:42.000Z"), "user-since-copy": datetime("2007-04-05T01:11:42.000Z"), "friend-ids": {{ 38515770, 23212874, 6000594, 27957554, 28093880, 3726628, 22800428, 42313894, 23190476, 18537188, 22083915, 43478674, 33364444, 19158958, 1590605, 36792931, 42057988, 33286729, 29580197, 25232028 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2008-09-28") } ] }
-{ "id": 9736855, "id-copy": 9736855, "alias": "Sudie", "name": "SudieAlbright", "user-since": datetime("2011-10-08T08:46:27.000Z"), "user-since-copy": datetime("2011-10-08T08:46:27.000Z"), "friend-ids": {{ 20506190, 13537252, 46211902, 4320089 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2004-12-07"), "end-date": date("2010-07-02") } ] }
-{ "id": 9752227, "id-copy": 9752227, "alias": "Audley", "name": "AudleyPeters", "user-since": datetime("2006-07-27T01:15:35.000Z"), "user-since-copy": datetime("2006-07-27T01:15:35.000Z"), "friend-ids": {{ 877448, 29611844, 2844046, 42493473, 28216181, 353847, 44172105, 36184409, 44010617 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2002-12-17") } ] }
-{ "id": 9760834, "id-copy": 9760834, "alias": "Lavette", "name": "LavettePirl", "user-since": datetime("2006-02-12T07:28:53.000Z"), "user-since-copy": datetime("2006-02-12T07:28:53.000Z"), "friend-ids": {{ 27450797, 36415787 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2002-09-20") } ] }
-{ "id": 9765517, "id-copy": 9765517, "alias": "Alexia", "name": "AlexiaTownsend", "user-since": datetime("2006-02-23T13:26:33.000Z"), "user-since-copy": datetime("2006-02-23T13:26:33.000Z"), "friend-ids": {{ 39892441, 43413199, 45070224, 46877180, 24247279, 26450737, 29111107, 46768934, 11833332, 25913646, 43063781 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2003-02-08") } ] }
-{ "id": 9774613, "id-copy": 9774613, "alias": "Kaycee", "name": "KayceeGeyer", "user-since": datetime("2008-12-19T06:09:36.000Z"), "user-since-copy": datetime("2008-12-19T06:09:36.000Z"), "friend-ids": {{ 35485847, 33668074, 21309976, 40428525, 40450508, 30804358, 1365381, 5197688, 37844952, 4076960, 28446817, 20696590, 23896488, 33454126, 21411087, 9300550, 12986775, 36731809, 47850175, 9503217, 22481614, 29556396, 15013896, 14407126 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2001-03-23"), "end-date": date("2003-01-16") } ] }
-{ "id": 9784687, "id-copy": 9784687, "alias": "Larrie", "name": "LarrieStroh", "user-since": datetime("2005-12-03T13:45:30.000Z"), "user-since-copy": datetime("2005-12-03T13:45:30.000Z"), "friend-ids": {{ 38055237, 43436653, 21194063, 30405058, 7754813, 14616686, 3434657, 24778389, 5653770, 8600235, 44560871, 4379727, 32140404, 35445864, 24133933, 21379278, 45626842, 25710375, 25970333, 16831917 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2006-09-18") } ] }
-{ "id": 9809977, "id-copy": 9809977, "alias": "Kassandra", "name": "KassandraHarding", "user-since": datetime("2007-05-01T06:22:22.000Z"), "user-since-copy": datetime("2007-05-01T06:22:22.000Z"), "friend-ids": {{ 29945374, 38811992, 41372042, 28714909, 16897620, 5020268, 24134801, 26310926, 32871167, 18787983, 47295432, 31873694, 36300817, 42779931, 27486692 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-08-26") } ] }
-{ "id": 9812020, "id-copy": 9812020, "alias": "Elias", "name": "EliasBuck", "user-since": datetime("2012-08-03T07:52:34.000Z"), "user-since-copy": datetime("2012-08-03T07:52:34.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2008-07-05"), "end-date": date("2008-12-18") } ] }
-{ "id": 9842389, "id-copy": 9842389, "alias": "Nicolas", "name": "NicolasHynes", "user-since": datetime("2005-08-10T23:35:18.000Z"), "user-since-copy": datetime("2005-08-10T23:35:18.000Z"), "friend-ids": {{ 40180500, 33396487, 26907885, 4321366, 10229201, 41118923 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2006-10-23"), "end-date": date("2010-03-11") } ] }
-{ "id": 9854788, "id-copy": 9854788, "alias": "Mathilda", "name": "MathildaVanleer", "user-since": datetime("2007-01-05T08:45:07.000Z"), "user-since-copy": datetime("2007-01-05T08:45:07.000Z"), "friend-ids": {{ 20510022, 1353061, 24801201, 11438611, 30281530, 15596343, 29404248, 2024925, 3425369, 18530400 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-07-22"), "end-date": date("2011-02-24") } ] }
-{ "id": 9882241, "id-copy": 9882241, "alias": "Dillon", "name": "DillonSimpson", "user-since": datetime("2006-03-20T13:21:16.000Z"), "user-since-copy": datetime("2006-03-20T13:21:16.000Z"), "friend-ids": {{ 22747996, 6266176, 22832223, 30880579, 35481343, 48005259, 381757, 27560756, 6053858, 42532723, 33355330, 40374460, 39019469, 35869327 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2007-06-13"), "end-date": date("2011-08-15") } ] }
-{ "id": 9883165, "id-copy": 9883165, "alias": "Dean", "name": "DeanKern", "user-since": datetime("2005-11-02T13:10:37.000Z"), "user-since-copy": datetime("2005-11-02T13:10:37.000Z"), "friend-ids": {{ 33343261, 27280204, 31345192, 723310, 11949431, 4787422, 28427922, 11974873, 24553234, 19067609, 12178905, 38171944, 26832701, 47422914, 47782561, 26391811, 28206950, 17135029, 37069726, 40613638, 11509775 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2003-02-16"), "end-date": date("2009-12-16") } ] }
-{ "id": 9885289, "id-copy": 9885289, "alias": "Kayla", "name": "KaylaDugger", "user-since": datetime("2007-10-20T12:55:38.000Z"), "user-since-copy": datetime("2007-10-20T12:55:38.000Z"), "friend-ids": {{ 1821427, 46609485, 4532131 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2009-02-15"), "end-date": date("2009-11-17") } ] }
-{ "id": 9890854, "id-copy": 9890854, "alias": "Linwood", "name": "LinwoodBrown", "user-since": datetime("2005-09-09T12:38:00.000Z"), "user-since-copy": datetime("2005-09-09T12:38:00.000Z"), "friend-ids": {{ 13728190, 31562633, 3437344, 13841675, 38528685 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-05-08"), "end-date": date("2009-08-26") } ] }
-{ "id": 9897094, "id-copy": 9897094, "alias": "Raynard", "name": "RaynardWade", "user-since": datetime("2010-05-12T19:44:55.000Z"), "user-since-copy": datetime("2010-05-12T19:44:55.000Z"), "friend-ids": {{ 21246472, 34504200, 43744110, 30518742, 1016046, 17644601, 47173648, 11643135, 22382871, 38535297, 17156487, 30328939, 14770807, 9365820, 36893585, 30122942, 37610936, 44304872 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-02-18") } ] }
-{ "id": 9910003, "id-copy": 9910003, "alias": "Arline", "name": "ArlineElinor", "user-since": datetime("2012-07-20T16:57:36.000Z"), "user-since-copy": datetime("2012-07-20T16:57:36.000Z"), "friend-ids": {{ 34121202, 19342891, 45323168, 17272278, 6471047, 3726738, 48003127, 32423724, 38588754, 44816854, 13688032, 12876442 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-07-19"), "end-date": date("2009-04-17") } ] }
-{ "id": 9934939, "id-copy": 9934939, "alias": "Camilla", "name": "CamillaRhinehart", "user-since": datetime("2008-12-06T10:44:45.000Z"), "user-since-copy": datetime("2008-12-06T10:44:45.000Z"), "friend-ids": {{ 17020237, 36188716, 32765819, 20068359, 23060675, 16692600 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-04-05") } ] }
-{ "id": 9951325, "id-copy": 9951325, "alias": "Sarah", "name": "SarahRockwell", "user-since": datetime("2009-08-25T01:56:51.000Z"), "user-since-copy": datetime("2009-08-25T01:56:51.000Z"), "friend-ids": {{ 14846488, 32939876, 43509116, 36687501, 6496360, 47346160, 20558288, 21828060 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2002-11-18") } ] }
-{ "id": 9955486, "id-copy": 9955486, "alias": "Jerrod", "name": "JerrodBeach", "user-since": datetime("2007-04-18T07:24:36.000Z"), "user-since-copy": datetime("2007-04-18T07:24:36.000Z"), "friend-ids": {{ 9760902, 36268051, 11373781, 42337286, 41818514, 20451257, 23673069, 14313303, 6548991, 34820597, 17346574, 46871090, 263833, 38179383, 14434022 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2003-09-01"), "end-date": date("2007-06-11") } ] }
-{ "id": 9958378, "id-copy": 9958378, "alias": "Floyd", "name": "FloydErrett", "user-since": datetime("2006-07-06T02:51:46.000Z"), "user-since-copy": datetime("2006-07-06T02:51:46.000Z"), "friend-ids": {{ 38108839, 44502073, 19244279, 45055684, 32489890, 25184431, 34275591, 47288414, 46973922, 28264345, 10024409, 4791958, 40576138, 33446414, 359486, 25595793, 25140170, 23149057, 47032976, 4283407 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2001-08-08") } ] }
-{ "id": 9959077, "id-copy": 9959077, "alias": "Josephine", "name": "JosephineLauffer", "user-since": datetime("2006-12-27T17:33:39.000Z"), "user-since-copy": datetime("2006-12-27T17:33:39.000Z"), "friend-ids": {{ 41423014, 33024139, 26147665, 14776436, 4726952, 12688804 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2001-03-26") } ] }
-{ "id": 9967888, "id-copy": 9967888, "alias": "Andrea", "name": "AndreaBerry", "user-since": datetime("2007-05-03T20:18:51.000Z"), "user-since-copy": datetime("2007-05-03T20:18:51.000Z"), "friend-ids": {{ 1106859, 38049440, 23056791, 16253206, 7727164, 19267641, 31798723, 30455164, 24738450, 15142413, 15111012, 3782070, 11502933, 44299958, 30277689, 3512757, 41960838, 7667284, 9192069, 12267931, 34901540, 20633036, 37186032, 1734718 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2007-04-01"), "end-date": date("2011-09-07") } ] }
-{ "id": 9974485, "id-copy": 9974485, "alias": "Leo", "name": "LeoRawls", "user-since": datetime("2005-02-12T12:01:58.000Z"), "user-since-copy": datetime("2005-02-12T12:01:58.000Z"), "friend-ids": {{ 41189338, 33744557, 2485502, 8308490, 43237410 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2002-11-05"), "end-date": date("2009-04-12") } ] }
-{ "id": 9996817, "id-copy": 9996817, "alias": "Vere", "name": "VereWilkerson", "user-since": datetime("2012-02-05T22:05:44.000Z"), "user-since-copy": datetime("2012-02-05T22:05:44.000Z"), "friend-ids": {{ 30010110, 31604568, 5741065, 29161468, 22429704, 16954129, 26525860, 1490181, 11444321, 24455724, 10411850, 39851031, 16059860, 32050795, 13116007, 12071588 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2004-11-04") } ] }
-{ "id": 10001410, "id-copy": 10001410, "alias": "Denzil", "name": "DenzilLedgerwood", "user-since": datetime("2006-12-24T10:56:58.000Z"), "user-since-copy": datetime("2006-12-24T10:56:58.000Z"), "friend-ids": {{ 25633920, 39748697, 3557647, 44396047, 25225495, 38723684, 5854330 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-08-14"), "end-date": date("2011-07-20") } ] }
-{ "id": 10017829, "id-copy": 10017829, "alias": "Adam", "name": "AdamTrovato", "user-since": datetime("2009-04-15T20:21:48.000Z"), "user-since-copy": datetime("2009-04-15T20:21:48.000Z"), "friend-ids": {{ 7572792, 20961281, 47727918, 25262896, 33740076, 14418354, 42807653, 34174665, 12459426, 28777106, 44409513, 39753872, 9172361, 36746114, 196755 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2007-09-25") } ] }
-{ "id": 10025086, "id-copy": 10025086, "alias": "Peggy", "name": "PeggyOlphert", "user-since": datetime("2009-06-24T16:14:48.000Z"), "user-since-copy": datetime("2009-06-24T16:14:48.000Z"), "friend-ids": {{ 13659719, 46045788, 35841713, 32392118, 24785179, 45483286, 47287227, 42691471, 7471992, 47671331, 25747076, 2368606, 34452743, 14570607, 31436760, 36423303, 31381129, 29414651, 10005587, 14082638, 13311890, 11592210, 1585557 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2008-07-20") } ] }
-{ "id": 10026061, "id-copy": 10026061, "alias": "Nonie", "name": "NonieChappel", "user-since": datetime("2007-06-22T10:06:38.000Z"), "user-since-copy": datetime("2007-06-22T10:06:38.000Z"), "friend-ids": {{ 38760716, 16809503, 6592849, 3736630, 32388289, 40487693, 27146403, 22621793, 35615399, 10839746, 693037, 25222841, 46448329, 40740448, 21652202, 30069817, 21957966 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2010-08-19"), "end-date": date("2010-08-17") } ] }
-{ "id": 10047001, "id-copy": 10047001, "alias": "Darcy", "name": "DarcyKava", "user-since": datetime("2012-02-25T17:16:18.000Z"), "user-since-copy": datetime("2012-02-25T17:16:18.000Z"), "friend-ids": {{ 15613341, 46557569, 20439965, 22442508, 32423739, 40757483, 36365324, 40706148, 12537361, 47741886, 24508947, 34168899, 10674474, 34285157, 28222068, 11113263 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2008-01-27") } ] }
-{ "id": 10047373, "id-copy": 10047373, "alias": "Rexana", "name": "RexanaDennis", "user-since": datetime("2010-01-05T15:43:34.000Z"), "user-since-copy": datetime("2010-01-05T15:43:34.000Z"), "friend-ids": {{ 1594, 40130182 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2004-07-04"), "end-date": date("2007-12-28") } ] }
-{ "id": 10073002, "id-copy": 10073002, "alias": "Josefa", "name": "JosefaNewman", "user-since": datetime("2010-10-06T09:28:29.000Z"), "user-since-copy": datetime("2010-10-06T09:28:29.000Z"), "friend-ids": {{ 7549910, 7287709, 24063891, 41208589, 22325854, 16465930, 45067165, 42784968, 26414870, 16479308, 22681119, 40811475, 9603161, 23525416, 15131604, 4782290, 36997646, 35862360, 42008502, 438438, 25913601, 39300786, 15041382, 37410001 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2011-02-05"), "end-date": date("2011-10-24") } ] }
-{ "id": 10100707, "id-copy": 10100707, "alias": "Brittni", "name": "BrittniEaster", "user-since": datetime("2008-10-03T02:27:48.000Z"), "user-since-copy": datetime("2008-10-03T02:27:48.000Z"), "friend-ids": {{ 28725707, 8497950, 18892135, 1016149, 32023719, 34079976, 39582966, 15469248, 14059091, 6681733, 18398487, 41385960 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2006-04-21") } ] }
-{ "id": 10151953, "id-copy": 10151953, "alias": "Howard", "name": "HowardHoopengarner", "user-since": datetime("2006-07-23T01:43:57.000Z"), "user-since-copy": datetime("2006-07-23T01:43:57.000Z"), "friend-ids": {{ 32564548, 19333543, 27610653, 27936980, 7471201, 1353451, 30864511, 41582907, 22918030, 6011307, 21622284, 44695813, 34728110, 33062051, 29420834, 37472592, 3655974, 34618485, 21615748, 14107596, 15317302, 21805666, 4563480 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2012-06-08") } ] }
-{ "id": 10162495, "id-copy": 10162495, "alias": "Malina", "name": "MalinaTrout", "user-since": datetime("2006-12-19T12:12:55.000Z"), "user-since-copy": datetime("2006-12-19T12:12:55.000Z"), "friend-ids": {{ 40578475, 43374248, 7059820, 18838227, 45149295, 47680877, 11640348, 19081155, 9959453, 46807478, 45192583, 39333999, 4869981, 42888726, 32789666, 19653202 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2000-11-08") } ] }
-{ "id": 10166767, "id-copy": 10166767, "alias": "Leon", "name": "LeonWardle", "user-since": datetime("2008-05-19T07:05:45.000Z"), "user-since-copy": datetime("2008-05-19T07:05:45.000Z"), "friend-ids": {{ 41883510, 44504996, 36617462, 32609381, 11246739, 18717645, 32225763, 25136144, 18258339, 4951535, 40063362, 38810936, 1994155, 16613514, 25411748, 34221779, 44135463 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2006-10-11") } ] }
-{ "id": 10173691, "id-copy": 10173691, "alias": "Elissa", "name": "ElissaWilliams", "user-since": datetime("2011-09-26T16:07:17.000Z"), "user-since-copy": datetime("2011-09-26T16:07:17.000Z"), "friend-ids": {{ 2526422 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2001-07-22") } ] }
-{ "id": 10190329, "id-copy": 10190329, "alias": "Rachyl", "name": "RachylAdams", "user-since": datetime("2005-08-25T14:09:48.000Z"), "user-since-copy": datetime("2005-08-25T14:09:48.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2002-11-17") } ] }
-{ "id": 10197700, "id-copy": 10197700, "alias": "Frederica", "name": "FredericaCherry", "user-since": datetime("2006-04-10T01:23:53.000Z"), "user-since-copy": datetime("2006-04-10T01:23:53.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-01-01"), "end-date": date("2009-07-14") } ] }
-{ "id": 10212385, "id-copy": 10212385, "alias": "Alice", "name": "AliceJones", "user-since": datetime("2009-05-16T16:08:03.000Z"), "user-since-copy": datetime("2009-05-16T16:08:03.000Z"), "friend-ids": {{ 4158604, 3204211, 21491737, 39619715, 9750334 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2012-04-19") } ] }
-{ "id": 10241767, "id-copy": 10241767, "alias": "Lewin", "name": "LewinBurkett", "user-since": datetime("2008-03-24T21:09:05.000Z"), "user-since-copy": datetime("2008-03-24T21:09:05.000Z"), "friend-ids": {{ 5503, 32598090, 36950887, 22362781, 16089120, 30220805, 6197105, 44773004, 17924848, 36033966, 41338779, 38304288, 18528858, 6384026, 46633327, 18024168, 13983021, 7158391, 31922078, 1082072 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2004-02-17") } ] }
-{ "id": 10252147, "id-copy": 10252147, "alias": "Concha", "name": "ConchaMckinnon", "user-since": datetime("2009-12-21T03:27:35.000Z"), "user-since-copy": datetime("2009-12-21T03:27:35.000Z"), "friend-ids": {{ 8837048, 7758233, 2108777, 31062874, 34698247, 33766563, 10653492, 25103733, 24629375, 38758275, 37539109, 47252638, 41559516, 41883197, 9608881, 26501553, 39435548, 43307321, 46890131, 29908109 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-05-09") } ] }
-{ "id": 10261300, "id-copy": 10261300, "alias": "Nick", "name": "NickRohtin", "user-since": datetime("2007-01-24T17:56:52.000Z"), "user-since-copy": datetime("2007-01-24T17:56:52.000Z"), "friend-ids": {{ 37649902 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2004-03-06"), "end-date": date("2007-05-20") } ] }
-{ "id": 10270597, "id-copy": 10270597, "alias": "Ava", "name": "AvaTanner", "user-since": datetime("2010-04-23T11:49:39.000Z"), "user-since-copy": datetime("2010-04-23T11:49:39.000Z"), "friend-ids": {{ 38894360, 9403074, 25855965, 36511208, 4947767, 10318201, 3532083, 28684767, 22730535, 17994309, 21209113, 14980333, 5611975, 31951870, 16697364, 5033131, 13637894, 18107216, 9769275, 25479923, 15320268, 28897820, 22865104 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2011-04-12"), "end-date": date("2011-09-07") } ] }
-{ "id": 10272571, "id-copy": 10272571, "alias": "Jarrett", "name": "JarrettGoldvogel", "user-since": datetime("2010-04-28T23:24:22.000Z"), "user-since-copy": datetime("2010-04-28T23:24:22.000Z"), "friend-ids": {{ 47024505, 36647273, 32152567, 28239957, 11739703, 47515825, 17408763, 41224279, 41487670, 43339913 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2004-04-06"), "end-date": date("2010-02-14") } ] }
-{ "id": 10277731, "id-copy": 10277731, "alias": "Gallagher", "name": "GallagherMagor", "user-since": datetime("2007-07-02T07:37:02.000Z"), "user-since-copy": datetime("2007-07-02T07:37:02.000Z"), "friend-ids": {{ 22730683, 9352614, 42748868, 24014877, 21749502, 30751403, 41768964, 13317192, 31877814, 35318552, 26843471, 21232937, 11268529, 21902785 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-09-03") } ] }
-{ "id": 10278550, "id-copy": 10278550, "alias": "Parker", "name": "ParkerWinton", "user-since": datetime("2008-03-02T18:54:35.000Z"), "user-since-copy": datetime("2008-03-02T18:54:35.000Z"), "friend-ids": {{ 281420, 13481584, 25554653, 2922131, 15313837, 33567564, 20182917, 20143660, 35884326, 22038516, 183180 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2002-12-16"), "end-date": date("2010-08-04") } ] }
-{ "id": 10284583, "id-copy": 10284583, "alias": "Salal", "name": "SalalButterfill", "user-since": datetime("2011-02-05T13:39:36.000Z"), "user-since-copy": datetime("2011-02-05T13:39:36.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2008-08-10"), "end-date": date("2011-05-02") } ] }
-{ "id": 10298530, "id-copy": 10298530, "alias": "Natalee", "name": "NataleeBell", "user-since": datetime("2010-09-07T14:14:59.000Z"), "user-since-copy": datetime("2010-09-07T14:14:59.000Z"), "friend-ids": {{ 36077399, 47946678, 4189158, 42122618, 14179077, 26433248, 25903252, 23116624, 33542934, 1071320, 31914369, 28408518, 40811454, 19212473, 25057330, 42758915 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-02-17") } ] }
-{ "id": 10299298, "id-copy": 10299298, "alias": "Belinda", "name": "BelindaRockwell", "user-since": datetime("2005-03-08T07:13:05.000Z"), "user-since-copy": datetime("2005-03-08T07:13:05.000Z"), "friend-ids": {{ 31301282, 34653696, 23868758 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2008-08-19") } ] }
-{ "id": 10307155, "id-copy": 10307155, "alias": "Rhetta", "name": "RhettaGarneys", "user-since": datetime("2008-03-17T00:33:40.000Z"), "user-since-copy": datetime("2008-03-17T00:33:40.000Z"), "friend-ids": {{ 5658375, 40536479, 47961112, 28517297, 26103231, 32434876, 44285321, 44471686 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2006-06-07"), "end-date": date("2010-10-03") } ] }
-{ "id": 10390954, "id-copy": 10390954, "alias": "Lucinda", "name": "LucindaWatson", "user-since": datetime("2006-11-16T21:20:41.000Z"), "user-since-copy": datetime("2006-11-16T21:20:41.000Z"), "friend-ids": {{ 36017573, 9298650, 16054222, 21985420, 23378246, 30163820, 20942039, 28917630, 20851877, 41794807, 45887537, 39768986, 42476881, 5070921, 29487760, 24953551, 32065985, 16342096, 41522555, 41923127, 34675252, 10040601, 32604114, 23852658 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2009-03-14") } ] }
-{ "id": 10397017, "id-copy": 10397017, "alias": "Holly", "name": "HollyHatch", "user-since": datetime("2006-04-12T03:26:11.000Z"), "user-since-copy": datetime("2006-04-12T03:26:11.000Z"), "friend-ids": {{ 1504006, 21411501, 20934982, 24019384, 8634101, 25659178, 16581112, 2481631, 15544800 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2000-12-04") } ] }
-{ "id": 10405423, "id-copy": 10405423, "alias": "Pauletta", "name": "PaulettaGuess", "user-since": datetime("2007-06-11T02:54:36.000Z"), "user-since-copy": datetime("2007-06-11T02:54:36.000Z"), "friend-ids": {{ 14845791, 24263161, 2648994, 30766767, 10127359, 20706390 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-10-27") } ] }
-{ "id": 10412287, "id-copy": 10412287, "alias": "Wren", "name": "WrenElizabeth", "user-since": datetime("2009-06-25T07:26:48.000Z"), "user-since-copy": datetime("2009-06-25T07:26:48.000Z"), "friend-ids": {{ 23487913, 35496582, 14824955, 5998721, 10925419, 38937432, 6285652 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2012-07-20"), "end-date": date("2012-07-12") } ] }
-{ "id": 10464121, "id-copy": 10464121, "alias": "Enriqueta", "name": "EnriquetaHincken", "user-since": datetime("2005-11-19T09:43:20.000Z"), "user-since-copy": datetime("2005-11-19T09:43:20.000Z"), "friend-ids": {{ 31238269, 29421316, 14426443, 30128291, 9926275, 33523504, 19113054, 402505, 12662005, 36090974, 8733776, 18706660, 14174144, 46009221, 17906304, 41780430, 21807110, 22521282, 21492740, 34033053, 16784027, 11948555 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2008-09-23") } ] }
-{ "id": 10473718, "id-copy": 10473718, "alias": "Elissa", "name": "ElissaStainforth", "user-since": datetime("2007-06-20T07:46:54.000Z"), "user-since-copy": datetime("2007-06-20T07:46:54.000Z"), "friend-ids": {{ 1645948, 612724, 46091510, 32750261, 40622752, 10190250, 42030152, 28645649, 27513961 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2003-06-21"), "end-date": date("2011-09-05") } ] }
-{ "id": 10474273, "id-copy": 10474273, "alias": "Juliana", "name": "JulianaLing", "user-since": datetime("2005-05-04T20:58:12.000Z"), "user-since-copy": datetime("2005-05-04T20:58:12.000Z"), "friend-ids": {{ 8881381, 34113161, 15553599, 40081858, 12450920, 42147178, 568875, 11891228, 13309462, 39127120, 34765111, 19162279, 29505162, 891909, 33485893, 25658561, 36146447, 37027867, 39396759 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2008-03-03") } ] }
-{ "id": 10478512, "id-copy": 10478512, "alias": "Remona", "name": "RemonaPittman", "user-since": datetime("2007-06-19T12:20:07.000Z"), "user-since-copy": datetime("2007-06-19T12:20:07.000Z"), "friend-ids": {{ 12750727 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2001-02-02") } ] }
-{ "id": 10514428, "id-copy": 10514428, "alias": "Eliseo", "name": "EliseoHoffhants", "user-since": datetime("2012-08-24T08:40:51.000Z"), "user-since-copy": datetime("2012-08-24T08:40:51.000Z"), "friend-ids": {{ 45751891, 26026786, 24531389, 26239368, 34021241 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2010-03-01"), "end-date": date("2010-08-02") } ] }
-{ "id": 10532791, "id-copy": 10532791, "alias": "Byrne", "name": "ByrneLafortune", "user-since": datetime("2010-03-13T13:21:05.000Z"), "user-since-copy": datetime("2010-03-13T13:21:05.000Z"), "friend-ids": {{ 35020297, 40002497, 16857157, 47134232, 37864297, 31029450, 36968713, 36672267, 15503365, 43888732, 29395734, 35372186, 19093208, 21774877, 9785166, 22833579 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-12-21") } ] }
-{ "id": 10577128, "id-copy": 10577128, "alias": "Charnette", "name": "CharnettePyle", "user-since": datetime("2008-08-20T21:25:22.000Z"), "user-since-copy": datetime("2008-08-20T21:25:22.000Z"), "friend-ids": {{ 30078840, 16315930, 12006652, 31984600, 12053254, 41773411, 43318427, 21592935, 40739515, 30608076, 21922300, 5687640 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2001-11-25"), "end-date": date("2002-08-12") } ] }
-{ "id": 10594069, "id-copy": 10594069, "alias": "Clinton", "name": "ClintonMiller", "user-since": datetime("2007-03-12T05:19:19.000Z"), "user-since-copy": datetime("2007-03-12T05:19:19.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "itlab", "start-date": date("2010-06-06") } ] }
-{ "id": 10607341, "id-copy": 10607341, "alias": "Evander", "name": "EvanderPycroft", "user-since": datetime("2005-08-09T23:36:46.000Z"), "user-since-copy": datetime("2005-08-09T23:36:46.000Z"), "friend-ids": {{ 46200658, 38004155 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2004-06-13") } ] }
-{ "id": 10613617, "id-copy": 10613617, "alias": "Jeanie", "name": "JeanieEiford", "user-since": datetime("2007-02-09T12:16:09.000Z"), "user-since-copy": datetime("2007-02-09T12:16:09.000Z"), "friend-ids": {{ 24843944, 3651507, 25077638, 18662161, 46723847, 31558857, 11235682, 15640606, 31889112, 45342233, 25865191, 1530020, 39187188, 4939030, 19220487, 19619126, 25284665, 1206869, 40740763 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2001-07-22") } ] }
-{ "id": 10637896, "id-copy": 10637896, "alias": "Hiram", "name": "HiramRohtin", "user-since": datetime("2006-11-05T14:44:03.000Z"), "user-since-copy": datetime("2006-11-05T14:44:03.000Z"), "friend-ids": {{ 1387663, 11367203, 24828245 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-02-18"), "end-date": date("2012-02-12") } ] }
-{ "id": 10640851, "id-copy": 10640851, "alias": "Tabitha", "name": "TabithaWhitten", "user-since": datetime("2010-01-28T14:25:58.000Z"), "user-since-copy": datetime("2010-01-28T14:25:58.000Z"), "friend-ids": {{ 42792549, 5330514, 24582133, 43384590, 38083439, 31221232, 18064537, 21736064, 7919520, 18998284, 20165148, 28492287, 21987533, 23638155 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2006-06-18"), "end-date": date("2007-07-20") } ] }
-{ "id": 10655089, "id-copy": 10655089, "alias": "Quinn", "name": "QuinnHays", "user-since": datetime("2009-11-25T04:42:39.000Z"), "user-since-copy": datetime("2009-11-25T04:42:39.000Z"), "friend-ids": {{ 17385636, 24378500, 37614592, 32315940, 18046144, 45823175, 29709981, 28423306, 23783823, 10623867, 27782698 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2012-06-09") } ] }
-{ "id": 10714447, "id-copy": 10714447, "alias": "Leone", "name": "LeoneCoughenour", "user-since": datetime("2012-06-13T05:05:11.000Z"), "user-since-copy": datetime("2012-06-13T05:05:11.000Z"), "friend-ids": {{ 13098839, 21185838, 26566436, 37464340, 8086775, 37143068, 40377316, 39371296 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2005-04-16") } ] }
-{ "id": 10729942, "id-copy": 10729942, "alias": "Valda", "name": "ValdaFea", "user-since": datetime("2005-07-16T09:31:53.000Z"), "user-since-copy": datetime("2005-07-16T09:31:53.000Z"), "friend-ids": {{ 20145015, 42027050, 38819467, 3406065, 4977132, 47154979, 23685067 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2008-10-12") } ] }
-{ "id": 10754107, "id-copy": 10754107, "alias": "Jeri", "name": "JeriSanner", "user-since": datetime("2009-11-15T23:47:08.000Z"), "user-since-copy": datetime("2009-11-15T23:47:08.000Z"), "friend-ids": {{ 19868241, 28778419, 16761189, 28588239, 1592484, 41256056, 36550491, 10555328, 3086612, 37431116, 45976270 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2004-11-06") } ] }
-{ "id": 10760020, "id-copy": 10760020, "alias": "Emeline", "name": "EmelineCowher", "user-since": datetime("2006-03-11T07:02:10.000Z"), "user-since-copy": datetime("2006-03-11T07:02:10.000Z"), "friend-ids": {{ 2652618, 22247716, 39487944, 16288504, 8109009, 34390947, 2041892, 27800644, 5979423, 12674908 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2007-12-26"), "end-date": date("2007-09-04") } ] }
-{ "id": 10766221, "id-copy": 10766221, "alias": "Rosalyn", "name": "RosalynBaxter", "user-since": datetime("2009-04-16T15:46:54.000Z"), "user-since-copy": datetime("2009-04-16T15:46:54.000Z"), "friend-ids": {{ 43759575, 1264811, 9906031, 21579594, 45786210, 14876191, 10711745, 25134652, 25426644, 29987806, 1953812, 29568099, 38860088, 7073296, 13746927, 11395655, 36208297, 25317651, 21356968 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2000-07-04") } ] }
-{ "id": 10800157, "id-copy": 10800157, "alias": "Tiara", "name": "TiaraFuhrer", "user-since": datetime("2010-05-24T21:52:36.000Z"), "user-since-copy": datetime("2010-05-24T21:52:36.000Z"), "friend-ids": {{ 34031723 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2003-03-18"), "end-date": date("2005-09-20") } ] }
-{ "id": 10824484, "id-copy": 10824484, "alias": "Linda", "name": "LindaStanfield", "user-since": datetime("2009-03-03T12:54:55.000Z"), "user-since-copy": datetime("2009-03-03T12:54:55.000Z"), "friend-ids": {{ 39164563, 20321780, 19901289, 37969494, 15051354, 42576590, 14550253, 33649901, 6008727, 17749643, 7792769, 18652053, 8565400, 43899372, 7433016, 42506713 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2012-03-21") } ] }
-{ "id": 10835521, "id-copy": 10835521, "alias": "Margeret", "name": "MargeretEve", "user-since": datetime("2010-02-13T16:16:55.000Z"), "user-since-copy": datetime("2010-02-13T16:16:55.000Z"), "friend-ids": {{ 40363275, 44184724, 42855751, 10492711, 561147, 45516609, 38567828, 9695088, 40235757 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2012-06-08"), "end-date": date("2012-06-27") } ] }
-{ "id": 10851595, "id-copy": 10851595, "alias": "Juan", "name": "JuanSoames", "user-since": datetime("2006-02-16T05:34:28.000Z"), "user-since-copy": datetime("2006-02-16T05:34:28.000Z"), "friend-ids": {{ 34589906, 8801547, 38357163, 39649840, 18254469, 38911658, 17825991, 26015024, 29742264, 13155934, 28459597, 34931012, 20376527 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2008-11-17"), "end-date": date("2009-01-13") } ] }
-{ "id": 10853926, "id-copy": 10853926, "alias": "Kennard", "name": "KennardGarland", "user-since": datetime("2007-11-28T20:40:40.000Z"), "user-since-copy": datetime("2007-11-28T20:40:40.000Z"), "friend-ids": {{ 47687855, 28575858 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2005-07-17") } ] }
-{ "id": 10861183, "id-copy": 10861183, "alias": "Zilla", "name": "ZillaOneal", "user-since": datetime("2008-03-12T23:37:18.000Z"), "user-since-copy": datetime("2008-03-12T23:37:18.000Z"), "friend-ids": {{ 26262188, 17172669, 43068853, 47767064, 34552281, 33602720, 35448839, 6347557, 11913432, 45186875, 10451537, 46881437, 27965706 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-09-03"), "end-date": date("2009-07-22") } ] }
-{ "id": 10867444, "id-copy": 10867444, "alias": "Tetty", "name": "TettyZundel", "user-since": datetime("2012-07-26T17:54:45.000Z"), "user-since-copy": datetime("2012-07-26T17:54:45.000Z"), "friend-ids": {{ 17830961, 13154371, 12005619, 15279158, 15766172, 3071670, 4314512, 29378453, 33264674, 32657723, 37875054, 6208013, 23310809, 11994927, 9787690, 25069760, 11104605, 44517542, 45829337, 26593992 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2001-09-25") } ] }
-{ "id": 10878898, "id-copy": 10878898, "alias": "Webster", "name": "WebsterCarr", "user-since": datetime("2006-07-28T21:17:56.000Z"), "user-since-copy": datetime("2006-07-28T21:17:56.000Z"), "friend-ids": {{ 11755002, 37594815, 4340697, 27424145, 22193377, 31509516, 31372689, 47386546, 30347891, 4070454, 18531894, 28306285, 14110568, 17830332 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2002-03-12") } ] }
-{ "id": 10882393, "id-copy": 10882393, "alias": "Erica", "name": "EricaHynes", "user-since": datetime("2006-09-16T16:39:05.000Z"), "user-since-copy": datetime("2006-09-16T16:39:05.000Z"), "friend-ids": {{ 23491370, 13390922, 19685128, 47763240, 9493285, 10823383, 45076071, 14858340, 12545499, 40367152, 2150593, 45723007, 21362425, 25435409, 776198, 8016739, 21691528, 21036410, 3131225, 20078710, 28405287, 15599245, 39126345, 36208574 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2012-05-14"), "end-date": date("2012-05-22") } ] }
-{ "id": 10883062, "id-copy": 10883062, "alias": "Lamar", "name": "LamarFelbrigge", "user-since": datetime("2005-02-12T03:19:28.000Z"), "user-since-copy": datetime("2005-02-12T03:19:28.000Z"), "friend-ids": {{ 26304238, 21048260, 26614197, 41153844, 17163890, 27772117, 26679939, 22001103, 46907785, 21321841, 46215643, 31285577, 14997749, 46997910, 44367495, 13858871, 20405288, 36784906, 33752927, 30769058, 43188289, 34006518, 23022696 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2012-06-16") } ] }
-{ "id": 10884241, "id-copy": 10884241, "alias": "Anamaria", "name": "AnamariaMoon", "user-since": datetime("2005-03-28T11:38:17.000Z"), "user-since-copy": datetime("2005-03-28T11:38:17.000Z"), "friend-ids": {{ 21445295, 42154978, 41608378, 3406391, 26013137, 45437958, 22377352, 26150886, 25726611, 31834547, 17506680, 22932063, 16700407, 22939810, 152978, 45307280, 42212660, 30124140, 9494103, 35217706, 41538534, 26586744, 26538590 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2011-09-10"), "end-date": date("2011-02-06") } ] }
-{ "id": 10889389, "id-copy": 10889389, "alias": "Roselyn", "name": "RoselynLlora", "user-since": datetime("2012-03-25T15:21:06.000Z"), "user-since-copy": datetime("2012-03-25T15:21:06.000Z"), "friend-ids": {{ 38921827, 1378686, 22284385, 17464785, 16302500, 47598267, 25016712, 11151378, 16381115, 16371401 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2005-12-02") } ] }
-{ "id": 10892830, "id-copy": 10892830, "alias": "Audrie", "name": "AudrieHawkins", "user-since": datetime("2011-11-19T00:51:33.000Z"), "user-since-copy": datetime("2011-11-19T00:51:33.000Z"), "friend-ids": {{ 8838768, 18321840, 16958648, 27000957, 19090823, 11772058, 18573458, 24662627, 27415154, 4998699, 44522833, 44994903, 6514403, 43833807, 38512495, 6964420, 11334788, 14298721, 25316052, 11632302 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2005-02-04") } ] }
-{ "id": 10901047, "id-copy": 10901047, "alias": "Salvador", "name": "SalvadorBynum", "user-since": datetime("2012-01-13T02:30:17.000Z"), "user-since-copy": datetime("2012-01-13T02:30:17.000Z"), "friend-ids": {{ 29122263, 27975257, 7988516, 9270552, 17837898, 42339445, 46097101, 32303800, 17233223, 10656090, 36709955, 17535336, 27157992, 30360627, 15304415, 28922979, 27243261, 9307382, 43171015, 31593421, 21246902, 40452339, 25735551, 23716187 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2010-11-27") } ] }
-{ "id": 10902049, "id-copy": 10902049, "alias": "Fae", "name": "FaeRing", "user-since": datetime("2008-06-15T12:54:57.000Z"), "user-since-copy": datetime("2008-06-15T12:54:57.000Z"), "friend-ids": {{ 2667467, 46445373, 11696423, 42003744, 47667382, 34088774, 4279683, 29934858, 21213543, 44195034, 38786294, 14946433, 38805114, 9972575, 3309290, 5324029, 32663319, 20577589, 9110909, 27272396, 47622938 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2007-11-15") } ] }
-{ "id": 10902649, "id-copy": 10902649, "alias": "Makenzie", "name": "MakenzieWerner", "user-since": datetime("2005-12-20T00:23:45.000Z"), "user-since-copy": datetime("2005-12-20T00:23:45.000Z"), "friend-ids": {{ 9011568, 38173487, 45649445, 11873586 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2000-01-06"), "end-date": date("2009-03-24") } ] }
-{ "id": 10907953, "id-copy": 10907953, "alias": "Wymond", "name": "WymondSnyder", "user-since": datetime("2006-02-25T03:33:22.000Z"), "user-since-copy": datetime("2006-02-25T03:33:22.000Z"), "friend-ids": {{ 16280602, 26846293, 39235173, 4686537, 30457440, 23649561, 34348317, 28099021, 1622222, 24073647, 4742953, 14925763, 17026705, 46257859, 22592244 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2012-07-22") } ] }
-{ "id": 10912441, "id-copy": 10912441, "alias": "Janae", "name": "JanaeErschoff", "user-since": datetime("2009-04-17T09:26:36.000Z"), "user-since-copy": datetime("2009-04-17T09:26:36.000Z"), "friend-ids": {{ 11445243, 13239218, 2302326, 37976140, 45374131, 14136536, 2051767, 7824391, 42808044, 41836900, 35275542, 33493951, 8497237, 42991362, 24049395, 32159562, 23378256, 4723574, 47010157 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-04-20"), "end-date": date("2012-04-04") } ] }
-{ "id": 10931563, "id-copy": 10931563, "alias": "Laraine", "name": "LaraineCountryman", "user-since": datetime("2012-03-17T17:06:59.000Z"), "user-since-copy": datetime("2012-03-17T17:06:59.000Z"), "friend-ids": {{ 17266368, 75990, 37678426, 43207424, 37434492, 26338447, 33450799, 5401110, 44962643, 5514847 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2008-09-08") } ] }
-{ "id": 10931647, "id-copy": 10931647, "alias": "Bertina", "name": "BertinaStraub", "user-since": datetime("2011-05-25T19:21:43.000Z"), "user-since-copy": datetime("2011-05-25T19:21:43.000Z"), "friend-ids": {{ 12208030, 43810737, 43870253, 20720324, 7601394, 22266404, 21210273, 10076577, 25757258, 1909792, 26189079, 37799329, 24923233, 31687015, 37580896, 44906728, 46928405, 10679805, 14520239, 1690125, 37459202, 36684838, 30982356 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2011-02-17"), "end-date": date("2011-06-20") } ] }
-{ "id": 10933138, "id-copy": 10933138, "alias": "Gwendoline", "name": "GwendolineCypret", "user-since": datetime("2006-04-10T03:55:29.000Z"), "user-since-copy": datetime("2006-04-10T03:55:29.000Z"), "friend-ids": {{ 9996028, 18756914, 15079751, 34129343, 44558538, 25387070, 44250368, 37560291, 5178625, 10379959, 39639296, 8784216, 13429736, 22802431, 11154064, 2453387, 24748342, 34032462, 32570963, 4861587, 19421488, 10848442 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2008-12-24"), "end-date": date("2010-05-20") } ] }
-{ "id": 10943026, "id-copy": 10943026, "alias": "Raeburn", "name": "RaeburnAllshouse", "user-since": datetime("2008-08-26T04:51:27.000Z"), "user-since-copy": datetime("2008-08-26T04:51:27.000Z"), "friend-ids": {{ 6784667, 1651647, 45052591, 21630976, 20049039, 37839759, 38694475, 23340828, 8641638, 4568782, 35684305, 20895609, 2213341, 8612199, 14260231, 8621325, 21926952, 41656664, 45180955 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2007-09-28") } ] }
-{ "id": 10943104, "id-copy": 10943104, "alias": "Prudence", "name": "PrudencePriebe", "user-since": datetime("2006-04-27T21:00:43.000Z"), "user-since-copy": datetime("2006-04-27T21:00:43.000Z"), "friend-ids": {{ 43633941, 38710166, 34456560, 11324015, 21000755, 23356715, 21056830, 27295754 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-30") } ] }
-{ "id": 10951918, "id-copy": 10951918, "alias": "Doran", "name": "DoranBell", "user-since": datetime("2005-08-22T14:07:50.000Z"), "user-since-copy": datetime("2005-08-22T14:07:50.000Z"), "friend-ids": {{ 6952033, 22223086, 5858716, 35128893, 22115927, 5821006, 16264772, 4151991, 40384467, 19801357, 42871024, 46855275, 35241988, 17208259, 47420533, 25182232, 14247140, 19664015, 33132502, 47813026, 12819081, 29321093, 42851957, 30756972 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2012-01-22") } ] }
-{ "id": 10962466, "id-copy": 10962466, "alias": "Zoey", "name": "ZoeyCady", "user-since": datetime("2012-07-15T20:02:23.000Z"), "user-since-copy": datetime("2012-07-15T20:02:23.000Z"), "friend-ids": {{ 12726157, 268799, 29381478, 15699674, 1150948, 8000369, 41608951, 11382366, 770690, 25889785, 37815043, 40437016, 38679636, 32956275, 34853801 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2012-05-08") } ] }
-{ "id": 10972447, "id-copy": 10972447, "alias": "Loretta", "name": "LorettaBriggs", "user-since": datetime("2005-07-01T10:25:33.000Z"), "user-since-copy": datetime("2005-07-01T10:25:33.000Z"), "friend-ids": {{ 6898813, 6606991, 14092255, 9865734, 23960698, 47354873, 19345256 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2005-06-02") } ] }
-{ "id": 10993267, "id-copy": 10993267, "alias": "Esmund", "name": "EsmundDunkle", "user-since": datetime("2005-11-16T21:18:20.000Z"), "user-since-copy": datetime("2005-11-16T21:18:20.000Z"), "friend-ids": {{ 1277480, 11393524, 32336542, 41857626, 7807437, 25280677, 17518254, 7723810, 18423045, 11937236, 21507800 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-12-03"), "end-date": date("2011-11-26") } ] }
-{ "id": 11004067, "id-copy": 11004067, "alias": "Vickie", "name": "VickieRosenstiehl", "user-since": datetime("2012-04-15T02:37:43.000Z"), "user-since-copy": datetime("2012-04-15T02:37:43.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2012-08-01"), "end-date": date("2012-08-06") } ] }
-{ "id": 11007700, "id-copy": 11007700, "alias": "Elly", "name": "EllyWard", "user-since": datetime("2009-04-20T08:46:09.000Z"), "user-since-copy": datetime("2009-04-20T08:46:09.000Z"), "friend-ids": {{ 9712756, 6523354 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2007-12-07"), "end-date": date("2007-07-27") } ] }
-{ "id": 11015908, "id-copy": 11015908, "alias": "Giuseppe", "name": "GiuseppeWard", "user-since": datetime("2008-09-14T16:37:40.000Z"), "user-since-copy": datetime("2008-09-14T16:37:40.000Z"), "friend-ids": {{ 9972151, 40271551, 46207899, 29987388, 19876511, 47546614, 17051350, 1579198, 2151480, 26507940, 18177808, 25866392, 40253780 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2008-02-24") } ] }
-{ "id": 11032186, "id-copy": 11032186, "alias": "Tabby", "name": "TabbySealis", "user-since": datetime("2007-12-10T21:45:46.000Z"), "user-since-copy": datetime("2007-12-10T21:45:46.000Z"), "friend-ids": {{ 8190058, 5089537, 18167034, 19113649, 38817127, 7644664, 12427817, 39615196, 11451538, 27188211, 27425673, 33084974, 10726858, 40696324, 41487982, 42282364, 17084607, 41647211, 40268195, 29075837, 41802984, 9719771, 29747340, 28103359 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-07-13"), "end-date": date("2010-12-04") } ] }
-{ "id": 11032477, "id-copy": 11032477, "alias": "Wilmer", "name": "WilmerWortman", "user-since": datetime("2007-06-03T19:27:24.000Z"), "user-since-copy": datetime("2007-06-03T19:27:24.000Z"), "friend-ids": {{ 18685187, 2599612, 27305395, 20825021, 20327586, 21301262, 29222955, 20377452, 11211553, 37446807, 20533832, 10098143, 43828837, 37254072, 46029810, 16401947, 7537056, 41738273, 4665729, 27400110, 146251, 14185116 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2006-03-17"), "end-date": date("2011-08-03") } ] }
-{ "id": 11039716, "id-copy": 11039716, "alias": "Piedad", "name": "PiedadHowe", "user-since": datetime("2011-02-23T17:18:37.000Z"), "user-since-copy": datetime("2011-02-23T17:18:37.000Z"), "friend-ids": {{ 13323345 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2009-03-26"), "end-date": date("2009-06-17") } ] }
-{ "id": 11051014, "id-copy": 11051014, "alias": "Tad", "name": "TadWilson", "user-since": datetime("2011-05-05T14:48:34.000Z"), "user-since-copy": datetime("2011-05-05T14:48:34.000Z"), "friend-ids": {{ 42862096, 17517240, 8058482, 9927174, 4207109, 4924943, 11531213 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2005-01-25"), "end-date": date("2010-11-14") } ] }
-{ "id": 11081539, "id-copy": 11081539, "alias": "Haidee", "name": "HaideeStyle", "user-since": datetime("2012-06-13T11:37:34.000Z"), "user-since-copy": datetime("2012-06-13T11:37:34.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2001-03-05"), "end-date": date("2003-11-17") } ] }
-{ "id": 11090788, "id-copy": 11090788, "alias": "Randy", "name": "RandyClose", "user-since": datetime("2005-07-26T19:29:20.000Z"), "user-since-copy": datetime("2005-07-26T19:29:20.000Z"), "friend-ids": {{ 43392502, 7581874, 13279708, 16989391, 32340594, 7048512, 33084049, 16279611, 21735714, 23485799, 18185370, 43945382, 41653020, 13517043, 35395274, 24133848, 15355027, 4752815, 15007500, 25733540, 2114558, 37909789, 2805493, 16521087 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2004-09-14") } ] }
-{ "id": 11092324, "id-copy": 11092324, "alias": "Paul", "name": "PaulOneal", "user-since": datetime("2006-11-20T10:50:19.000Z"), "user-since-copy": datetime("2006-11-20T10:50:19.000Z"), "friend-ids": {{ 44707820, 20249424, 18862268, 32895394, 29899430 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2003-01-06") } ] }
-{ "id": 11097556, "id-copy": 11097556, "alias": "Tia", "name": "TiaHair", "user-since": datetime("2010-10-28T01:21:36.000Z"), "user-since-copy": datetime("2010-10-28T01:21:36.000Z"), "friend-ids": {{ 19746022, 42650092, 45679457, 43873545, 5490025, 42900988, 32855768, 20717716, 15007194, 23035301, 24322095, 27796211, 27751858, 4726224, 5570083, 18421959, 28424121, 22311092, 13781420, 18215783, 19934706, 18408890, 24792739, 4022527 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2003-04-03") } ] }
-{ "id": 11103856, "id-copy": 11103856, "alias": "Dennise", "name": "DenniseGarland", "user-since": datetime("2008-10-19T11:09:14.000Z"), "user-since-copy": datetime("2008-10-19T11:09:14.000Z"), "friend-ids": {{ 2613052, 4777379, 29911213, 30822813, 44182985, 803163, 32630608, 7433428, 43625503, 19274272, 20950244, 21434389, 44059623, 40416129, 47937344, 12392360 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2005-04-10"), "end-date": date("2005-07-26") } ] }
-{ "id": 11113168, "id-copy": 11113168, "alias": "Daphne", "name": "DaphneHindman", "user-since": datetime("2011-11-09T02:55:42.000Z"), "user-since-copy": datetime("2011-11-09T02:55:42.000Z"), "friend-ids": {{ 194785, 11696942, 23072861, 37052204, 17574763, 14099428, 44155581 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2002-07-16"), "end-date": date("2006-11-08") } ] }
-{ "id": 11140483, "id-copy": 11140483, "alias": "Nena", "name": "NenaBullard", "user-since": datetime("2008-02-23T10:24:08.000Z"), "user-since-copy": datetime("2008-02-23T10:24:08.000Z"), "friend-ids": {{ 26438400, 45201681, 12155417, 43414633, 14267296, 40906639, 8768744, 46840439, 43848021, 24521652, 41247005, 44999926, 13062334, 47731182 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2001-05-16") } ] }
-{ "id": 11152162, "id-copy": 11152162, "alias": "Tennille", "name": "TennilleGongaware", "user-since": datetime("2008-12-22T17:22:19.000Z"), "user-since-copy": datetime("2008-12-22T17:22:19.000Z"), "friend-ids": {{ 38167013, 48016045, 45757020, 26256748, 14740496, 36818162, 43284365, 29637839, 30820213, 535748, 31611626 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2005-07-27") } ] }
-{ "id": 11162977, "id-copy": 11162977, "alias": "Orson", "name": "OrsonFlick", "user-since": datetime("2010-02-17T21:05:53.000Z"), "user-since-copy": datetime("2010-02-17T21:05:53.000Z"), "friend-ids": {{ 12213318, 19062680, 20035734, 5154338, 24649936, 30379574, 38611249, 36143038, 13393939, 14976281, 34963200, 4510968, 45722224, 18820241 }}, "employment": [ { "organization-name": "Strongtone", "start-date": date("2001-03-14"), "end-date": date("2001-10-15") } ] }
-{ "id": 11174689, "id-copy": 11174689, "alias": "Thao", "name": "ThaoBrandenburg", "user-since": datetime("2012-04-21T05:25:58.000Z"), "user-since-copy": datetime("2012-04-21T05:25:58.000Z"), "friend-ids": {{ 37540210, 3918403, 33043564, 33664166 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2001-08-22"), "end-date": date("2004-11-19") } ] }
-{ "id": 11175613, "id-copy": 11175613, "alias": "Cuthbert", "name": "CuthbertHoover", "user-since": datetime("2008-04-25T01:12:49.000Z"), "user-since-copy": datetime("2008-04-25T01:12:49.000Z"), "friend-ids": {{ 27333562, 43896730, 6549030, 19576014, 4728367, 15430069, 22146931, 44593208, 14070342, 27801009, 6735368, 35798322, 47213791, 2388166 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2004-07-18") } ] }
-{ "id": 11188879, "id-copy": 11188879, "alias": "Corrie", "name": "CorrieOsterwise", "user-since": datetime("2011-01-20T21:11:19.000Z"), "user-since-copy": datetime("2011-01-20T21:11:19.000Z"), "friend-ids": {{ 47499393, 41394452, 27330253, 14958477, 14558879, 47694640, 28440147, 3437209, 40720108, 26390443 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2007-08-28") } ] }
-{ "id": 11209297, "id-copy": 11209297, "alias": "Merlin", "name": "MerlinLambert", "user-since": datetime("2012-07-01T09:30:07.000Z"), "user-since-copy": datetime("2012-07-01T09:30:07.000Z"), "friend-ids": {{ 28451212, 22119974, 1386726, 20860479, 37160852, 38281524, 17165711, 41076637, 19118162 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2012-06-26"), "end-date": date("2012-06-09") } ] }
-{ "id": 11216260, "id-copy": 11216260, "alias": "Randy", "name": "RandyEckhardstein", "user-since": datetime("2006-12-05T07:09:34.000Z"), "user-since-copy": datetime("2006-12-05T07:09:34.000Z"), "friend-ids": {{ 39744737, 14315897, 1342674, 1761832, 41393930, 21351330, 17845632, 39034426, 15297881, 11656496, 11376855 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2009-12-19") } ] }
-{ "id": 11224090, "id-copy": 11224090, "alias": "Alayna", "name": "AlaynaHay", "user-since": datetime("2008-12-27T11:44:03.000Z"), "user-since-copy": datetime("2008-12-27T11:44:03.000Z"), "friend-ids": {{ 9220004, 31827642, 27616881, 26175415, 43152043, 36272681, 669731, 40783516, 31718359, 47123044, 24487696, 31178381, 39602057, 2619975, 27562896, 29215321, 35104306, 909466, 18897009, 35295634 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2003-02-01"), "end-date": date("2007-02-07") } ] }
-{ "id": 11226055, "id-copy": 11226055, "alias": "Tony", "name": "TonyBowman", "user-since": datetime("2011-06-27T19:37:38.000Z"), "user-since-copy": datetime("2011-06-27T19:37:38.000Z"), "friend-ids": {{ 38143523, 845148, 17273955, 5476646, 28032520, 29082922, 26004648, 7037738, 34413190, 22897549, 19873990, 22338498, 10902206, 43469888, 21968875, 5127825, 11962760, 43764181, 20623302, 23901531, 3402018, 15386752, 30847912, 205201 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-09-13"), "end-date": date("2011-01-10") } ] }
-{ "id": 11250445, "id-copy": 11250445, "alias": "Charlie", "name": "CharlieHaynes", "user-since": datetime("2009-06-08T22:50:05.000Z"), "user-since-copy": datetime("2009-06-08T22:50:05.000Z"), "friend-ids": {{ 18548568, 33185990, 25924893, 44738376, 17285644, 30895698, 40664753, 45663520, 13757940, 46543434, 27472319, 7112791, 45257808, 29363383, 24726693, 39990597, 36277676, 6623887, 42795972, 29019649, 22035134, 1362080, 9071131 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-02-21"), "end-date": date("2009-12-28") } ] }
-{ "id": 11268778, "id-copy": 11268778, "alias": "Chuck", "name": "ChuckRamos", "user-since": datetime("2005-09-24T12:19:57.000Z"), "user-since-copy": datetime("2005-09-24T12:19:57.000Z"), "friend-ids": {{ 2142650, 15399676, 40659179, 32507535, 32269323, 46947373, 46293990, 4237301, 41447393, 21345670, 47299716, 8515646, 27204593, 6676856, 21757183, 13647535, 28951520, 23198255, 1618106, 18189425, 46835891, 7056692, 26622607 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2004-06-24"), "end-date": date("2006-01-05") } ] }
-{ "id": 11269867, "id-copy": 11269867, "alias": "Bettye", "name": "BettyeTeagarden", "user-since": datetime("2006-02-15T08:28:04.000Z"), "user-since-copy": datetime("2006-02-15T08:28:04.000Z"), "friend-ids": {{ 3227122, 9086278, 26175058, 16380287, 15179776, 6343969, 15198730, 7420831, 38504400, 5337815, 35914644, 42885098, 2521174, 43359140, 17884442, 3131060, 35723204, 14956242, 78003, 7455524, 3371831, 46465463, 9947087 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2000-07-21"), "end-date": date("2007-10-28") } ] }
-{ "id": 11272591, "id-copy": 11272591, "alias": "Caris", "name": "CarisCatleay", "user-since": datetime("2007-01-27T07:35:12.000Z"), "user-since-copy": datetime("2007-01-27T07:35:12.000Z"), "friend-ids": {{ 26014944 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2012-07-15"), "end-date": date("2012-07-01") } ] }
-{ "id": 11306677, "id-copy": 11306677, "alias": "Chong", "name": "ChongPawle", "user-since": datetime("2007-09-13T00:31:41.000Z"), "user-since-copy": datetime("2007-09-13T00:31:41.000Z"), "friend-ids": {{ 11341417, 23669364, 41504484, 29889550, 268223, 26888454, 43915376, 23795433, 14021648, 25630355, 19831181, 15828987 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2011-01-06"), "end-date": date("2011-10-06") } ] }
-{ "id": 11309383, "id-copy": 11309383, "alias": "Lyn", "name": "LynKnapp", "user-since": datetime("2010-07-21T15:29:58.000Z"), "user-since-copy": datetime("2010-07-21T15:29:58.000Z"), "friend-ids": {{ 27610153 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2012-08-28"), "end-date": date("2012-08-29") } ] }
-{ "id": 11335972, "id-copy": 11335972, "alias": "Emmett", "name": "EmmettBaxter", "user-since": datetime("2008-04-25T01:22:30.000Z"), "user-since-copy": datetime("2008-04-25T01:22:30.000Z"), "friend-ids": {{ 23133373, 28796661, 13045317, 34201656, 44749284, 42654826, 988887, 5039257, 18280226, 30366668, 22387991, 32676638, 24149069, 6307083, 17556069, 16687473, 4101198, 41964241, 39245728 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-11-22") } ] }
-{ "id": 11347261, "id-copy": 11347261, "alias": "Linda", "name": "LindaBaldwin", "user-since": datetime("2010-04-21T08:05:44.000Z"), "user-since-copy": datetime("2010-04-21T08:05:44.000Z"), "friend-ids": {{ 1423464, 7534626, 19522889, 25132532, 19933077, 36713596, 31725151, 46644015, 17758352, 37356325, 43714985, 29437022, 21616894, 32487769, 18527683, 32632034, 5598064, 47187635, 23490346 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2005-06-22"), "end-date": date("2007-02-18") } ] }
-{ "id": 11348356, "id-copy": 11348356, "alias": "Chery", "name": "CherySandford", "user-since": datetime("2011-04-23T21:22:21.000Z"), "user-since-copy": datetime("2011-04-23T21:22:21.000Z"), "friend-ids": {{ 14076544, 42221517 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2000-07-25") } ] }
-{ "id": 11350432, "id-copy": 11350432, "alias": "Fletcher", "name": "FletcherRowley", "user-since": datetime("2012-01-22T12:30:57.000Z"), "user-since-copy": datetime("2012-01-22T12:30:57.000Z"), "friend-ids": {{ 43655299, 46172971, 29175610, 22537183, 30612976, 21304031, 40531272, 6719806, 42232806, 18593968, 29334159 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2002-02-17"), "end-date": date("2011-03-16") } ] }
-{ "id": 11403742, "id-copy": 11403742, "alias": "Neil", "name": "NeilHobbs", "user-since": datetime("2012-02-26T07:07:17.000Z"), "user-since-copy": datetime("2012-02-26T07:07:17.000Z"), "friend-ids": {{ 28387528, 39844931, 32868894, 45540524, 35239986, 44255870, 20859099 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2008-11-28"), "end-date": date("2009-06-01") } ] }
-{ "id": 11405905, "id-copy": 11405905, "alias": "Maria", "name": "MariaMoore", "user-since": datetime("2010-05-22T22:23:16.000Z"), "user-since-copy": datetime("2010-05-22T22:23:16.000Z"), "friend-ids": {{ 31883861, 37245457, 28570944, 34781997, 8502652, 44653970, 20757487, 13575261, 13950179, 14347829, 35701908, 35781889, 12226908, 35939258, 5106463, 43910072, 10696743, 21876393, 2309465, 1889615 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2008-03-27") } ] }
-{ "id": 11417455, "id-copy": 11417455, "alias": "Malka", "name": "MalkaWilkinson", "user-since": datetime("2012-04-11T17:22:49.000Z"), "user-since-copy": datetime("2012-04-11T17:22:49.000Z"), "friend-ids": {{ 29261780, 13274200, 41060932, 8851180, 34769837, 3296096, 19488423, 41776348, 44518076, 16669411, 19983817, 26799511, 16166476, 31396373, 4090033, 37968801, 36665813 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2004-03-12") } ] }
-{ "id": 11423752, "id-copy": 11423752, "alias": "Eliott", "name": "EliottRoche", "user-since": datetime("2007-07-01T04:36:16.000Z"), "user-since-copy": datetime("2007-07-01T04:36:16.000Z"), "friend-ids": {{ 34273508, 10643569, 13667612, 19808579, 46658485, 43209365, 7962014, 24567991, 25086057 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2005-11-26") } ] }
-{ "id": 11424097, "id-copy": 11424097, "alias": "Vernie", "name": "VernieWynter", "user-since": datetime("2009-02-15T02:35:16.000Z"), "user-since-copy": datetime("2009-02-15T02:35:16.000Z"), "friend-ids": {{ 41874621, 26330221, 38930134, 39892396, 42859035, 8165423, 36128938, 5692990, 28144348, 40741492 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2002-04-06") } ] }
-{ "id": 11437771, "id-copy": 11437771, "alias": "Brittani", "name": "BrittaniMoore", "user-since": datetime("2007-11-16T20:56:35.000Z"), "user-since-copy": datetime("2007-11-16T20:56:35.000Z"), "friend-ids": {{ 30502334, 18483492, 37360877, 25153720, 9181228, 28352241, 37928337, 13522608, 20974146, 30187156, 22832401, 20899789, 44606652, 3333090, 39581573, 34303132, 33802071, 27053375, 32467186, 40213342, 37254307, 7275338, 2622767 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2010-02-07") } ] }
-{ "id": 11463820, "id-copy": 11463820, "alias": "Gaye", "name": "GayeWelty", "user-since": datetime("2005-01-04T14:32:34.000Z"), "user-since-copy": datetime("2005-01-04T14:32:34.000Z"), "friend-ids": {{ 44428980, 1291384, 10830264, 2433795, 17582948, 17416624, 21578025, 14538036, 41470487, 34384402, 42863727, 35119046, 35673193, 14814350, 29380258, 30253821, 41180218, 13945680, 15533641, 26510747 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2011-03-01"), "end-date": date("2011-09-13") } ] }
-{ "id": 11489143, "id-copy": 11489143, "alias": "Clover", "name": "CloverWest", "user-since": datetime("2012-04-14T13:56:22.000Z"), "user-since-copy": datetime("2012-04-14T13:56:22.000Z"), "friend-ids": {{ 14606516, 25835971, 10555192, 4853088, 43631398, 45670230, 43866490, 25690294, 22040370, 7047997, 3374421, 34831455, 31517002, 2998558, 40893307, 40067725, 1601716, 43041725, 8953042, 33848939 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2005-02-03"), "end-date": date("2006-06-26") } ] }
-{ "id": 11518480, "id-copy": 11518480, "alias": "Amada", "name": "AmadaTanner", "user-since": datetime("2006-05-06T12:27:31.000Z"), "user-since-copy": datetime("2006-05-06T12:27:31.000Z"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-04-02") } ] }
-{ "id": 11525302, "id-copy": 11525302, "alias": "Marissa", "name": "MarissaEndsley", "user-since": datetime("2006-09-26T08:55:36.000Z"), "user-since-copy": datetime("2006-09-26T08:55:36.000Z"), "friend-ids": {{ 35476434, 12502442, 19198691, 35401830, 14414490, 11372357, 28886265, 3490052, 13587860, 8127851, 20732439, 44816539, 6616740, 12785784, 16907259, 10942007, 26207, 21026660, 39284170, 25761798, 20688453, 45805952, 15912564 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-10-07"), "end-date": date("2010-09-09") } ] }
-{ "id": 11559262, "id-copy": 11559262, "alias": "Herb", "name": "HerbPaul", "user-since": datetime("2011-04-09T22:23:26.000Z"), "user-since-copy": datetime("2011-04-09T22:23:26.000Z"), "friend-ids": {{ 46915837, 26659094 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2011-07-05"), "end-date": date("2011-07-07") } ] }
-{ "id": 11570386, "id-copy": 11570386, "alias": "Hollis", "name": "HollisIseman", "user-since": datetime("2009-07-11T12:26:25.000Z"), "user-since-copy": datetime("2009-07-11T12:26:25.000Z"), "friend-ids": {{ 28136044, 6945424, 35390131, 12649451, 38331381, 30399822, 47834313 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2011-02-12") } ] }
-{ "id": 11570617, "id-copy": 11570617, "alias": "Deshawn", "name": "DeshawnBashline", "user-since": datetime("2006-04-14T01:05:38.000Z"), "user-since-copy": datetime("2006-04-14T01:05:38.000Z"), "friend-ids": {{ 9319940, 45556479, 44222390, 22928539, 27909778, 21162548, 8657905, 15375082, 38338906, 21416203, 7519884, 30405265, 32148274, 35560776, 29973785, 19277384, 44256954, 40425041, 30328494, 39977803, 40280359, 3079013, 18841024, 23001903 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2009-02-14") } ] }
-{ "id": 11571085, "id-copy": 11571085, "alias": "Reina", "name": "ReinaWheeler", "user-since": datetime("2010-04-28T08:05:29.000Z"), "user-since-copy": datetime("2010-04-28T08:05:29.000Z"), "friend-ids": {{ 25357083, 40592075, 10585644, 33173927, 42515085 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2000-08-03") } ] }
-{ "id": 11571217, "id-copy": 11571217, "alias": "Modesto", "name": "ModestoPark", "user-since": datetime("2006-01-18T06:28:01.000Z"), "user-since-copy": datetime("2006-01-18T06:28:01.000Z"), "friend-ids": {{ 3765450, 13287809, 17696557, 32161653, 46823306, 2818286, 38794110, 24894266, 33129431, 26474332, 9356762, 38679272, 40502952, 34470547, 30005230, 32074010, 38611550 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2001-09-01"), "end-date": date("2003-04-11") } ] }
-{ "id": 11595592, "id-copy": 11595592, "alias": "Bert", "name": "BertAtkinson", "user-since": datetime("2011-09-03T07:24:42.000Z"), "user-since-copy": datetime("2011-09-03T07:24:42.000Z"), "friend-ids": {{ 36724561, 45824456, 33567747, 21400268, 11419574, 47463040, 6480088, 45216774, 26857982, 7140352, 1884512, 29610211, 2626672, 41371388, 43582371, 42445087, 14734124, 3580372, 40134022 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2006-06-27"), "end-date": date("2007-06-07") } ] }
-{ "id": 11617963, "id-copy": 11617963, "alias": "Sherry", "name": "SherryPirl", "user-since": datetime("2010-08-26T06:37:30.000Z"), "user-since-copy": datetime("2010-08-26T06:37:30.000Z"), "friend-ids": {{ 30179664, 7140787, 14622079, 5810238, 32189583, 17103583 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2000-02-07"), "end-date": date("2004-11-24") } ] }
-{ "id": 11637820, "id-copy": 11637820, "alias": "Aislin", "name": "AislinPyle", "user-since": datetime("2005-01-04T00:11:51.000Z"), "user-since-copy": datetime("2005-01-04T00:11:51.000Z"), "friend-ids": {{ 17232277, 46376966, 22503632, 14771156, 37550654, 3930020, 7116826, 38303815, 30210948, 10532544, 44382464, 32051602 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2004-05-06") } ] }
-{ "id": 11642026, "id-copy": 11642026, "alias": "Brenden", "name": "BrendenLucy", "user-since": datetime("2010-09-18T13:14:17.000Z"), "user-since-copy": datetime("2010-09-18T13:14:17.000Z"), "friend-ids": {{ 4037044, 13420154, 10023579, 7611523, 10090302, 36514218, 24369151, 10481696, 341494 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-07-05") } ] }
-{ "id": 11646016, "id-copy": 11646016, "alias": "Millard", "name": "MillardCribbs", "user-since": datetime("2012-07-01T13:28:56.000Z"), "user-since-copy": datetime("2012-07-01T13:28:56.000Z"), "friend-ids": {{ 29358027, 24800104, 1146956, 29116484, 12223225, 6324161, 46576675 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2004-04-28") } ] }
-{ "id": 11670331, "id-copy": 11670331, "alias": "Deetta", "name": "DeettaCrom", "user-since": datetime("2008-04-01T00:12:47.000Z"), "user-since-copy": datetime("2008-04-01T00:12:47.000Z"), "friend-ids": {{ 34871046, 45366633, 40484162, 45505621, 47279131, 5464046, 18435436, 24937987, 18253019, 5870229, 46379232, 13988659, 37921800, 2085103, 21652843, 4802881, 11658526, 40771399, 32938488, 8409007, 27179341, 4496744 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2003-10-21"), "end-date": date("2008-06-06") } ] }
-{ "id": 11670739, "id-copy": 11670739, "alias": "Rudyard", "name": "RudyardErrett", "user-since": datetime("2005-03-08T18:26:12.000Z"), "user-since-copy": datetime("2005-03-08T18:26:12.000Z"), "friend-ids": {{ 13253132, 38903405, 45479471, 11551894, 44803858, 34016119, 2477206, 27909363, 2584557, 29078732, 13687500, 1038800, 14467502, 3369722, 11731177, 15702876, 37034289, 21943459 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-01-20") } ] }
-{ "id": 11672578, "id-copy": 11672578, "alias": "Juli", "name": "JuliMcclymonds", "user-since": datetime("2010-07-17T13:53:57.000Z"), "user-since-copy": datetime("2010-07-17T13:53:57.000Z"), "friend-ids": {{ 16548983, 7350585, 44497037 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2003-05-23"), "end-date": date("2009-08-01") } ] }
-{ "id": 11678242, "id-copy": 11678242, "alias": "Andy", "name": "AndyPritchard", "user-since": datetime("2008-05-26T06:52:12.000Z"), "user-since-copy": datetime("2008-05-26T06:52:12.000Z"), "friend-ids": {{ 24351029, 7396495, 11653891, 24314059, 17256129, 19177689, 23024021, 15135862, 9201238, 24204194 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2004-02-06"), "end-date": date("2011-10-22") } ] }
-{ "id": 11697754, "id-copy": 11697754, "alias": "Jeanette", "name": "JeanetteBullard", "user-since": datetime("2005-11-20T09:56:59.000Z"), "user-since-copy": datetime("2005-11-20T09:56:59.000Z"), "friend-ids": {{ 22439123, 42241829, 21396058, 6050318, 4951741, 4940964, 22719195, 21108984, 1496059, 41986346, 20838301, 34979646, 19524886, 6383593, 37747505, 26787944, 45486736, 7537516 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2006-02-20") } ] }
-{ "id": 11755633, "id-copy": 11755633, "alias": "Amina", "name": "AminaBurkett", "user-since": datetime("2012-03-22T02:05:59.000Z"), "user-since-copy": datetime("2012-03-22T02:05:59.000Z"), "friend-ids": {{ 18177270, 40223354, 29458819, 37905784, 43047863, 2679271, 9768971, 32443429, 37829920, 35493852, 28086857, 11910843, 31003179, 40873211, 42786132, 44388462 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2009-11-21"), "end-date": date("2011-03-16") } ] }
-{ "id": 11780581, "id-copy": 11780581, "alias": "Simona", "name": "SimonaDrumm", "user-since": datetime("2010-09-10T00:03:56.000Z"), "user-since-copy": datetime("2010-09-10T00:03:56.000Z"), "friend-ids": {{ 14930223, 14107902, 18276584, 12824637, 44738306, 252529, 17504815, 26802467, 33312123, 15516170, 9060069, 42300993, 15746839, 61844, 1966381, 31284798, 40145954, 31282156, 15764470, 9894586, 41833755 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-03-27") } ] }
-{ "id": 11788096, "id-copy": 11788096, "alias": "Camie", "name": "CamieCressman", "user-since": datetime("2007-10-25T23:38:14.000Z"), "user-since-copy": datetime("2007-10-25T23:38:14.000Z"), "friend-ids": {{ 29310801, 37328820, 47367940, 36796774, 21244245, 7126676, 8254586, 47578674, 39514952, 33623672, 12854915, 6679164, 44128364, 44434013, 20530444, 12243267 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-06-20") } ] }
-{ "id": 11791471, "id-copy": 11791471, "alias": "Robt", "name": "RobtChristman", "user-since": datetime("2009-08-08T21:01:18.000Z"), "user-since-copy": datetime("2009-08-08T21:01:18.000Z"), "friend-ids": {{ 9265036, 17976405, 32435071, 7236713, 21936800, 42691957, 35478695, 40052609, 14063303, 43864025, 1254074, 39237113, 11307270, 37061951, 17360733, 21102633, 21364546, 35445000, 44857867 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-01-02"), "end-date": date("2010-05-19") } ] }
-{ "id": 11809528, "id-copy": 11809528, "alias": "Donya", "name": "DonyaNash", "user-since": datetime("2008-06-09T09:42:48.000Z"), "user-since-copy": datetime("2008-06-09T09:42:48.000Z"), "friend-ids": {{ 25365000, 20270987, 39083310, 16364767, 1960249, 39747742, 17169019, 780802, 37012712, 27956954, 35502958, 10600365, 38247667, 47815777, 25182855, 13670701, 27795853, 24952265 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-10-15") } ] }
-{ "id": 11830822, "id-copy": 11830822, "alias": "Lincoln", "name": "LincolnFuchs", "user-since": datetime("2008-01-22T19:08:51.000Z"), "user-since-copy": datetime("2008-01-22T19:08:51.000Z"), "friend-ids": {{ 29014579, 29789039, 2225447, 37872940, 37026231, 3223799, 40601178 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2006-01-14"), "end-date": date("2010-04-24") } ] }
-{ "id": 11867464, "id-copy": 11867464, "alias": "Emmerson", "name": "EmmersonMoore", "user-since": datetime("2006-12-26T00:15:40.000Z"), "user-since-copy": datetime("2006-12-26T00:15:40.000Z"), "friend-ids": {{ 5310233, 16498267, 12436996, 24801626, 44135326, 45729147, 6922158, 25920138, 16324404, 30272475, 22873357, 720070, 9722837, 29718785, 5402637, 287196, 32557949 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2007-06-16"), "end-date": date("2007-02-05") } ] }
-{ "id": 11912419, "id-copy": 11912419, "alias": "Wallis", "name": "WallisFuchs", "user-since": datetime("2012-01-07T08:13:18.000Z"), "user-since-copy": datetime("2012-01-07T08:13:18.000Z"), "friend-ids": {{ 11115387, 19639311, 33957302, 8746808, 20140328, 35866755, 29492622, 24246926, 14412186, 1610423, 1139443, 23667812, 6972455, 18354247, 7072427, 43742595, 20711654, 7179925, 66544, 12066267, 8914321, 35602734 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2008-10-23"), "end-date": date("2008-06-18") } ] }
-{ "id": 11918764, "id-copy": 11918764, "alias": "Jamison", "name": "JamisonKnight", "user-since": datetime("2012-02-28T12:46:09.000Z"), "user-since-copy": datetime("2012-02-28T12:46:09.000Z"), "friend-ids": {{ 5296309, 37783012, 18620712, 8255206, 10270999, 47361618, 39691488, 33528430, 22926601, 12751125, 34000354, 32638692, 19461108, 9760202, 30157968, 265361, 24683869, 19612648, 29021437, 40094162 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2003-04-16"), "end-date": date("2011-08-28") } ] }
-{ "id": 11934781, "id-copy": 11934781, "alias": "Titus", "name": "TitusGertraht", "user-since": datetime("2011-05-02T12:41:28.000Z"), "user-since-copy": datetime("2011-05-02T12:41:28.000Z"), "friend-ids": {{ 32699552, 17016611, 46281182, 32515791, 12860342, 22463323, 33042577, 4477908, 37152051, 5462628, 45666108, 42424199, 44831639, 44546969, 30686685, 40580034 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2000-04-16") } ] }
-{ "id": 11945014, "id-copy": 11945014, "alias": "Lavern", "name": "LavernRahl", "user-since": datetime("2005-08-13T08:07:58.000Z"), "user-since-copy": datetime("2005-08-13T08:07:58.000Z"), "friend-ids": {{ 15127940, 37543274, 13877909, 8961585, 13712343, 38178056, 21469501, 2994082, 24368304, 33508930, 41765591, 37858577, 42295002 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-07-20") } ] }
-{ "id": 11951098, "id-copy": 11951098, "alias": "Tera", "name": "TeraByers", "user-since": datetime("2012-08-03T19:41:26.000Z"), "user-since-copy": datetime("2012-08-03T19:41:26.000Z"), "friend-ids": {{ 15537238, 13699967, 10587728, 23542817, 12703626, 25024772, 19223339, 5547239, 42576945, 27351017, 22726496, 25268071, 4361323, 24631578, 38669047, 44781738, 34646381 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2008-01-04"), "end-date": date("2011-01-14") } ] }
-{ "id": 11957011, "id-copy": 11957011, "alias": "Frannie", "name": "FrannieRoose", "user-since": datetime("2007-04-05T18:00:20.000Z"), "user-since-copy": datetime("2007-04-05T18:00:20.000Z"), "friend-ids": {{ 9114095, 4905395, 41862236, 21901856, 39479601, 4025127, 1517878, 16698416, 10853001, 18625728, 15395201, 17825510, 40384476, 18779630, 1832149, 41381869, 40010653, 21121933, 18598397, 12806945, 11465558 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2006-12-22") } ] }
-{ "id": 11965318, "id-copy": 11965318, "alias": "Donella", "name": "DonellaPriebe", "user-since": datetime("2010-10-25T19:45:41.000Z"), "user-since-copy": datetime("2010-10-25T19:45:41.000Z"), "friend-ids": {{ 40521325 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2010-11-19"), "end-date": date("2011-08-18") } ] }
+{ "id": 9005038, "id-copy": 9005038, "alias": "Anabel", "name": "AnabelWheeler", "user-since": datetime("2006-12-12T13:40:23.000"), "user-since-copy": datetime("2006-12-12T13:40:23.000"), "friend-ids": {{ 18713256, 35193719, 42245821, 37249622, 12210708, 15557948, 467039, 43997520, 45171035, 43682410, 47884198, 43102086, 39620955, 36438278, 42976932, 11158113, 21543594, 9861181, 36944403, 47928849, 29593861, 37897057, 42360015, 27956902 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2000-03-21") } ] }
+{ "id": 9008185, "id-copy": 9008185, "alias": "Francene", "name": "FranceneZoucks", "user-since": datetime("2009-10-18T08:37:00.000"), "user-since-copy": datetime("2009-10-18T08:37:00.000"), "friend-ids": {{ 47321113, 34578577, 25011033, 19259482, 6221464, 4912987, 20361608, 27957639, 33209653, 46928253, 37111867, 11534180, 31643335, 39967918, 8490889, 23713207, 28827713, 22143989, 21710696, 3545622, 13887489, 41557233, 26554092 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2004-02-01"), "end-date": date("2011-10-10") } ] }
+{ "id": 9012382, "id-copy": 9012382, "alias": "Laureen", "name": "LaureenOneal", "user-since": datetime("2009-12-10T22:17:58.000"), "user-since-copy": datetime("2009-12-10T22:17:58.000"), "friend-ids": {{ 25012654, 4572832, 38401260, 3015853, 42975956, 16328675, 39626774, 26936410, 15112607, 3302431 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2007-11-10") } ] }
+{ "id": 9012778, "id-copy": 9012778, "alias": "Godfrey", "name": "GodfreyBraun", "user-since": datetime("2010-03-18T19:15:53.000"), "user-since-copy": datetime("2010-03-18T19:15:53.000"), "friend-ids": {{ 3867712, 22078166 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2005-10-02") } ] }
+{ "id": 9042022, "id-copy": 9042022, "alias": "Fran", "name": "FranIronmonger", "user-since": datetime("2006-05-22T03:51:10.000"), "user-since-copy": datetime("2006-05-22T03:51:10.000"), "friend-ids": {{ 38546356, 31805246 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2002-06-06") } ] }
+{ "id": 9102208, "id-copy": 9102208, "alias": "Lottie", "name": "LottieReddish", "user-since": datetime("2007-05-22T00:42:45.000"), "user-since-copy": datetime("2007-05-22T00:42:45.000"), "friend-ids": {{ 45227463, 22488433, 39033954, 40377121, 17357169, 8890953, 1623690, 11657739, 489001, 26227491, 29459012, 39985553, 3584598, 6381312, 22457740, 43317482, 40035088, 29397671, 18293877, 6788834, 44860241 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2009-04-08") } ] }
+{ "id": 9129220, "id-copy": 9129220, "alias": "Lessie", "name": "LessieGoodman", "user-since": datetime("2008-09-01T06:07:35.000"), "user-since-copy": datetime("2008-09-01T06:07:35.000"), "friend-ids": {{ 16418186, 35990435, 22056439, 36479650, 36405609, 12039460, 33551878, 10736746, 41967761, 20046069, 8949956, 26571267 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2004-10-23"), "end-date": date("2011-05-08") } ] }
+{ "id": 9142198, "id-copy": 9142198, "alias": "Sherry", "name": "SherryFea", "user-since": datetime("2011-03-28T23:09:22.000"), "user-since-copy": datetime("2011-03-28T23:09:22.000"), "friend-ids": {{ 6835080, 34471872, 30942941, 34858577, 5996593, 47293442, 43097072, 44809621, 33969893, 26410931, 6628186, 29944391, 35957320, 20326929, 40284077, 11681583, 43878314, 40265961, 16871274, 28406169, 1349311 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2004-07-28") } ] }
+{ "id": 9185848, "id-copy": 9185848, "alias": "Brendon", "name": "BrendonJelliman", "user-since": datetime("2008-10-13T17:36:00.000"), "user-since-copy": datetime("2008-10-13T17:36:00.000"), "friend-ids": {{ 12675636, 6787931, 19218962, 12655930 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2008-06-09"), "end-date": date("2009-10-16") } ] }
+{ "id": 9199078, "id-copy": 9199078, "alias": "Erwin", "name": "ErwinErrett", "user-since": datetime("2011-04-20T12:44:31.000"), "user-since-copy": datetime("2011-04-20T12:44:31.000"), "friend-ids": {{ 31928109, 8101864, 44247743, 21370948 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2000-03-06") } ] }
+{ "id": 9207832, "id-copy": 9207832, "alias": "Tammy", "name": "TammyHozier", "user-since": datetime("2005-08-24T14:34:19.000"), "user-since-copy": datetime("2005-08-24T14:34:19.000"), "friend-ids": {{ 26919119, 35729176, 28949827 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2006-09-14") } ] }
+{ "id": 9211711, "id-copy": 9211711, "alias": "Seraphina", "name": "SeraphinaFlanders", "user-since": datetime("2009-05-19T18:39:15.000"), "user-since-copy": datetime("2009-05-19T18:39:15.000"), "friend-ids": {{ 34432294, 10796959, 46386746, 32318131, 10393677, 12832313, 34490791, 6187782, 46595448, 30591963, 35530646, 22485004, 18950892, 19762388, 19181134, 13928403, 22513246, 24969298 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2008-05-14"), "end-date": date("2009-06-17") } ] }
+{ "id": 9212815, "id-copy": 9212815, "alias": "Erica", "name": "EricaBraun", "user-since": datetime("2009-01-11T07:32:03.000"), "user-since-copy": datetime("2009-01-11T07:32:03.000"), "friend-ids": {{ 1314906, 6581233, 35117578, 11133528, 19606776, 37833518, 40040803, 44107209, 38804989, 35779440, 41138709 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2008-04-03") } ] }
+{ "id": 9243769, "id-copy": 9243769, "alias": "Florentino", "name": "FlorentinoRiggle", "user-since": datetime("2012-04-04T17:10:31.000"), "user-since-copy": datetime("2012-04-04T17:10:31.000"), "friend-ids": {{ 41929020, 22354873 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2007-07-23") } ] }
+{ "id": 9265747, "id-copy": 9265747, "alias": "Nicolas", "name": "NicolasPirl", "user-since": datetime("2011-11-07T13:52:49.000"), "user-since-copy": datetime("2011-11-07T13:52:49.000"), "friend-ids": {{ 5832017, 30839617, 27328653, 9766355, 35973149, 21029594, 18840511, 43035135, 44902336, 11576374, 21756219, 23374243, 42201568, 12860309 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2003-11-10"), "end-date": date("2010-03-27") } ] }
+{ "id": 9311659, "id-copy": 9311659, "alias": "Kate", "name": "KateBender", "user-since": datetime("2007-06-10T05:55:50.000"), "user-since-copy": datetime("2007-06-10T05:55:50.000"), "friend-ids": {{ 27875958, 10379355, 4286877, 26410945, 10609943, 15960135 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-01-04") } ] }
+{ "id": 9318094, "id-copy": 9318094, "alias": "Carlo", "name": "CarloKelley", "user-since": datetime("2012-07-19T09:18:41.000"), "user-since-copy": datetime("2012-07-19T09:18:41.000"), "friend-ids": {{ 39873731, 29304807, 519851, 16423529, 10838418, 9915172, 3040071, 39730361, 23320290, 20572900, 7293676, 35037765, 1744053, 38875858 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2008-08-15") } ] }
+{ "id": 9326218, "id-copy": 9326218, "alias": "Lindsay", "name": "LindsayPaynter", "user-since": datetime("2011-08-27T00:03:13.000"), "user-since-copy": datetime("2011-08-27T00:03:13.000"), "friend-ids": {{ 3006430, 25941368, 46866627, 21404266, 35141764, 14931901 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-04-06"), "end-date": date("2008-03-02") } ] }
+{ "id": 9343705, "id-copy": 9343705, "alias": "Ramsey", "name": "RamseyWarner", "user-since": datetime("2006-04-24T09:52:39.000"), "user-since-copy": datetime("2006-04-24T09:52:39.000"), "friend-ids": {{ 36909861, 36881715, 40993685, 18669519, 42428458, 2780280, 6070725, 10466662, 26215221, 16329040, 38464211, 14024902, 8083000, 27857433, 14282674, 1976238, 6345526, 35452338, 21503723, 34910137, 26860195, 426384, 27759959 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2003-10-28") } ] }
+{ "id": 9379330, "id-copy": 9379330, "alias": "Esther", "name": "EstherReichard", "user-since": datetime("2006-09-23T09:53:43.000"), "user-since-copy": datetime("2006-09-23T09:53:43.000"), "friend-ids": {{ 29035495, 33601969, 32342695, 28995226, 34638799, 38330225, 38512256 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2006-05-27") } ] }
+{ "id": 9379975, "id-copy": 9379975, "alias": "Kyra", "name": "KyraLangston", "user-since": datetime("2012-01-18T06:06:56.000"), "user-since-copy": datetime("2012-01-18T06:06:56.000"), "friend-ids": {{ 46662872, 1388016, 21715152, 3266023, 18080709, 25857347, 29710885, 22300787, 25086634, 25220921, 17189604, 21754574, 27820275, 7441940, 10911235, 46304871, 6518794 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2008-04-03"), "end-date": date("2008-04-07") } ] }
+{ "id": 9417499, "id-copy": 9417499, "alias": "Wendell", "name": "WendellJoyce", "user-since": datetime("2011-07-25T14:30:30.000"), "user-since-copy": datetime("2011-07-25T14:30:30.000"), "friend-ids": {{ 10079972, 29246113, 40533159, 31279768, 31969044, 46120195, 35004468, 24465042, 2702879, 44166678, 20176481, 32056309, 38254930, 20950061, 4687108 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2006-03-18") } ] }
+{ "id": 9430849, "id-copy": 9430849, "alias": "Emil", "name": "EmilGarland", "user-since": datetime("2008-07-03T15:56:07.000"), "user-since-copy": datetime("2008-07-03T15:56:07.000"), "friend-ids": {{ 40429008, 45432330, 22293451, 2129366, 19514477, 20108162, 28656704, 35403173, 33855801, 14660181 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2010-02-10") } ] }
+{ "id": 9442978, "id-copy": 9442978, "alias": "Osborne", "name": "OsborneHiles", "user-since": datetime("2012-07-28T10:59:39.000"), "user-since-copy": datetime("2012-07-28T10:59:39.000"), "friend-ids": {{ 40833026, 39533118, 6206868, 27383373, 3010465, 14776443, 43239645, 21956253, 4112089, 27667721, 34336067, 38377619, 32701403, 20907262, 32732275, 30488150, 12349697, 47468946, 20956164, 16141416 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2011-08-21") } ] }
+{ "id": 9461098, "id-copy": 9461098, "alias": "Teodoro", "name": "TeodoroBullard", "user-since": datetime("2010-07-24T07:40:44.000"), "user-since-copy": datetime("2010-07-24T07:40:44.000"), "friend-ids": {{ 8278091, 1756629, 9893864, 11184021, 2292251, 20614604, 48014557, 23491569, 11328678, 11572435, 45790306, 44930978, 34910222, 16655255, 29338869, 27169036, 19669405, 20512510, 33598988, 38104427 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2003-01-17"), "end-date": date("2007-05-28") } ] }
+{ "id": 9471385, "id-copy": 9471385, "alias": "Weldon", "name": "WeldonMaclagan", "user-since": datetime("2010-01-24T22:21:59.000"), "user-since-copy": datetime("2010-01-24T22:21:59.000"), "friend-ids": {{ 42864267, 16710494, 27436346, 7324905, 3901396, 11812437, 31490561, 3906397 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2002-09-07"), "end-date": date("2006-07-08") } ] }
+{ "id": 9477040, "id-copy": 9477040, "alias": "Chery", "name": "CheryWatson", "user-since": datetime("2012-05-02T14:27:00.000"), "user-since-copy": datetime("2012-05-02T14:27:00.000"), "friend-ids": {{ 36360097, 36835617, 25761112, 30806900, 22340413, 16802957 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2008-12-26"), "end-date": date("2009-03-17") } ] }
+{ "id": 9477994, "id-copy": 9477994, "alias": "Cory", "name": "CoryKeener", "user-since": datetime("2012-02-27T22:03:31.000"), "user-since-copy": datetime("2012-02-27T22:03:31.000"), "friend-ids": {{ 22204843, 35394804, 22795967, 16575437, 31764908, 27359073, 50023, 26383393, 36534917, 23478654, 31022293, 43803666, 24764841, 19469389, 6401330, 10543085, 5159571 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2012-02-09"), "end-date": date("2012-02-19") } ] }
+{ "id": 9481756, "id-copy": 9481756, "alias": "Esmaralda", "name": "EsmaraldaAgg", "user-since": datetime("2012-06-26T19:57:38.000"), "user-since-copy": datetime("2012-06-26T19:57:38.000"), "friend-ids": {{ 40976868 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2008-11-26"), "end-date": date("2008-01-13") } ] }
+{ "id": 9490342, "id-copy": 9490342, "alias": "Gisela", "name": "GiselaTomlinson", "user-since": datetime("2011-10-21T20:36:09.000"), "user-since-copy": datetime("2011-10-21T20:36:09.000"), "friend-ids": {{ 27609144, 42495049, 21250269, 22561106, 29149509, 16776721, 16980559, 19600765 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2003-02-23") } ] }
+{ "id": 9512971, "id-copy": 9512971, "alias": "Algar", "name": "AlgarKepplinger", "user-since": datetime("2011-10-11T02:54:01.000"), "user-since-copy": datetime("2011-10-11T02:54:01.000"), "friend-ids": {{ 1076656, 1837449, 43428033, 21710004, 41167492, 17526252 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2003-04-26"), "end-date": date("2006-02-24") } ] }
+{ "id": 9516652, "id-copy": 9516652, "alias": "Emmanuel", "name": "EmmanuelStrickland", "user-since": datetime("2006-01-14T03:08:13.000"), "user-since-copy": datetime("2006-01-14T03:08:13.000"), "friend-ids": {{ 21213113, 8011145, 9382308, 14949454, 114459, 30046906, 40091327, 22275481, 14642211, 5602065, 15265189, 22736575, 12746303, 46033445, 17273286, 39395247, 6653955, 14664612, 35055957 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-10-15") } ] }
+{ "id": 9525361, "id-copy": 9525361, "alias": "Leonardo", "name": "LeonardoSurrency", "user-since": datetime("2008-12-21T10:09:26.000"), "user-since-copy": datetime("2008-12-21T10:09:26.000"), "friend-ids": {{ 12471014, 47714763, 18071069, 32545366, 46041462, 35261185, 20826834, 29002678, 47207065, 7370034, 38283272, 47090645, 33425043, 16014552, 15633873, 24101778, 26168621, 21955493, 17856723, 18158610 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2011-12-06"), "end-date": date("2011-04-04") } ] }
+{ "id": 9532474, "id-copy": 9532474, "alias": "Chester", "name": "ChesterAshmore", "user-since": datetime("2012-02-03T20:36:34.000"), "user-since-copy": datetime("2012-02-03T20:36:34.000"), "friend-ids": {{ 11340481, 15957237, 47048138, 41603112, 6953329, 6926093, 20866295, 329274, 16187993, 13406075, 34601684, 46151089, 26165473, 2882718, 20731108 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2009-03-14") } ] }
+{ "id": 9546133, "id-copy": 9546133, "alias": "Renae", "name": "RenaeWhitehead", "user-since": datetime("2012-04-21T14:38:30.000"), "user-since-copy": datetime("2012-04-21T14:38:30.000"), "friend-ids": {{ 31261211, 19892104, 35568606, 12050300, 42512152, 37032282, 27185051 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2012-02-20"), "end-date": date("2012-07-04") } ] }
+{ "id": 9552016, "id-copy": 9552016, "alias": "Shantelle", "name": "ShantelleDealtry", "user-since": datetime("2006-05-03T06:49:13.000"), "user-since-copy": datetime("2006-05-03T06:49:13.000"), "friend-ids": {{ 35758396, 16562240, 23596680, 16342769, 19892813, 46485447, 25711418, 23765073, 11303996, 36451291, 17586370, 38010455, 29457199, 25847013, 12604123, 46533018, 26999208, 24740610, 35225441, 33613663 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2003-08-07"), "end-date": date("2003-07-17") } ] }
+{ "id": 9560251, "id-copy": 9560251, "alias": "Nivek", "name": "NivekJowers", "user-since": datetime("2007-02-04T08:02:07.000"), "user-since-copy": datetime("2007-02-04T08:02:07.000"), "friend-ids": {{ 15730417, 36745553, 26133088, 38675683, 14617495, 39244216, 4651791, 639869, 8377526, 15158817, 13368295, 15386494, 5649384, 8449938, 34497809, 6644713, 45481442, 27678941, 14214532, 5753112, 9991855, 25975202, 9530884, 19069924 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2003-08-15") } ] }
+{ "id": 9562348, "id-copy": 9562348, "alias": "Jefferson", "name": "JeffersonKeister", "user-since": datetime("2005-06-11T01:42:58.000"), "user-since-copy": datetime("2005-06-11T01:42:58.000"), "friend-ids": {{ 43801762 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2005-07-26"), "end-date": date("2011-12-02") } ] }
+{ "id": 9577729, "id-copy": 9577729, "alias": "Jann", "name": "JannPorter", "user-since": datetime("2006-05-03T08:57:08.000"), "user-since-copy": datetime("2006-05-03T08:57:08.000"), "friend-ids": {{ 7711959, 4131696, 10146353, 46418552, 37999454, 38333059, 16381326, 45028736, 16829150 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2006-10-19") } ] }
+{ "id": 9591646, "id-copy": 9591646, "alias": "Hoyt", "name": "HoytGilman", "user-since": datetime("2011-05-13T07:22:20.000"), "user-since-copy": datetime("2011-05-13T07:22:20.000"), "friend-ids": {{ 11207445 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2004-04-27") } ] }
+{ "id": 9595279, "id-copy": 9595279, "alias": "Emmaline", "name": "EmmalineSchuth", "user-since": datetime("2008-09-12T22:25:17.000"), "user-since-copy": datetime("2008-09-12T22:25:17.000"), "friend-ids": {{ 26784778, 6200196, 37440596, 12250319, 21921557, 19278082, 583040, 12012653, 21578028, 16395818, 29088493, 29578064, 37745574, 41998781, 22594273, 38002130, 2166585, 7823908, 18253304, 6162341, 40270219, 41832701, 36455204 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2009-02-13") } ] }
+{ "id": 9606691, "id-copy": 9606691, "alias": "Reva", "name": "RevaChristman", "user-since": datetime("2010-03-04T11:53:00.000"), "user-since-copy": datetime("2010-03-04T11:53:00.000"), "friend-ids": {{ 21390421 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2010-12-13") } ] }
+{ "id": 9621157, "id-copy": 9621157, "alias": "Trixie", "name": "TrixieFair", "user-since": datetime("2010-12-25T23:36:49.000"), "user-since-copy": datetime("2010-12-25T23:36:49.000"), "friend-ids": {{ 17519006, 17545060, 27836293, 11477603, 37895380, 23251592, 12010503, 25406806 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2003-09-23") } ] }
+{ "id": 9629395, "id-copy": 9629395, "alias": "Julius", "name": "JuliusWire", "user-since": datetime("2008-03-22T13:36:24.000"), "user-since-copy": datetime("2008-03-22T13:36:24.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2006-11-19") } ] }
+{ "id": 9634393, "id-copy": 9634393, "alias": "Burt", "name": "BurtPearson", "user-since": datetime("2007-11-01T14:25:29.000"), "user-since-copy": datetime("2007-11-01T14:25:29.000"), "friend-ids": {{ 26065414, 8710639, 22639162, 23787625, 24443211, 42598742, 45171006, 38246985, 25125478, 23071168, 22455706, 24720860, 34917747, 24262081, 2259812, 14262605, 37533604 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-05-07") } ] }
+{ "id": 9640915, "id-copy": 9640915, "alias": "Harrison", "name": "HarrisonHildyard", "user-since": datetime("2009-05-25T11:56:05.000"), "user-since-copy": datetime("2009-05-25T11:56:05.000"), "friend-ids": {{ 41488832, 16139664, 18327029, 38811764, 38271538, 13106137, 26450611, 11574808, 33108523, 31639017, 9208159, 18456510, 47955463, 2606160, 29293146, 13981743, 39967993, 23629640, 32666499, 35046044, 2402842, 1117025, 17741007, 14997808 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2010-03-06") } ] }
+{ "id": 9643768, "id-copy": 9643768, "alias": "Gil", "name": "GilVeith", "user-since": datetime("2006-04-26T11:42:30.000"), "user-since-copy": datetime("2006-04-26T11:42:30.000"), "friend-ids": {{ 22270431, 9614818, 9080111, 6500797, 37876717, 28122656, 13971193, 20936637, 19883735, 37455193, 32129291, 40710966, 17779823, 41523128, 41276564, 34424817, 19326867, 26058281 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2002-02-04") } ] }
+{ "id": 9669178, "id-copy": 9669178, "alias": "Gerard", "name": "GerardBeck", "user-since": datetime("2011-04-24T15:49:24.000"), "user-since-copy": datetime("2011-04-24T15:49:24.000"), "friend-ids": {{ 30087138, 44736614, 1531569 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2003-09-25"), "end-date": date("2005-06-28") } ] }
+{ "id": 9680644, "id-copy": 9680644, "alias": "Mirtha", "name": "MirthaRahl", "user-since": datetime("2008-02-09T04:05:03.000"), "user-since-copy": datetime("2008-02-09T04:05:03.000"), "friend-ids": {{ 25328638, 9009324, 16627989, 46602908, 32685062, 10538437, 22403363, 4205292, 27910567, 28430833, 8519372, 39774027, 12120028, 1211979 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2006-12-19") } ] }
+{ "id": 9683656, "id-copy": 9683656, "alias": "Antone", "name": "AntoneMays", "user-since": datetime("2006-07-24T22:48:29.000"), "user-since-copy": datetime("2006-07-24T22:48:29.000"), "friend-ids": {{ 11275116, 40325672, 41154035, 8987353, 31187312, 11505721, 11584703, 42743337, 23225356, 8653923 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2011-06-12") } ] }
+{ "id": 9695773, "id-copy": 9695773, "alias": "Daron", "name": "DaronFiddler", "user-since": datetime("2006-12-25T17:08:50.000"), "user-since-copy": datetime("2006-12-25T17:08:50.000"), "friend-ids": {{ 14397778, 33469556, 41690231, 7827360, 42196316 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2007-12-24") } ] }
+{ "id": 9707074, "id-copy": 9707074, "alias": "Melvyn", "name": "MelvynSybilla", "user-since": datetime("2012-06-07T16:06:49.000"), "user-since-copy": datetime("2012-06-07T16:06:49.000"), "friend-ids": {{ 4487400, 488933, 15650706, 44692005, 25068052, 16975927 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2010-12-13") } ] }
+{ "id": 9740008, "id-copy": 9740008, "alias": "Woodrow", "name": "WoodrowBlois", "user-since": datetime("2011-12-18T11:34:56.000"), "user-since-copy": datetime("2011-12-18T11:34:56.000"), "friend-ids": {{ 1753941, 17603348, 44569557, 6816408, 17403631, 29707555, 21215516, 9837919, 35887854, 35236051, 7897485, 9880491, 16145458, 33128036, 41471362, 44171952, 23542112, 36155237, 2596261, 36702766 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2007-08-02") } ] }
+{ "id": 9747652, "id-copy": 9747652, "alias": "Graham", "name": "GrahamGarratt", "user-since": datetime("2006-04-16T19:35:33.000"), "user-since-copy": datetime("2006-04-16T19:35:33.000"), "friend-ids": {{ 9995821, 7082678, 29813051, 33625501, 32785793, 23170533, 26581328, 35564866, 9147486, 17626916, 12721534, 22070579, 25749282, 27771492, 35217137, 6426437, 4217778, 6819045, 6410966, 43080321, 32112201, 20323505 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2005-09-26") } ] }
+{ "id": 9761152, "id-copy": 9761152, "alias": "Royle", "name": "RoyleStewart", "user-since": datetime("2010-05-15T17:14:18.000"), "user-since-copy": datetime("2010-05-15T17:14:18.000"), "friend-ids": {{ 21868661, 15545005, 11285872, 45768523, 12486235 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2002-11-20") } ] }
+{ "id": 9779623, "id-copy": 9779623, "alias": "Alberto", "name": "AlbertoCraig", "user-since": datetime("2009-11-25T14:48:04.000"), "user-since-copy": datetime("2009-11-25T14:48:04.000"), "friend-ids": {{ 6737836, 26882597, 30254391, 4861442, 18105612 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2012-01-25") } ] }
+{ "id": 9799264, "id-copy": 9799264, "alias": "Bradley", "name": "BradleyTodd", "user-since": datetime("2011-05-18T23:42:33.000"), "user-since-copy": datetime("2011-05-18T23:42:33.000"), "friend-ids": {{ 8836368, 35488923, 26777243, 46550104, 9866525, 965209 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2007-12-22") } ] }
+{ "id": 9799591, "id-copy": 9799591, "alias": "Royston", "name": "RoystonChurchill", "user-since": datetime("2011-01-21T13:57:31.000"), "user-since-copy": datetime("2011-01-21T13:57:31.000"), "friend-ids": {{ 22757950, 4629721, 19522595, 27737642, 39393176, 9321441, 13496995, 43301849, 3869585, 34993450, 24876688 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2000-07-15") } ] }
+{ "id": 9804973, "id-copy": 9804973, "alias": "Harriette", "name": "HarrietteHoopengarner", "user-since": datetime("2011-08-14T20:51:52.000"), "user-since-copy": datetime("2011-08-14T20:51:52.000"), "friend-ids": {{ 18754696, 27799194, 36904141, 29647419, 8521621, 35146470, 45194388, 43397176, 12596887, 33315, 39826335, 31228413, 123596, 35927645, 11445687, 33208186, 21941268 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2003-03-24") } ] }
+{ "id": 9819796, "id-copy": 9819796, "alias": "Emerson", "name": "EmersonWardle", "user-since": datetime("2006-08-20T20:22:11.000"), "user-since-copy": datetime("2006-08-20T20:22:11.000"), "friend-ids": {{ 5697147, 42936553, 12624322, 45309083, 10785774, 4176618 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-05-16") } ] }
+{ "id": 9820681, "id-copy": 9820681, "alias": "Caitlin", "name": "CaitlinWolfe", "user-since": datetime("2012-05-23T07:59:39.000"), "user-since-copy": datetime("2012-05-23T07:59:39.000"), "friend-ids": {{ 22005473, 7664709, 22913945, 16078115, 11724028, 45958589, 33357270, 6935384, 2696233, 28938665, 37992833, 11929142, 16203505, 20365802 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2010-06-18") } ] }
+{ "id": 9826402, "id-copy": 9826402, "alias": "Rachyl", "name": "RachylRumbaugh", "user-since": datetime("2006-01-05T03:38:59.000"), "user-since-copy": datetime("2006-01-05T03:38:59.000"), "friend-ids": {{ 11891915, 15900581, 38420311, 21084667, 24569500, 9181299, 32167823, 9967774, 18138704, 10742133, 29173609, 1113683, 21048344, 33794587, 42308958, 9303744 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2005-04-24"), "end-date": date("2008-08-17") } ] }
+{ "id": 9829834, "id-copy": 9829834, "alias": "Darryl", "name": "DarrylSullivan", "user-since": datetime("2011-07-24T00:12:33.000"), "user-since-copy": datetime("2011-07-24T00:12:33.000"), "friend-ids": {{ 8297654, 6071837, 27236382, 4657522, 9035310, 40427605, 2360931, 19796421, 7301200, 1264845, 12653555, 27518516 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2005-01-18"), "end-date": date("2010-05-20") } ] }
+{ "id": 9840013, "id-copy": 9840013, "alias": "Inger", "name": "IngerRuhl", "user-since": datetime("2009-05-27T20:14:42.000"), "user-since-copy": datetime("2009-05-27T20:14:42.000"), "friend-ids": {{ 36044692 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2010-09-15") } ] }
+{ "id": 9845113, "id-copy": 9845113, "alias": "Chia", "name": "ChiaGeddinge", "user-since": datetime("2008-12-12T16:50:57.000"), "user-since-copy": datetime("2008-12-12T16:50:57.000"), "friend-ids": {{ 16725476, 120161, 762756, 40795640, 34195102, 27938737 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2001-01-03"), "end-date": date("2001-11-03") } ] }
+{ "id": 9859726, "id-copy": 9859726, "alias": "Taryn", "name": "TarynGisiko", "user-since": datetime("2010-12-28T21:42:56.000"), "user-since-copy": datetime("2010-12-28T21:42:56.000"), "friend-ids": {{ 45036313, 47860435, 40658528, 4106429, 25411752, 7216290, 20549107, 28317961, 43600081, 6359672, 36131464, 19078372, 4379305, 884797, 11605059, 6467240, 23316141 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2007-06-26"), "end-date": date("2010-08-04") } ] }
+{ "id": 9872791, "id-copy": 9872791, "alias": "Yasmine", "name": "YasmineCanham", "user-since": datetime("2005-06-08T14:45:42.000"), "user-since-copy": datetime("2005-06-08T14:45:42.000"), "friend-ids": {{ 7340569, 16137560, 43341029, 31700386, 24881875, 17852264, 42730676, 32655012 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2004-05-09"), "end-date": date("2011-02-28") } ] }
+{ "id": 9877837, "id-copy": 9877837, "alias": "Marilee", "name": "MarileeDowning", "user-since": datetime("2007-09-06T15:02:25.000"), "user-since-copy": datetime("2007-09-06T15:02:25.000"), "friend-ids": {{ 3032720, 7000379, 16658012, 33487490, 624779, 13480315, 8308906, 6949934, 9472007, 36568244, 41737195, 1310478, 42870077, 46663613 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2005-10-09") } ] }
+{ "id": 9919033, "id-copy": 9919033, "alias": "Bailey", "name": "BaileyHay", "user-since": datetime("2005-01-06T07:43:18.000"), "user-since-copy": datetime("2005-01-06T07:43:18.000"), "friend-ids": {{ 28198532 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2009-02-08"), "end-date": date("2010-06-08") } ] }
+{ "id": 9922381, "id-copy": 9922381, "alias": "Cecilia", "name": "CeciliaOsteen", "user-since": datetime("2009-06-03T03:58:36.000"), "user-since-copy": datetime("2009-06-03T03:58:36.000"), "friend-ids": {{ 22246989, 9095240, 8953245, 16326669, 38845534, 13608449, 35076758, 42004583 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2010-08-02") } ] }
+{ "id": 9929866, "id-copy": 9929866, "alias": "Emilie", "name": "EmilieJohns", "user-since": datetime("2009-10-01T00:51:03.000"), "user-since-copy": datetime("2009-10-01T00:51:03.000"), "friend-ids": {{ 45496950, 38109555, 46259676, 14141368, 31720484, 35564907, 23226721, 36026226, 34003258, 47176035, 46593035, 5050811, 27858647, 3784968 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2012-08-10"), "end-date": date("2012-08-24") } ] }
+{ "id": 9931588, "id-copy": 9931588, "alias": "Sheri", "name": "SheriHindman", "user-since": datetime("2011-02-19T03:55:37.000"), "user-since-copy": datetime("2011-02-19T03:55:37.000"), "friend-ids": {{ 10993709, 28005344, 31884585, 1581885, 46332238, 47401902, 38814902, 39736365, 24318394, 15329318, 35794552, 14913021, 8723328, 28102869, 27218765, 21310255 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2011-08-17"), "end-date": date("2011-12-15") } ] }
+{ "id": 9937957, "id-copy": 9937957, "alias": "Corey", "name": "CoreyTaggart", "user-since": datetime("2005-11-25T16:13:03.000"), "user-since-copy": datetime("2005-11-25T16:13:03.000"), "friend-ids": {{ 40105038, 9364511, 47362703, 1876955, 3505769, 41708385, 36179634, 7022850 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2005-09-24") } ] }
+{ "id": 9945208, "id-copy": 9945208, "alias": "Thelma", "name": "ThelmaGettemy", "user-since": datetime("2006-12-21T11:17:06.000"), "user-since-copy": datetime("2006-12-21T11:17:06.000"), "friend-ids": {{ 26578648, 43730418, 18099472, 11787057, 41534206, 16778979, 41142786, 25761045, 18556835, 25378849, 38984390, 37528215, 2531696 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2008-03-25") } ] }
+{ "id": 9952339, "id-copy": 9952339, "alias": "Dacia", "name": "DaciaStaymates", "user-since": datetime("2009-09-27T09:55:51.000"), "user-since-copy": datetime("2009-09-27T09:55:51.000"), "friend-ids": {{ 5177020, 46967179, 24156959, 17828131, 41565753, 1929360, 33761670, 27544454, 9964059, 25582191 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2000-10-12"), "end-date": date("2007-01-20") } ] }
+{ "id": 9962236, "id-copy": 9962236, "alias": "Craig", "name": "CraigKight", "user-since": datetime("2010-02-15T15:58:03.000"), "user-since-copy": datetime("2010-02-15T15:58:03.000"), "friend-ids": {{ 45604304, 40911167, 39517053, 6912584, 898627, 8412812, 33530827, 30135549, 14762146, 46313211, 21143796, 39820220, 11462372, 23575315 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-02-05"), "end-date": date("2008-01-04") } ] }
+{ "id": 9968869, "id-copy": 9968869, "alias": "Shemika", "name": "ShemikaNickolson", "user-since": datetime("2005-02-20T10:34:04.000"), "user-since-copy": datetime("2005-02-20T10:34:04.000"), "friend-ids": {{ 30287118, 877645, 9968776, 31800907 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2009-07-23") } ] }
+{ "id": 9975778, "id-copy": 9975778, "alias": "Marmaduke", "name": "MarmadukeElizabeth", "user-since": datetime("2012-07-18T02:21:55.000"), "user-since-copy": datetime("2012-07-18T02:21:55.000"), "friend-ids": {{ 17424696, 34807936, 8912699, 40534595, 36049658, 31706902, 7626256, 16178188, 36944385, 47878361, 8190132, 34365280, 13576207, 42728095 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2006-08-19") } ] }
+{ "id": 9988417, "id-copy": 9988417, "alias": "Coline", "name": "ColineLane", "user-since": datetime("2010-01-01T00:12:39.000"), "user-since-copy": datetime("2010-01-01T00:12:39.000"), "friend-ids": {{ 17656229, 42804152 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2012-05-01") } ] }
+{ "id": 9993001, "id-copy": 9993001, "alias": "Herbie", "name": "HerbieStall", "user-since": datetime("2010-06-14T03:01:11.000"), "user-since-copy": datetime("2010-06-14T03:01:11.000"), "friend-ids": {{ 12003033, 40923715, 34166285, 47927261, 638933, 17338590 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2009-07-12") } ] }
+{ "id": 10054327, "id-copy": 10054327, "alias": "Poppy", "name": "PoppyKellogg", "user-since": datetime("2010-03-28T09:43:49.000"), "user-since-copy": datetime("2010-03-28T09:43:49.000"), "friend-ids": {{ 10785684, 26545687, 942400, 18147517, 12133643, 17848751, 40864121, 18975370, 26159158, 42348235, 21795276, 40155922, 35240759 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2012-03-24") } ] }
+{ "id": 10065250, "id-copy": 10065250, "alias": "Debbie", "name": "DebbieBrinigh", "user-since": datetime("2012-01-05T15:05:48.000"), "user-since-copy": datetime("2012-01-05T15:05:48.000"), "friend-ids": {{ 23794420, 31166549, 3372724, 35955226, 45241312, 33488036, 17353508, 10094234, 12751868 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2000-06-28"), "end-date": date("2005-06-03") } ] }
+{ "id": 10066711, "id-copy": 10066711, "alias": "Nichelle", "name": "NichelleErschoff", "user-since": datetime("2009-11-10T21:17:50.000"), "user-since-copy": datetime("2009-11-10T21:17:50.000"), "friend-ids": {{ 19024226, 24428716, 24428406, 10686682, 46410623, 45809403, 33158503 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2004-06-21"), "end-date": date("2005-08-01") } ] }
+{ "id": 10069987, "id-copy": 10069987, "alias": "Andrina", "name": "AndrinaFisher", "user-since": datetime("2012-07-21T07:28:30.000"), "user-since-copy": datetime("2012-07-21T07:28:30.000"), "friend-ids": {{ 42024943, 39627436, 28414443, 36703363, 45477433, 37499278, 28548620, 6687009, 22700392, 47812034, 16805789, 33222895, 36328879, 20191886, 32457353, 14008353 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2004-12-11"), "end-date": date("2004-09-07") } ] }
+{ "id": 10073440, "id-copy": 10073440, "alias": "Mat", "name": "MatHasely", "user-since": datetime("2007-02-15T12:28:32.000"), "user-since-copy": datetime("2007-02-15T12:28:32.000"), "friend-ids": {{ 18317132, 16303558, 35197704, 41199497, 17394418, 18594954, 13332602, 15164806, 20807780, 18284264, 17164369, 6418744, 26535302, 47287046, 7169299, 22825706, 34007482, 38108004, 14449725, 16993574, 28055503 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2005-09-04"), "end-date": date("2006-06-02") } ] }
+{ "id": 10079965, "id-copy": 10079965, "alias": "Mason", "name": "MasonReamer", "user-since": datetime("2008-08-10T02:16:36.000"), "user-since-copy": datetime("2008-08-10T02:16:36.000"), "friend-ids": {{ 37149190, 37736572, 35955709, 28586597, 45460389 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-01-18"), "end-date": date("2010-12-09") } ] }
+{ "id": 10087876, "id-copy": 10087876, "alias": "Carlyle", "name": "CarlyleMoberly", "user-since": datetime("2009-09-12T03:44:36.000"), "user-since-copy": datetime("2009-09-12T03:44:36.000"), "friend-ids": {{ 22254101, 16994379, 42146906, 28928982 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2012-07-24"), "end-date": date("2012-07-09") } ] }
+{ "id": 10126408, "id-copy": 10126408, "alias": "Pen", "name": "PenFleming", "user-since": datetime("2005-11-11T08:50:34.000"), "user-since-copy": datetime("2005-11-11T08:50:34.000"), "friend-ids": {{ 38072630, 45021886, 23988042, 41084533, 4743969, 7223979, 19120365, 44219284, 4691449, 21072839, 32536521, 36335527, 47376347, 16882811, 43140173, 7610811, 28217191, 25488874, 27968660, 13102347, 40169395, 25952056, 17249838, 30971677 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2011-10-04"), "end-date": date("2011-01-10") } ] }
+{ "id": 10135477, "id-copy": 10135477, "alias": "Jasmine", "name": "JasmineEva", "user-since": datetime("2009-04-03T11:48:27.000"), "user-since-copy": datetime("2009-04-03T11:48:27.000"), "friend-ids": {{ 3776073 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2000-11-14"), "end-date": date("2001-05-19") } ] }
+{ "id": 10148251, "id-copy": 10148251, "alias": "Ghislaine", "name": "GhislaineFowler", "user-since": datetime("2005-12-08T05:25:56.000"), "user-since-copy": datetime("2005-12-08T05:25:56.000"), "friend-ids": {{ 14692731, 29964772 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2008-12-27"), "end-date": date("2008-04-02") } ] }
+{ "id": 10177078, "id-copy": 10177078, "alias": "Fausto", "name": "FaustoLotherington", "user-since": datetime("2005-06-23T22:18:16.000"), "user-since-copy": datetime("2005-06-23T22:18:16.000"), "friend-ids": {{ 9405744, 13732034 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2007-12-27") } ] }
+{ "id": 10177300, "id-copy": 10177300, "alias": "Chase", "name": "ChaseKnapp", "user-since": datetime("2005-09-27T16:41:30.000"), "user-since-copy": datetime("2005-09-27T16:41:30.000"), "friend-ids": {{ 12805247, 6093464, 39416190, 35877238, 26583227, 37835412, 46337730, 18107636, 43948720, 21031949, 11688759, 13980476, 25486392, 20775628 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2006-03-07"), "end-date": date("2006-05-09") } ] }
+{ "id": 10178518, "id-copy": 10178518, "alias": "Rudyard", "name": "RudyardMcmullen", "user-since": datetime("2011-05-06T14:57:22.000"), "user-since-copy": datetime("2011-05-06T14:57:22.000"), "friend-ids": {{ 25647527, 14445589, 47924548, 24945241, 13505530, 39640007, 6132209, 815976, 31529708, 28281922, 17886251, 42402860, 18330827, 13619952 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2006-06-24") } ] }
+{ "id": 10179538, "id-copy": 10179538, "alias": "Orlando", "name": "OrlandoBaxter", "user-since": datetime("2006-02-06T08:33:07.000"), "user-since-copy": datetime("2006-02-06T08:33:07.000"), "friend-ids": {{ 6233497, 33888281, 44259464, 19279042, 22534429, 13084190, 38886041, 41675566, 3155617 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2009-07-06") } ] }
+{ "id": 10189600, "id-copy": 10189600, "alias": "Melisa", "name": "MelisaGarry", "user-since": datetime("2010-05-10T10:35:49.000"), "user-since-copy": datetime("2010-05-10T10:35:49.000"), "friend-ids": {{ 18172527, 26205741, 32077713, 41214698, 33783052, 5734397, 46101468, 30210046, 27425699 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-05-20"), "end-date": date("2011-07-20") } ] }
+{ "id": 10195063, "id-copy": 10195063, "alias": "Rose", "name": "RoseHatcher", "user-since": datetime("2008-10-11T02:17:54.000"), "user-since-copy": datetime("2008-10-11T02:17:54.000"), "friend-ids": {{ 9820231, 12294967, 46911959, 47936560, 7881400, 11585414, 45934029, 18009898, 11594812, 13760171, 41894550, 13254896, 28025170, 20007524, 13027888 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-03-26") } ] }
+{ "id": 10206877, "id-copy": 10206877, "alias": "Tammie", "name": "TammieBerry", "user-since": datetime("2009-10-14T12:57:11.000"), "user-since-copy": datetime("2009-10-14T12:57:11.000"), "friend-ids": {{ 23748102, 37944735, 42193629, 11409119, 41246083, 35024235 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2008-05-21") } ] }
+{ "id": 10247557, "id-copy": 10247557, "alias": "Shanita", "name": "ShanitaReed", "user-since": datetime("2006-08-01T23:58:30.000"), "user-since-copy": datetime("2006-08-01T23:58:30.000"), "friend-ids": {{ 39665727, 7906210, 46234266, 15304695, 4362978, 43689749, 11688287, 11377882, 33955818, 29447417, 23667673, 7373357, 45056089, 34964516, 13871603, 41976105, 10661879, 11112019, 17797460 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2004-12-08"), "end-date": date("2005-04-04") } ] }
+{ "id": 10287028, "id-copy": 10287028, "alias": "Wilfred", "name": "WilfredChurchill", "user-since": datetime("2007-08-01T14:14:25.000"), "user-since-copy": datetime("2007-08-01T14:14:25.000"), "friend-ids": {{ 38355737, 39891840, 41036196, 39165706, 1155288, 15280633, 9744287, 11567914, 11225763, 2297894, 14386027, 67174, 28097703, 28721858, 6504409, 6743503, 22860419, 17773814, 34697084, 5419586, 45771084 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2002-08-08") } ] }
+{ "id": 10297336, "id-copy": 10297336, "alias": "Gayelord", "name": "GayelordCypret", "user-since": datetime("2005-09-28T10:01:31.000"), "user-since-copy": datetime("2005-09-28T10:01:31.000"), "friend-ids": {{ 43657472, 21189656, 43018991, 42333420, 27203617, 12389046, 44062328, 15441240, 31806533, 44999377, 30592890, 12304605, 6752099, 9488471, 5719065, 16290550, 23175098, 6432261 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-05-15") } ] }
+{ "id": 10305280, "id-copy": 10305280, "alias": "Isabella", "name": "IsabellaWilo", "user-since": datetime("2007-01-03T11:54:28.000"), "user-since-copy": datetime("2007-01-03T11:54:28.000"), "friend-ids": {{ 46537100, 26395353, 23044918 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2005-01-06") } ] }
+{ "id": 10317160, "id-copy": 10317160, "alias": "Maria", "name": "MariaHair", "user-since": datetime("2006-05-21T16:06:00.000"), "user-since-copy": datetime("2006-05-21T16:06:00.000"), "friend-ids": {{ 7063473, 43027344, 2119671, 39231388, 34041933, 5141408, 20278936 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2005-10-20") } ] }
+{ "id": 10318882, "id-copy": 10318882, "alias": "Skyler", "name": "SkylerConrad", "user-since": datetime("2007-03-04T08:56:54.000"), "user-since-copy": datetime("2007-03-04T08:56:54.000"), "friend-ids": {{ 4254240, 3778434, 23914534, 16376376, 39143316, 37229152, 32778982, 30182686, 13077652, 20439638, 34086734, 12101909, 47011547, 28666460, 31034524, 47508299, 17267782, 1260337, 43500601, 914291, 1786773 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2009-01-15") } ] }
+{ "id": 10338907, "id-copy": 10338907, "alias": "Leah", "name": "LeahStroble", "user-since": datetime("2010-12-07T08:23:00.000"), "user-since-copy": datetime("2010-12-07T08:23:00.000"), "friend-ids": {{ 25263375, 47112518, 47910837, 14446727, 35708710, 41365949, 8534511, 34992353, 1706302, 21380997, 47197876, 29441929, 4157771, 8674755, 14520863, 22041433, 47176591, 4072306, 47354501 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2003-03-25") } ] }
+{ "id": 10348309, "id-copy": 10348309, "alias": "Bernard", "name": "BernardAltman", "user-since": datetime("2010-09-23T09:08:33.000"), "user-since-copy": datetime("2010-09-23T09:08:33.000"), "friend-ids": {{ 7859503, 40438517, 7050233, 41735514, 8274833, 12496793, 41853402, 23751827, 23485505, 35520895, 17406459, 20238814, 42333149 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2007-07-27") } ] }
+{ "id": 10349656, "id-copy": 10349656, "alias": "Woodrow", "name": "WoodrowRichter", "user-since": datetime("2006-09-18T16:22:12.000"), "user-since-copy": datetime("2006-09-18T16:22:12.000"), "friend-ids": {{ 12344306, 36484394, 30889842, 47572749, 42102868, 22350773, 7166034, 16132372, 45197714, 34516830, 47108654, 4399888, 24401048, 32578065, 16593311, 33394001, 7356357, 29943304, 30866764, 11942891 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2003-11-18"), "end-date": date("2004-10-16") } ] }
+{ "id": 10357477, "id-copy": 10357477, "alias": "Rosy", "name": "RosyMitchell", "user-since": datetime("2005-08-13T13:44:24.000"), "user-since-copy": datetime("2005-08-13T13:44:24.000"), "friend-ids": {{ 13370964, 4479736, 44060098, 28936173, 42239651, 18380035, 17854869, 36485096, 7662833 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-05-12") } ] }
+{ "id": 10391077, "id-copy": 10391077, "alias": "Tracy", "name": "TracyHiles", "user-since": datetime("2005-11-19T21:08:51.000"), "user-since-copy": datetime("2005-11-19T21:08:51.000"), "friend-ids": {{ 27119048, 1983772, 38766385, 35631268, 14736954, 7586158, 45840742, 27211063, 33946244, 1590669, 22363833, 19668917, 12778790, 31993728, 4498870, 68121, 13591025, 13285639 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2012-07-12") } ] }
+{ "id": 10392898, "id-copy": 10392898, "alias": "Rodger", "name": "RodgerLear", "user-since": datetime("2010-03-05T20:39:12.000"), "user-since-copy": datetime("2010-03-05T20:39:12.000"), "friend-ids": {{ 23638180, 34355575, 28958329, 17287883, 46069191, 4055459, 36969931, 13059600, 6957015, 41374655, 44549230, 1943320, 39878243 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2002-12-22") } ] }
+{ "id": 10398562, "id-copy": 10398562, "alias": "Brendon", "name": "BrendonMaclagan", "user-since": datetime("2012-02-23T06:18:49.000"), "user-since-copy": datetime("2012-02-23T06:18:49.000"), "friend-ids": {{ 39206829, 37980663, 36889290, 9114653, 26448451, 15142055, 23349234, 11668644, 22072984, 2091972, 957976, 26110137, 20947598, 32127830, 35850034, 39029675, 21265582, 26725192, 13963111, 4392994, 37042547 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2001-09-21") } ] }
+{ "id": 10422310, "id-copy": 10422310, "alias": "Edmundo", "name": "EdmundoShaw", "user-since": datetime("2012-07-02T11:10:15.000"), "user-since-copy": datetime("2012-07-02T11:10:15.000"), "friend-ids": {{ 4235436, 16381036, 12579129, 43280339, 16455681, 28445764, 10796826, 28577255, 15173785, 47982248, 11990921, 2093558, 6244669, 4830927, 34859603, 22246754, 45142656 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2011-01-27") } ] }
+{ "id": 10423588, "id-copy": 10423588, "alias": "Shirlene", "name": "ShirleneRuch", "user-since": datetime("2006-04-09T05:52:24.000"), "user-since-copy": datetime("2006-04-09T05:52:24.000"), "friend-ids": {{ 15418780, 12724265, 27282306, 13592995, 24753166, 32824252, 40619106, 27563604, 12337625, 45387219, 27749581, 44912564, 37470078, 19663516 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2003-06-17") } ] }
+{ "id": 10453837, "id-copy": 10453837, "alias": "Leila", "name": "LeilaHunter", "user-since": datetime("2007-12-08T12:41:34.000"), "user-since-copy": datetime("2007-12-08T12:41:34.000"), "friend-ids": {{ 2310862, 19014920 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2011-02-06") } ] }
+{ "id": 10472248, "id-copy": 10472248, "alias": "Harry", "name": "HarryDugmore", "user-since": datetime("2012-02-18T05:46:12.000"), "user-since-copy": datetime("2012-02-18T05:46:12.000"), "friend-ids": {{ 30193978, 30762534, 24660208, 29628319, 30687391, 39795396, 33525293, 23739628, 28969085, 30275276, 3497701, 17091988, 15259527, 25164171, 34052417, 4318314, 1876063, 29984074, 3421436, 16610126 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2012-01-19"), "end-date": date("2012-01-02") } ] }
+{ "id": 10484578, "id-copy": 10484578, "alias": "Troy", "name": "TroyWheeler", "user-since": datetime("2006-12-19T11:23:18.000"), "user-since-copy": datetime("2006-12-19T11:23:18.000"), "friend-ids": {{ 13536585, 23059550, 16602050, 12025612, 25014410, 13465266 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2011-10-23") } ] }
+{ "id": 10501429, "id-copy": 10501429, "alias": "Danielle", "name": "DanielleYoung", "user-since": datetime("2010-04-24T05:46:06.000"), "user-since-copy": datetime("2010-04-24T05:46:06.000"), "friend-ids": {{ 7960737, 27505427 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2002-07-27"), "end-date": date("2004-07-28") } ] }
+{ "id": 10505419, "id-copy": 10505419, "alias": "Anderson", "name": "AndersonSoames", "user-since": datetime("2009-04-01T01:24:07.000"), "user-since-copy": datetime("2009-04-01T01:24:07.000"), "friend-ids": {{ 25420744, 34012676, 8558565, 45471514, 12117008, 35275, 4952379, 46480100, 29394067, 15504329, 18153717, 8476606, 19867236, 35743164, 38523474, 6479207, 31151752, 19687338, 5379846, 32574974, 26920356 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2005-08-01") } ] }
+{ "id": 10508467, "id-copy": 10508467, "alias": "Quincey", "name": "QuinceyKettlewell", "user-since": datetime("2009-11-08T14:09:57.000"), "user-since-copy": datetime("2009-11-08T14:09:57.000"), "friend-ids": {{ 16037923, 33757766, 22829568, 34589661, 10645853, 43124745, 41785968, 27704416, 42381402, 11993654, 31993782, 37761743, 15571469, 33326934, 22719288, 18321279, 19252211, 42927515, 22390312, 37655021, 37511969, 47740024, 1015876 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2009-02-24") } ] }
+{ "id": 10529809, "id-copy": 10529809, "alias": "Aric", "name": "AricLauffer", "user-since": datetime("2007-05-18T09:08:29.000"), "user-since-copy": datetime("2007-05-18T09:08:29.000"), "friend-ids": {{ 36647795, 13183862, 5313167, 36450019, 46412788, 47789981, 4012027, 35872968, 3903895 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2011-09-22") } ] }
+{ "id": 10541299, "id-copy": 10541299, "alias": "Derrick", "name": "DerrickLarson", "user-since": datetime("2009-09-04T09:42:12.000"), "user-since-copy": datetime("2009-09-04T09:42:12.000"), "friend-ids": {{ 39544341, 9620318, 40218798, 34927427, 28533075, 44505091, 29066144, 31724565, 46052997, 3011652, 24709291, 24805644, 41125094, 14186985, 24967210, 32420881, 31162758, 2356654, 11854218, 47933360, 9668743, 26801113 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2000-11-25") } ] }
+{ "id": 10547020, "id-copy": 10547020, "alias": "Reita", "name": "ReitaBlunt", "user-since": datetime("2006-01-18T16:51:49.000"), "user-since-copy": datetime("2006-01-18T16:51:49.000"), "friend-ids": {{ 34373903, 36464697, 37171525, 19138424, 24675436, 16269152, 43940985, 2735762, 32760257, 42561749, 45516984, 39110107, 21610913, 1805884, 3342035, 40703512, 11665984, 29345992, 41497492, 30054924, 18098215 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-12-01") } ] }
+{ "id": 10548142, "id-copy": 10548142, "alias": "Dannie", "name": "DannieTillson", "user-since": datetime("2007-03-07T04:57:23.000"), "user-since-copy": datetime("2007-03-07T04:57:23.000"), "friend-ids": {{ 37443492, 21615683, 5655492, 24162015, 46418787, 46328489, 26669127, 38324141 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2012-01-03") } ] }
+{ "id": 10554112, "id-copy": 10554112, "alias": "Virgil", "name": "VirgilBickerson", "user-since": datetime("2006-03-14T07:07:42.000"), "user-since-copy": datetime("2006-03-14T07:07:42.000"), "friend-ids": {{ 21584501, 3506050, 31062036, 20425233, 6548274, 12613206, 16607156 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2004-08-25"), "end-date": date("2006-11-11") } ] }
+{ "id": 10561624, "id-copy": 10561624, "alias": "Marielle", "name": "MarielleBrandenburg", "user-since": datetime("2005-07-17T10:28:02.000"), "user-since-copy": datetime("2005-07-17T10:28:02.000"), "friend-ids": {{ 1231477, 14598987 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2000-03-06"), "end-date": date("2005-09-25") } ] }
+{ "id": 10579345, "id-copy": 10579345, "alias": "Rexana", "name": "RexanaSchaeffer", "user-since": datetime("2006-01-20T15:37:57.000"), "user-since-copy": datetime("2006-01-20T15:37:57.000"), "friend-ids": {{ 20070497, 44547094, 38571608, 30731404, 7825730, 8433351, 25090042, 38943273, 3599029, 28517891, 17427828, 6853394, 32856065, 46627870, 43885788 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-01-22") } ] }
+{ "id": 10580422, "id-copy": 10580422, "alias": "Travers", "name": "TraversSadley", "user-since": datetime("2011-02-09T08:22:49.000"), "user-since-copy": datetime("2011-02-09T08:22:49.000"), "friend-ids": {{ 36067992, 8651663, 43180149, 732576, 35709545, 30999437 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2009-05-27") } ] }
+{ "id": 10585294, "id-copy": 10585294, "alias": "Bryan", "name": "BryanEliza", "user-since": datetime("2005-02-03T16:20:19.000"), "user-since-copy": datetime("2005-02-03T16:20:19.000"), "friend-ids": {{ 6407647, 24838863, 45997254, 42728806, 37001718, 46932382 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-03-15"), "end-date": date("2008-04-24") } ] }
+{ "id": 10587655, "id-copy": 10587655, "alias": "Del", "name": "DelLester", "user-since": datetime("2006-04-22T06:14:51.000"), "user-since-copy": datetime("2006-04-22T06:14:51.000"), "friend-ids": {{ 41382268, 41043817, 37053482, 27889226, 5182442, 46241085, 39510378, 25972421, 6234359, 2782513, 27042023, 20476198 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2001-10-18") } ] }
+{ "id": 10591498, "id-copy": 10591498, "alias": "Mick", "name": "MickVeith", "user-since": datetime("2006-02-21T06:58:53.000"), "user-since-copy": datetime("2006-02-21T06:58:53.000"), "friend-ids": {{ 33872347, 40692511, 18563650 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2011-08-07"), "end-date": date("2011-01-10") } ] }
+{ "id": 10595164, "id-copy": 10595164, "alias": "Jerome", "name": "JeromeLacon", "user-since": datetime("2009-09-24T09:47:36.000"), "user-since-copy": datetime("2009-09-24T09:47:36.000"), "friend-ids": {{ 31538601 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-07-26") } ] }
+{ "id": 10601758, "id-copy": 10601758, "alias": "Blossom", "name": "BlossomClark", "user-since": datetime("2011-08-16T23:44:16.000"), "user-since-copy": datetime("2011-08-16T23:44:16.000"), "friend-ids": {{ 22624576, 6945784, 47816004, 8072206, 23953052, 22668193, 8668574, 2269602, 39137309, 38996903, 23516086, 31166264, 28322741, 46296094, 36547681, 7287738, 15727604, 13556387, 2624138 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-10-15"), "end-date": date("2008-07-17") } ] }
+{ "id": 10610356, "id-copy": 10610356, "alias": "Jason", "name": "JasonGearhart", "user-since": datetime("2010-03-05T22:57:20.000"), "user-since-copy": datetime("2010-03-05T22:57:20.000"), "friend-ids": {{ 6967239, 47468231, 29517365, 9206260 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-03-16"), "end-date": date("2012-06-19") } ] }
+{ "id": 10624381, "id-copy": 10624381, "alias": "Ryana", "name": "RyanaKimmons", "user-since": datetime("2007-09-04T15:42:08.000"), "user-since-copy": datetime("2007-09-04T15:42:08.000"), "friend-ids": {{ 36219003, 5135252, 24653726, 4767631, 21595268, 4154414, 31857818, 9711256, 20793102, 14509650 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2006-06-21") } ] }
+{ "id": 10635319, "id-copy": 10635319, "alias": "Rusty", "name": "RustyStange", "user-since": datetime("2010-08-17T17:30:37.000"), "user-since-copy": datetime("2010-08-17T17:30:37.000"), "friend-ids": {{ 28180565, 25608756 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2006-07-07") } ] }
+{ "id": 10658977, "id-copy": 10658977, "alias": "Danny", "name": "DannyBailey", "user-since": datetime("2006-12-12T12:28:17.000"), "user-since-copy": datetime("2006-12-12T12:28:17.000"), "friend-ids": {{ 27744791, 5839976, 37243832, 42061553, 15660549, 26723434, 25864049, 8038100, 47690286, 29206337, 6169296, 1933137, 6500848, 45632949, 6329147, 15602171, 13477556, 25033716, 9515038, 4081408, 42840830 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2009-12-14"), "end-date": date("2009-03-11") } ] }
+{ "id": 10671115, "id-copy": 10671115, "alias": "Montague", "name": "MontagueLangston", "user-since": datetime("2007-09-20T00:32:15.000"), "user-since-copy": datetime("2007-09-20T00:32:15.000"), "friend-ids": {{ 18236000, 47490167, 40246549, 25232933, 22604487, 36974958, 44747862, 2137180, 39244601, 39608406, 23319330, 21166788, 21726220, 12703943, 36564459, 8379538, 43010567, 24538004, 173522, 6132291, 21199763, 26285128, 2350066 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2001-01-07") } ] }
+{ "id": 10678567, "id-copy": 10678567, "alias": "Detta", "name": "DettaIronmonger", "user-since": datetime("2006-05-01T08:52:26.000"), "user-since-copy": datetime("2006-05-01T08:52:26.000"), "friend-ids": {{ 11098679, 15763619, 12715761, 10175990, 43581466, 4595173, 17163835, 44918467, 38256765, 13239047, 25476309, 9075112, 19581524, 46478013, 24168854, 34121818, 25604978, 21114089 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2007-06-08") } ] }
+{ "id": 10690066, "id-copy": 10690066, "alias": "Abraham", "name": "AbrahamWardle", "user-since": datetime("2006-04-08T20:27:10.000"), "user-since-copy": datetime("2006-04-08T20:27:10.000"), "friend-ids": {{ 18105973, 39839261, 27532181, 2565949, 37077592, 28929530 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2005-06-03"), "end-date": date("2006-12-02") } ] }
+{ "id": 10701727, "id-copy": 10701727, "alias": "Paulita", "name": "PaulitaHays", "user-since": datetime("2009-11-15T15:25:08.000"), "user-since-copy": datetime("2009-11-15T15:25:08.000"), "friend-ids": {{ 31869253, 13336594, 19116516, 30920596 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2001-12-10") } ] }
+{ "id": 10708477, "id-copy": 10708477, "alias": "Zacharias", "name": "ZachariasRandolph", "user-since": datetime("2008-07-13T16:12:33.000"), "user-since-copy": datetime("2008-07-13T16:12:33.000"), "friend-ids": {{ 18251027, 47694844, 25569678, 33130234, 7351010, 32617025, 40619749, 28576965, 34970660, 34320919, 17056847, 46007935, 244756, 3130710, 5218614, 6968874, 19440356, 448790, 3336700, 44725864, 24738046, 6159443, 14380294, 20289778 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2007-09-09") } ] }
+{ "id": 10721059, "id-copy": 10721059, "alias": "Amandine", "name": "AmandineRockwell", "user-since": datetime("2008-09-24T21:50:39.000"), "user-since-copy": datetime("2008-09-24T21:50:39.000"), "friend-ids": {{ 10360854, 15197739, 28812340, 12172446, 9354363, 23580760, 6364957, 20048548 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2003-12-06") } ] }
+{ "id": 10733305, "id-copy": 10733305, "alias": "Dakota", "name": "DakotaSmith", "user-since": datetime("2009-11-17T19:52:42.000"), "user-since-copy": datetime("2009-11-17T19:52:42.000"), "friend-ids": {{ 21984282, 14492326, 18724474, 17361116, 26773641, 32118673, 8295454, 6804824 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2007-05-28") } ] }
+{ "id": 10733617, "id-copy": 10733617, "alias": "Leonardo", "name": "LeonardoKight", "user-since": datetime("2008-10-20T17:30:29.000"), "user-since-copy": datetime("2008-10-20T17:30:29.000"), "friend-ids": {{ 39687903, 7235506, 34696496, 25995345, 18435380, 47473591, 15710408, 44232442, 39520147, 36384026, 25160887, 245860, 1195579, 4587411, 536916, 47052672, 33953823, 13203710 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2007-07-12"), "end-date": date("2010-03-16") } ] }
+{ "id": 10738477, "id-copy": 10738477, "alias": "Kenith", "name": "KenithLeichter", "user-since": datetime("2012-07-10T15:21:51.000"), "user-since-copy": datetime("2012-07-10T15:21:51.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2006-07-28"), "end-date": date("2009-06-03") } ] }
+{ "id": 10742182, "id-copy": 10742182, "alias": "Tel", "name": "TelBowchiew", "user-since": datetime("2009-09-23T02:51:14.000"), "user-since-copy": datetime("2009-09-23T02:51:14.000"), "friend-ids": {{ 17515416, 42010238, 23580669, 26008148, 35744494 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2006-10-05"), "end-date": date("2007-05-26") } ] }
+{ "id": 10745200, "id-copy": 10745200, "alias": "Kaety", "name": "KaetyOppenheimer", "user-since": datetime("2008-11-21T08:11:11.000"), "user-since-copy": datetime("2008-11-21T08:11:11.000"), "friend-ids": {{ 32006369, 4542624, 28242708, 20936957, 11063561, 31392192, 34444041, 754368, 37317926 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-06-07") } ] }
+{ "id": 10745974, "id-copy": 10745974, "alias": "Gavin", "name": "GavinWard", "user-since": datetime("2008-11-23T02:59:13.000"), "user-since-copy": datetime("2008-11-23T02:59:13.000"), "friend-ids": {{ 45290227, 46308273, 4478698, 27613190, 34907694, 36182643 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-01-01"), "end-date": date("2011-01-17") } ] }
+{ "id": 10751260, "id-copy": 10751260, "alias": "Chrysanta", "name": "ChrysantaSanforth", "user-since": datetime("2009-06-02T12:54:32.000"), "user-since-copy": datetime("2009-06-02T12:54:32.000"), "friend-ids": {{ 6064707, 44017707, 22957433, 38426343, 24694205, 1061085, 24827089, 12192854, 40718843 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2011-01-19"), "end-date": date("2011-10-02") } ] }
+{ "id": 10767553, "id-copy": 10767553, "alias": "Titty", "name": "TittyCross", "user-since": datetime("2009-02-08T11:38:56.000"), "user-since-copy": datetime("2009-02-08T11:38:56.000"), "friend-ids": {{ 10869392, 39422025, 23051606, 43241994, 6257807, 37258783, 26946341, 33120713, 6481181, 13410766, 34576024, 42401239, 28793792, 37331232, 5979767 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2000-12-26"), "end-date": date("2006-01-17") } ] }
+{ "id": 10777441, "id-copy": 10777441, "alias": "Rosaline", "name": "RosalineFaast", "user-since": datetime("2005-05-23T08:24:59.000"), "user-since-copy": datetime("2005-05-23T08:24:59.000"), "friend-ids": {{ 25088415, 36453219, 42450810, 6845863, 23568088, 34305276, 28849557, 41593223, 18542045, 37652004, 9159129, 42079452 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-01-04") } ] }
+{ "id": 10786129, "id-copy": 10786129, "alias": "Ardelle", "name": "ArdelleHoopengarner", "user-since": datetime("2012-05-27T08:36:37.000"), "user-since-copy": datetime("2012-05-27T08:36:37.000"), "friend-ids": {{ 44854493, 13697746, 8918104, 22353878, 46059542, 23393155, 37374548, 1531344, 31554501, 30390740, 10076243, 19028830, 46174212, 4991316, 30988902, 6717568 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-03-08") } ] }
+{ "id": 10789207, "id-copy": 10789207, "alias": "Lucinda", "name": "LucindaFillmore", "user-since": datetime("2009-11-13T18:35:41.000"), "user-since-copy": datetime("2009-11-13T18:35:41.000"), "friend-ids": {{ 10917581, 24902161, 29393856, 35293349, 31477965, 44139676, 18083704, 46487557 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2005-11-04") } ] }
+{ "id": 10808932, "id-copy": 10808932, "alias": "Sharita", "name": "SharitaGregory", "user-since": datetime("2006-09-17T04:48:23.000"), "user-since-copy": datetime("2006-09-17T04:48:23.000"), "friend-ids": {{ 41622567, 16559791, 6346693, 18540237, 14753253, 23252825, 17163196, 46962665, 26442426, 14344279, 17332246, 36154890, 22814241, 22709064, 32887290, 42853122, 23782934, 27425228, 22941847 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2008-06-08"), "end-date": date("2011-01-28") } ] }
+{ "id": 10809322, "id-copy": 10809322, "alias": "Alden", "name": "AldenHiggens", "user-since": datetime("2011-02-06T01:31:58.000"), "user-since-copy": datetime("2011-02-06T01:31:58.000"), "friend-ids": {{ 44750450, 24564153, 42513064, 33316253, 21036452, 27132567, 29231674, 18040424, 36564417, 17474605, 14126628, 18988855, 35594147, 35685289, 40967850 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2002-09-26") } ] }
+{ "id": 10811875, "id-copy": 10811875, "alias": "Giovanni", "name": "GiovanniWarner", "user-since": datetime("2009-05-28T04:20:11.000"), "user-since-copy": datetime("2009-05-28T04:20:11.000"), "friend-ids": {{ 8005226, 21432611, 4037183, 40486007, 40666777, 24385549, 3686021, 12188144, 33646224, 46365125, 44351069, 34408172, 35904411, 4322876, 18767645, 10007322 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2005-07-18"), "end-date": date("2011-10-24") } ] }
+{ "id": 10832305, "id-copy": 10832305, "alias": "Briony", "name": "BrionyBaldwin", "user-since": datetime("2011-03-03T22:00:38.000"), "user-since-copy": datetime("2011-03-03T22:00:38.000"), "friend-ids": {{ 20436897, 36519715, 35325917, 31686319, 2644929, 3401668, 39344422, 18601722, 40274111, 30032679, 9312830, 5581755, 41164101, 35883066, 8274432, 4315219, 26200418, 43810182, 44718149, 6387153, 43086214, 39558538, 36036905, 25715671 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2007-01-21"), "end-date": date("2008-02-25") } ] }
+{ "id": 10833472, "id-copy": 10833472, "alias": "Monica", "name": "MonicaRyals", "user-since": datetime("2009-02-14T18:52:57.000"), "user-since-copy": datetime("2009-02-14T18:52:57.000"), "friend-ids": {{ 34417058, 24053823, 28067368, 16205470, 24168710, 9064471 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2003-12-03"), "end-date": date("2006-03-07") } ] }
+{ "id": 10840990, "id-copy": 10840990, "alias": "Libby", "name": "LibbyHayhurst", "user-since": datetime("2009-10-28T22:52:04.000"), "user-since-copy": datetime("2009-10-28T22:52:04.000"), "friend-ids": {{ 32146321, 47850956, 42432761, 28856789, 18595962, 23408710, 37015546 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2003-01-20") } ] }
+{ "id": 10867624, "id-copy": 10867624, "alias": "Fredric", "name": "FredricKimmons", "user-since": datetime("2005-05-14T23:08:00.000"), "user-since-copy": datetime("2005-05-14T23:08:00.000"), "friend-ids": {{ 25574899, 26822046, 3408550, 40738004, 3813112, 33045116, 9229839, 28557630, 36781441, 23585776 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2010-02-25"), "end-date": date("2011-07-06") } ] }
+{ "id": 10868761, "id-copy": 10868761, "alias": "Peronel", "name": "PeronelGongaware", "user-since": datetime("2010-01-25T14:26:30.000"), "user-since-copy": datetime("2010-01-25T14:26:30.000"), "friend-ids": {{ 28271989, 41567995, 31926358, 16420360, 15775849, 44023747, 39099521, 4517209, 39890594, 39784644, 43247769, 25427216, 46426794, 37704581, 46477208, 3213706 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2011-12-16") } ] }
+{ "id": 10869727, "id-copy": 10869727, "alias": "Jacquetta", "name": "JacquettaMaugham", "user-since": datetime("2010-07-11T22:43:19.000"), "user-since-copy": datetime("2010-07-11T22:43:19.000"), "friend-ids": {{ 36109878, 46889968, 19648550, 14051620, 14645938, 14933447, 33880415 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2002-12-09") } ] }
+{ "id": 10874791, "id-copy": 10874791, "alias": "Haydee", "name": "HaydeeGarratt", "user-since": datetime("2007-04-14T00:19:00.000"), "user-since-copy": datetime("2007-04-14T00:19:00.000"), "friend-ids": {{ 12247794, 10306863, 33161811, 43877113, 37745696 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2008-03-07"), "end-date": date("2011-12-27") } ] }
+{ "id": 10894411, "id-copy": 10894411, "alias": "Lacy", "name": "LacyShaw", "user-since": datetime("2006-04-06T00:11:24.000"), "user-since-copy": datetime("2006-04-06T00:11:24.000"), "friend-ids": {{ 4203591, 28370134, 5239468, 12951448, 39355113, 9126812, 5662652, 4633221, 11954172, 33269236, 11545355, 14018236, 21980886, 34750979, 22877356 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-04-12") } ] }
+{ "id": 10911220, "id-copy": 10911220, "alias": "Laurice", "name": "LauriceDuncan", "user-since": datetime("2008-08-05T15:55:34.000"), "user-since-copy": datetime("2008-08-05T15:55:34.000"), "friend-ids": {{ 212109 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2001-02-03") } ] }
+{ "id": 10911274, "id-copy": 10911274, "alias": "Bridgette", "name": "BridgetteBenford", "user-since": datetime("2007-02-15T06:18:45.000"), "user-since-copy": datetime("2007-02-15T06:18:45.000"), "friend-ids": {{ 10909520, 14433605 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2012-01-14") } ] }
+{ "id": 10915261, "id-copy": 10915261, "alias": "Lyle", "name": "LyleMuller", "user-since": datetime("2010-10-16T16:36:46.000"), "user-since-copy": datetime("2010-10-16T16:36:46.000"), "friend-ids": {{ 28409003, 7495999, 10776059, 23825626, 44321306, 15679301, 36736470, 24070644, 14041140, 4784196, 19462533, 47300197, 33544003 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-09-25") } ] }
+{ "id": 10925071, "id-copy": 10925071, "alias": "Gil", "name": "GilFocell", "user-since": datetime("2005-11-08T20:28:01.000"), "user-since-copy": datetime("2005-11-08T20:28:01.000"), "friend-ids": {{ 9416716, 42743353, 43396785, 44271346, 32924780, 44752785, 19741326, 39315503, 25154503, 29170056, 15457515, 14764269, 47861907, 15230067, 15326613, 6336542, 44127013, 1048087, 34624221, 19951452, 12778135 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2009-01-07") } ] }
+{ "id": 10936273, "id-copy": 10936273, "alias": "Hans", "name": "HansMench", "user-since": datetime("2008-08-08T12:00:48.000"), "user-since-copy": datetime("2008-08-08T12:00:48.000"), "friend-ids": {{ 36800139 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2005-04-15"), "end-date": date("2009-08-05") } ] }
+{ "id": 10937893, "id-copy": 10937893, "alias": "Katheleen", "name": "KatheleenEisenmann", "user-since": datetime("2012-06-17T05:15:08.000"), "user-since-copy": datetime("2012-06-17T05:15:08.000"), "friend-ids": {{ 30129247, 865896, 35091601, 19852276, 43238329, 46057691, 30405091, 3723169, 6577863, 12648596, 34726408, 19178848, 18365491, 28604299, 29242262, 12826786, 19046213, 23320700, 9318080, 35996590, 24812162, 9639554, 33615920, 6507511 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2006-07-26") } ] }
+{ "id": 10940377, "id-copy": 10940377, "alias": "Lory", "name": "LoryElless", "user-since": datetime("2011-03-21T19:07:17.000"), "user-since-copy": datetime("2011-03-21T19:07:17.000"), "friend-ids": {{ 38950352, 10596357, 43176277, 27274342, 27082326 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2007-04-19") } ] }
+{ "id": 10948003, "id-copy": 10948003, "alias": "August", "name": "AugustHatch", "user-since": datetime("2006-04-11T03:32:56.000"), "user-since-copy": datetime("2006-04-11T03:32:56.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2008-12-16"), "end-date": date("2009-01-21") } ] }
+{ "id": 10968562, "id-copy": 10968562, "alias": "Fox", "name": "FoxBillimek", "user-since": datetime("2012-03-24T07:32:17.000"), "user-since-copy": datetime("2012-03-24T07:32:17.000"), "friend-ids": {{ 8459327, 11505750, 30952882, 30467951, 6329439, 33947538, 19579432, 25135787, 41391398, 32456626, 6310287, 31211659 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2009-01-17") } ] }
+{ "id": 11016043, "id-copy": 11016043, "alias": "Ellis", "name": "EllisVorrasi", "user-since": datetime("2009-08-26T16:43:17.000"), "user-since-copy": datetime("2009-08-26T16:43:17.000"), "friend-ids": {{ 41000811, 12639978, 14487796, 39651858, 40189282, 7834125, 44416511, 28673665 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2008-01-21"), "end-date": date("2008-04-26") } ] }
+{ "id": 11027953, "id-copy": 11027953, "alias": "Angelika", "name": "AngelikaSanner", "user-since": datetime("2010-10-07T04:25:19.000"), "user-since-copy": datetime("2010-10-07T04:25:19.000"), "friend-ids": {{ 42662440, 6358862, 21758734, 28882210, 28157558, 39027509, 19068795, 45387055, 34737892, 32277859, 44713546, 24617807, 31067294, 12307376, 28568916, 31114183, 13997610, 15405045, 33587810, 32517419, 13452101, 8309328 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2006-02-25") } ] }
+{ "id": 11052748, "id-copy": 11052748, "alias": "Andriana", "name": "AndrianaYonkie", "user-since": datetime("2005-05-08T19:49:03.000"), "user-since-copy": datetime("2005-05-08T19:49:03.000"), "friend-ids": {{ 24372868, 41932219, 14088659, 33215970, 34384197, 16343164, 24230672, 20937997, 23129922, 33184913, 25421373, 12081379, 289577, 19330874, 31625333, 34885607, 34353478, 17694263, 34819024, 44837603 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2003-06-16"), "end-date": date("2008-02-15") } ] }
+{ "id": 11062330, "id-copy": 11062330, "alias": "Derick", "name": "DerickPennington", "user-since": datetime("2008-04-15T11:59:52.000"), "user-since-copy": datetime("2008-04-15T11:59:52.000"), "friend-ids": {{ 26471368, 22445928, 13709179, 16677606, 45234923, 5601330, 16510085, 27673980, 24365707, 42647605, 20473849, 40448252, 37480913, 38532114, 11022656, 799537, 38469920, 1291033, 31503804, 29154535, 5506108, 24609403, 35535409, 44197253 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-09-23") } ] }
+{ "id": 11066710, "id-copy": 11066710, "alias": "Caryl", "name": "CarylMaugham", "user-since": datetime("2007-02-10T03:38:03.000"), "user-since-copy": datetime("2007-02-10T03:38:03.000"), "friend-ids": {{ 41776362, 7370825, 35851510, 23733011, 27617379, 39377372, 3043067, 22122576, 11996852, 20708849, 40772627, 20108470, 4141780, 3724555, 31849764, 7347633 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2001-10-15") } ] }
+{ "id": 11087224, "id-copy": 11087224, "alias": "Zola", "name": "ZolaKnisely", "user-since": datetime("2005-11-18T05:30:00.000"), "user-since-copy": datetime("2005-11-18T05:30:00.000"), "friend-ids": {{ 6324130, 38065951, 14950455, 27869167, 32957819, 11157656, 10411400, 18072233, 35246039, 35345326, 23217009, 13495953, 18987122 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-05-04") } ] }
+{ "id": 11087839, "id-copy": 11087839, "alias": "Manfred", "name": "ManfredEdwards", "user-since": datetime("2009-10-01T09:12:15.000"), "user-since-copy": datetime("2009-10-01T09:12:15.000"), "friend-ids": {{ 7828089 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-07-25") } ] }
+{ "id": 11111890, "id-copy": 11111890, "alias": "Geordie", "name": "GeordieGraff", "user-since": datetime("2006-02-12T04:30:44.000"), "user-since-copy": datetime("2006-02-12T04:30:44.000"), "friend-ids": {{ 12852237, 10391003, 37679153, 6620205, 25381043, 19805548, 4534765, 11626709, 47369482, 15045527, 25177819, 15113002, 39634176, 40637870, 47662386, 8045236 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2005-06-18") } ] }
+{ "id": 11116594, "id-copy": 11116594, "alias": "Norwood", "name": "NorwoodErrett", "user-since": datetime("2008-10-04T16:36:27.000"), "user-since-copy": datetime("2008-10-04T16:36:27.000"), "friend-ids": {{ 30996403, 30788997, 22512789, 35425088, 12096858, 21391496, 41281428, 15854003, 47041757, 31205204, 36849089, 43015828, 27098245, 46735331, 9520980, 34482257, 36898055, 8962397 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2002-05-20") } ] }
+{ "id": 11117371, "id-copy": 11117371, "alias": "Jules", "name": "JulesRichardson", "user-since": datetime("2009-12-06T06:21:58.000"), "user-since-copy": datetime("2009-12-06T06:21:58.000"), "friend-ids": {{ 75701, 18653454, 5088871, 20583891, 46460448, 19742484, 2433030, 30869605, 9273775, 6556358 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2001-09-17"), "end-date": date("2006-06-05") } ] }
+{ "id": 11136910, "id-copy": 11136910, "alias": "Karl", "name": "KarlGarratt", "user-since": datetime("2006-12-22T01:58:50.000"), "user-since-copy": datetime("2006-12-22T01:58:50.000"), "friend-ids": {{ 753124, 31382435, 30698735, 25951267, 27027532, 34551403, 9451765, 37517863, 3719825, 37613952, 18670991, 39783690, 6592095, 27477830, 31739951, 24458195, 12317249 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-05-11") } ] }
+{ "id": 11147392, "id-copy": 11147392, "alias": "Sarina", "name": "SarinaFlickinger", "user-since": datetime("2011-09-26T12:41:56.000"), "user-since-copy": datetime("2011-09-26T12:41:56.000"), "friend-ids": {{ 17776087, 9254087, 14735666, 31097664, 36421253, 12595115, 40366588, 9491701, 29725314, 38852857, 46206259, 39281843, 36268114, 29939350, 804107, 36307361, 30999436, 47369074, 3820973, 46362092, 36413930, 8807546, 30260636, 15069463 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2005-03-15") } ] }
+{ "id": 11155816, "id-copy": 11155816, "alias": "Titty", "name": "TittyOneal", "user-since": datetime("2009-06-01T06:21:44.000"), "user-since-copy": datetime("2009-06-01T06:21:44.000"), "friend-ids": {{ 37016026, 32220220, 47720886, 10358045, 7678433, 22148913, 18800507, 17043803, 29852152, 11426875, 44761613, 32002053, 14686180, 26744098, 34991446, 38818677, 24977770 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2012-05-11"), "end-date": date("2012-05-08") } ] }
+{ "id": 11214976, "id-copy": 11214976, "alias": "Maxwell", "name": "MaxwellBailey", "user-since": datetime("2005-11-25T15:01:26.000"), "user-since-copy": datetime("2005-11-25T15:01:26.000"), "friend-ids": {{ 22027101, 5782023, 46909646, 27593651, 31079804, 31989634, 7337526, 34757530, 32792041 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2011-04-04") } ] }
+{ "id": 11241523, "id-copy": 11241523, "alias": "Gareth", "name": "GarethFylbrigg", "user-since": datetime("2011-01-05T16:02:25.000"), "user-since-copy": datetime("2011-01-05T16:02:25.000"), "friend-ids": {{ 45629812, 20113715, 13556523, 29410246, 37849964, 33688575, 35713924, 21492453, 32324177, 5765413, 4491937, 1592640, 2809253, 45152094, 36330032, 25347157, 199553, 16471761, 16621535, 20674800, 42682300, 11354218, 4830164 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2005-10-27"), "end-date": date("2005-12-10") } ] }
+{ "id": 11244283, "id-copy": 11244283, "alias": "Erica", "name": "EricaTilton", "user-since": datetime("2005-12-10T16:37:41.000"), "user-since-copy": datetime("2005-12-10T16:37:41.000"), "friend-ids": {{ 9476551, 22631836, 44127713, 32391437, 19413944, 4263930, 17603111, 24077268, 31120069, 30869992, 6040985, 3918705, 17640663, 22515182 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2002-02-05"), "end-date": date("2003-07-03") } ] }
+{ "id": 11246161, "id-copy": 11246161, "alias": "Jemima", "name": "JemimaJube", "user-since": datetime("2009-10-13T13:44:48.000"), "user-since-copy": datetime("2009-10-13T13:44:48.000"), "friend-ids": {{ 35264732, 26686176, 37947249, 9511009, 20544975, 21318354, 2417039, 15051823, 23702057, 34446389, 15435804, 42646090, 14791709 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2012-02-26") } ] }
+{ "id": 11253043, "id-copy": 11253043, "alias": "Joye", "name": "JoyeGadow", "user-since": datetime("2005-10-03T17:22:30.000"), "user-since-copy": datetime("2005-10-03T17:22:30.000"), "friend-ids": {{ 24978234, 7896483, 14560795, 18402417, 16619973, 5852675, 29679362, 19344221, 33721635, 14137068, 30581619, 9715250, 10966922, 24167091, 36509340 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2011-01-08"), "end-date": date("2011-08-10") } ] }
+{ "id": 11270020, "id-copy": 11270020, "alias": "Ursula", "name": "UrsulaSauter", "user-since": datetime("2006-09-17T06:18:31.000"), "user-since-copy": datetime("2006-09-17T06:18:31.000"), "friend-ids": {{ 13370394, 5537385, 6651824, 27208272, 3304500, 26518061, 44906267, 27803333, 8618582, 22074752, 20865682, 15343007 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2006-08-01") } ] }
+{ "id": 11271517, "id-copy": 11271517, "alias": "Amaryllis", "name": "AmaryllisNewlove", "user-since": datetime("2009-06-10T04:18:11.000"), "user-since-copy": datetime("2009-06-10T04:18:11.000"), "friend-ids": {{ 6594489, 17958014, 4087759, 38993546, 1741537, 8374107, 30133658, 33873746 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2004-12-21"), "end-date": date("2011-08-19") } ] }
+{ "id": 11273587, "id-copy": 11273587, "alias": "Timmy", "name": "TimmyBishop", "user-since": datetime("2011-11-08T13:46:03.000"), "user-since-copy": datetime("2011-11-08T13:46:03.000"), "friend-ids": {{ 42987870, 44400071, 27388256, 10579275, 12546323, 23276512, 382419, 4466999, 8068553, 33814105, 14872828, 35038629, 43462816, 44037440 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2001-08-08") } ] }
+{ "id": 11276305, "id-copy": 11276305, "alias": "Salome", "name": "SalomeGongaware", "user-since": datetime("2007-06-05T10:15:14.000"), "user-since-copy": datetime("2007-06-05T10:15:14.000"), "friend-ids": {{ 17354378, 35576200, 42905756, 44408264, 45572153, 18424890, 39234162, 42837501, 38464194, 45237502, 30396078, 16316605, 32231800, 35417394, 32796520, 13885091, 31520983, 4624403, 18144193, 45707906, 8211336, 2864876 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2002-03-16") } ] }
+{ "id": 11290870, "id-copy": 11290870, "alias": "Lanford", "name": "LanfordOsteen", "user-since": datetime("2009-03-04T15:04:12.000"), "user-since-copy": datetime("2009-03-04T15:04:12.000"), "friend-ids": {{ 4397941, 36140649, 12796618, 18235191, 8810154, 10521988, 6580979, 29578654, 46083953, 30113784, 25952539 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2009-08-06") } ] }
+{ "id": 11307037, "id-copy": 11307037, "alias": "Brett", "name": "BrettLeichter", "user-since": datetime("2011-02-24T01:38:23.000"), "user-since-copy": datetime("2011-02-24T01:38:23.000"), "friend-ids": {{ 16273758, 36959770, 26721660 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2011-11-23") } ] }
+{ "id": 11327029, "id-copy": 11327029, "alias": "Mallory", "name": "MalloryHughes", "user-since": datetime("2007-08-06T22:11:46.000"), "user-since-copy": datetime("2007-08-06T22:11:46.000"), "friend-ids": {{ 38924183, 22042572, 21014848, 46309217, 1120998, 19755064, 4413438, 38855205, 17626985, 5727472, 1293238 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2006-02-28"), "end-date": date("2006-08-24") } ] }
+{ "id": 11341747, "id-copy": 11341747, "alias": "Margaux", "name": "MargauxBynum", "user-since": datetime("2009-01-16T19:54:27.000"), "user-since-copy": datetime("2009-01-16T19:54:27.000"), "friend-ids": {{ 27056110, 1770280, 17190314, 18164827, 32684926, 32410281, 27173037, 16864868, 4664026, 31170366, 4296651 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2008-08-20") } ] }
+{ "id": 11362531, "id-copy": 11362531, "alias": "Garey", "name": "GareyChapman", "user-since": datetime("2005-10-13T04:24:29.000"), "user-since-copy": datetime("2005-10-13T04:24:29.000"), "friend-ids": {{ 20693565, 18896854, 17118168, 12285534, 21434048, 15453439, 42734432, 3627967, 30464042, 11556192, 22808282, 464074, 28100870, 29887664, 19046987, 34996619, 39964690, 22574200, 29497238 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2001-03-05") } ] }
+{ "id": 11366131, "id-copy": 11366131, "alias": "Cayley", "name": "CayleyGronko", "user-since": datetime("2005-03-06T13:24:19.000"), "user-since-copy": datetime("2005-03-06T13:24:19.000"), "friend-ids": {{ 26623267, 47792710, 27975124, 19721566, 45092752, 32954140, 25835098 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2006-09-14"), "end-date": date("2010-06-02") } ] }
+{ "id": 11378911, "id-copy": 11378911, "alias": "Courtney", "name": "CourtneyBashline", "user-since": datetime("2010-10-21T06:13:06.000"), "user-since-copy": datetime("2010-10-21T06:13:06.000"), "friend-ids": {{ 19627264, 13699162 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2002-06-21") } ] }
+{ "id": 11380807, "id-copy": 11380807, "alias": "Mckinley", "name": "MckinleyGeyer", "user-since": datetime("2008-02-17T13:01:21.000"), "user-since-copy": datetime("2008-02-17T13:01:21.000"), "friend-ids": {{ 16655526, 20048717, 15998744, 39702027, 28153175, 40825599, 38372618 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2010-11-26") } ] }
+{ "id": 11386210, "id-copy": 11386210, "alias": "Dale", "name": "DaleGreenwood", "user-since": datetime("2007-04-17T19:02:45.000"), "user-since-copy": datetime("2007-04-17T19:02:45.000"), "friend-ids": {{ 3669916 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2002-09-11") } ] }
+{ "id": 11404780, "id-copy": 11404780, "alias": "Carol", "name": "CarolCox", "user-since": datetime("2009-07-07T23:58:07.000"), "user-since-copy": datetime("2009-07-07T23:58:07.000"), "friend-ids": {{ 41450896, 12332484, 18515318, 39039576, 2336271, 47313837, 4655597, 40110200, 7357446, 24291515, 8898678, 28911118, 20372890, 1296082, 42558011, 5719716, 6830197 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2005-01-14") } ] }
+{ "id": 11412382, "id-copy": 11412382, "alias": "Gosse", "name": "GosseSutton", "user-since": datetime("2011-01-07T02:19:16.000"), "user-since-copy": datetime("2011-01-07T02:19:16.000"), "friend-ids": {{ 25790586, 42348812, 39275252, 32764855, 11642271, 15982736, 21971689, 13168697, 38246675, 40514837, 20840965 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2010-12-18"), "end-date": date("2011-01-09") } ] }
+{ "id": 11412640, "id-copy": 11412640, "alias": "Larry", "name": "LarryEisaman", "user-since": datetime("2005-04-23T10:38:04.000"), "user-since-copy": datetime("2005-04-23T10:38:04.000"), "friend-ids": {{ 15063821, 35006785, 18241384, 5967937, 45426140, 44234765, 3244540, 3222784, 36330320 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2001-07-05") } ] }
+{ "id": 11415055, "id-copy": 11415055, "alias": "Zavia", "name": "ZaviaLombardi", "user-since": datetime("2006-01-10T02:11:24.000"), "user-since-copy": datetime("2006-01-10T02:11:24.000"), "friend-ids": {{ 25953753, 952678, 31067065 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2003-06-27"), "end-date": date("2010-07-02") } ] }
+{ "id": 11427397, "id-copy": 11427397, "alias": "Oscar", "name": "OscarMillhouse", "user-since": datetime("2012-04-07T04:52:39.000"), "user-since-copy": datetime("2012-04-07T04:52:39.000"), "friend-ids": {{ 27577077, 26831616, 24024317, 24669981, 15864715, 41688094, 25689775, 19288762, 25015698, 24343183, 30170416, 39881555, 29378159, 6748762, 45948007 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2012-05-15") } ] }
+{ "id": 11428300, "id-copy": 11428300, "alias": "Major", "name": "MajorGreenawalt", "user-since": datetime("2006-12-02T06:43:13.000"), "user-since-copy": datetime("2006-12-02T06:43:13.000"), "friend-ids": {{ 8021918, 4810021, 34724015, 45030049, 36575685, 44527472 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2002-04-17") } ] }
+{ "id": 11445889, "id-copy": 11445889, "alias": "Milford", "name": "MilfordTeagarden", "user-since": datetime("2006-06-07T19:18:28.000"), "user-since-copy": datetime("2006-06-07T19:18:28.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "subtam", "start-date": date("2003-07-26") } ] }
+{ "id": 11448565, "id-copy": 11448565, "alias": "Martie", "name": "MartiePoley", "user-since": datetime("2010-07-02T14:37:46.000"), "user-since-copy": datetime("2010-07-02T14:37:46.000"), "friend-ids": {{ 45198632, 14347405, 14595348, 4990646, 44745176, 21949325, 9155582, 3970455, 10097690, 35781298, 46746615, 35535590, 16561713, 31169880, 22467369 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2006-02-08") } ] }
+{ "id": 11468158, "id-copy": 11468158, "alias": "Pamelia", "name": "PameliaShaner", "user-since": datetime("2005-07-11T18:28:07.000"), "user-since-copy": datetime("2005-07-11T18:28:07.000"), "friend-ids": {{ 8892753, 24751024, 7162523, 38425260, 8752332, 23371746, 6673241, 22278741, 46403700 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2006-02-04") } ] }
+{ "id": 11471689, "id-copy": 11471689, "alias": "Bevis", "name": "BevisWhishaw", "user-since": datetime("2011-03-05T23:14:53.000"), "user-since-copy": datetime("2011-03-05T23:14:53.000"), "friend-ids": {{ 27818002, 43784015, 39101258, 28170566, 38541659, 43935487, 907437, 25457112, 4731176, 35304801, 30364855, 33197014, 27028915, 21746182, 47624076, 41599425, 8592245 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2000-04-04"), "end-date": date("2009-05-08") } ] }
+{ "id": 11474374, "id-copy": 11474374, "alias": "Waldo", "name": "WaldoKnapp", "user-since": datetime("2008-08-17T21:17:28.000"), "user-since-copy": datetime("2008-08-17T21:17:28.000"), "friend-ids": {{ 33358772, 16499546, 8631001, 6045567, 45554236, 36229482, 354579, 11884970, 23657774, 32568373 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-11-18") } ] }
+{ "id": 11481961, "id-copy": 11481961, "alias": "Ralph", "name": "RalphMinnie", "user-since": datetime("2008-09-03T03:36:09.000"), "user-since-copy": datetime("2008-09-03T03:36:09.000"), "friend-ids": {{ 28795092, 15427393, 13323116, 6103928, 22507606, 38931008, 8419762, 30922606, 11217439, 41769747, 19668638, 26796252, 26750627, 4855539, 11170229, 30124829, 16596482, 15728547, 46139530, 43784722, 20640234, 22313927, 16136087, 39688415 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2006-02-13") } ] }
+{ "id": 11488420, "id-copy": 11488420, "alias": "Rik", "name": "RikSell", "user-since": datetime("2011-04-24T10:10:24.000"), "user-since-copy": datetime("2011-04-24T10:10:24.000"), "friend-ids": {{ 37808691, 28841986, 27850488, 28093210, 9165013, 45941806, 5194022, 39773028, 45473967, 44833113, 27429268 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2002-09-23"), "end-date": date("2010-06-23") } ] }
+{ "id": 11490220, "id-copy": 11490220, "alias": "Ernestine", "name": "ErnestineWheeler", "user-since": datetime("2005-01-27T23:36:35.000"), "user-since-copy": datetime("2005-01-27T23:36:35.000"), "friend-ids": {{ 12995063, 40353122, 11162426, 42762839, 9575788, 7725738, 29883894, 48002015, 5516807, 12731814, 33203496, 44912740, 19681146, 5849671, 4702317 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2008-06-16"), "end-date": date("2011-12-01") } ] }
+{ "id": 11529952, "id-copy": 11529952, "alias": "Charles", "name": "CharlesHarrow", "user-since": datetime("2008-11-24T19:27:12.000"), "user-since-copy": datetime("2008-11-24T19:27:12.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2005-06-08"), "end-date": date("2011-10-27") } ] }
+{ "id": 11533327, "id-copy": 11533327, "alias": "Miguel", "name": "MiguelSteiner", "user-since": datetime("2007-12-08T18:21:30.000"), "user-since-copy": datetime("2007-12-08T18:21:30.000"), "friend-ids": {{ 41619494, 4881397, 29302201, 26654760, 9690024, 15599321, 37163728, 2420315, 46258007, 15076674, 6757461 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2001-08-19"), "end-date": date("2008-10-15") } ] }
+{ "id": 11536078, "id-copy": 11536078, "alias": "Scot", "name": "ScotSwartzbaugh", "user-since": datetime("2007-06-02T13:28:19.000"), "user-since-copy": datetime("2007-06-02T13:28:19.000"), "friend-ids": {{ 160897, 11035428, 35908585, 14713740, 16036400, 21530456, 31659920, 33439685, 42771513, 42899492, 42315848, 17885118, 12371932, 47219421, 45350312, 33755309, 30284897, 34557464, 21531204, 26093690 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2002-02-23"), "end-date": date("2005-03-24") } ] }
+{ "id": 11538001, "id-copy": 11538001, "alias": "Milo", "name": "MiloGarland", "user-since": datetime("2007-09-12T09:40:42.000"), "user-since-copy": datetime("2007-09-12T09:40:42.000"), "friend-ids": {{ 7363153, 7252759 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2011-09-03"), "end-date": date("2011-10-27") } ] }
+{ "id": 11540278, "id-copy": 11540278, "alias": "Flora", "name": "FloraSaltser", "user-since": datetime("2007-11-20T08:52:26.000"), "user-since-copy": datetime("2007-11-20T08:52:26.000"), "friend-ids": {{ 44172124, 43836609, 2821020, 356092, 25456578, 14806637, 19970466, 15369859, 23267393, 34480680, 42574031, 39606777, 17221367, 19617483, 1364901, 21402012, 4999365, 31098654, 34512618, 44652673, 14757091, 9755310, 39190510 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2012-07-07") } ] }
+{ "id": 11542174, "id-copy": 11542174, "alias": "Pacey", "name": "PaceyTripp", "user-since": datetime("2011-11-07T08:36:12.000"), "user-since-copy": datetime("2011-11-07T08:36:12.000"), "friend-ids": {{ 35602078, 32622628, 34826581, 34837077, 41522736, 14908313, 42986568 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2006-07-08") } ] }
+{ "id": 11542519, "id-copy": 11542519, "alias": "Colten", "name": "ColtenDemuth", "user-since": datetime("2012-02-09T01:22:04.000"), "user-since-copy": datetime("2012-02-09T01:22:04.000"), "friend-ids": {{ 15666280, 36489446, 45424145, 47509110, 24198688, 42545568, 30526545, 43828073, 26402530, 23632737, 20385217, 35055795, 38789042, 34967858, 521531, 47834820, 20307524 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2008-04-10") } ] }
+{ "id": 11570326, "id-copy": 11570326, "alias": "Linden", "name": "LindenFilby", "user-since": datetime("2007-08-16T03:11:11.000"), "user-since-copy": datetime("2007-08-16T03:11:11.000"), "friend-ids": {{ 6549689, 15243636, 3147666 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2010-02-23"), "end-date": date("2010-04-22") } ] }
+{ "id": 11573350, "id-copy": 11573350, "alias": "Sommer", "name": "SommerGregory", "user-since": datetime("2007-08-25T21:50:51.000"), "user-since-copy": datetime("2007-08-25T21:50:51.000"), "friend-ids": {{ 6622046, 40071999, 24631984, 42427860, 13378139, 27659078, 32813734, 20145238, 15342806, 9562288, 24211264, 29951003, 3620479, 43701781, 22474191, 6298296, 4047189, 27133942, 8058121, 9928231, 31835361, 6234235, 6100660, 1575061 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-12-09"), "end-date": date("2010-01-16") } ] }
+{ "id": 11587666, "id-copy": 11587666, "alias": "Kathi", "name": "KathiJenner", "user-since": datetime("2012-02-20T01:58:30.000"), "user-since-copy": datetime("2012-02-20T01:58:30.000"), "friend-ids": {{ 37156773, 10519382, 11009989, 47883115, 13123467, 36990044, 8554049, 47075065, 11896169, 42580126, 43261036, 15337748, 35985068, 44438965, 33507413, 40063633, 32559158, 32202309, 25536635 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2001-01-01") } ] }
+{ "id": 11610913, "id-copy": 11610913, "alias": "Vic", "name": "VicDiegel", "user-since": datetime("2008-08-03T21:05:21.000"), "user-since-copy": datetime("2008-08-03T21:05:21.000"), "friend-ids": {{ 15275871, 8304749, 7803583, 45134147, 36058489, 7180792, 2104280, 4322584, 39304177, 43050196, 32955811, 4161448, 3187410, 47263593 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-03-11") } ] }
+{ "id": 11626990, "id-copy": 11626990, "alias": "Filiberto", "name": "FilibertoFonblanque", "user-since": datetime("2006-05-18T07:38:32.000"), "user-since-copy": datetime("2006-05-18T07:38:32.000"), "friend-ids": {{ 41443868, 30006940, 14137070, 14868792, 47991977, 39513958, 32787637, 1389727, 28607710, 21537795, 42395037, 11730902, 25246772, 24475669, 35786951, 32795214 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2007-08-11") } ] }
+{ "id": 11659888, "id-copy": 11659888, "alias": "Nannie", "name": "NannieWoodworth", "user-since": datetime("2006-12-11T15:30:08.000"), "user-since-copy": datetime("2006-12-11T15:30:08.000"), "friend-ids": {{ 30803046, 33105462, 14783423, 5069473, 15960335 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2006-10-12") } ] }
+{ "id": 11668552, "id-copy": 11668552, "alias": "Kassandra", "name": "KassandraJames", "user-since": datetime("2010-09-27T18:12:59.000"), "user-since-copy": datetime("2010-09-27T18:12:59.000"), "friend-ids": {{ 27400643, 15449089, 802964, 45059523, 9603951, 20911122, 46243977, 45487995, 34528880, 16093159, 22484957, 3951663, 12349433, 7887502, 34786818, 13014384, 28307526, 30476565, 7746152, 17600641, 36877141, 4513081, 25065078 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-04"), "end-date": date("2012-08-25") } ] }
+{ "id": 11674741, "id-copy": 11674741, "alias": "Soon", "name": "SoonBillimek", "user-since": datetime("2009-03-02T12:08:16.000"), "user-since-copy": datetime("2009-03-02T12:08:16.000"), "friend-ids": {{ 26069920, 16634341, 13963293, 27425934, 19271848, 22444876, 42264629, 39307655, 21118192, 27961060, 12398172, 13202296, 23221559, 34323488, 1588557, 42672479, 19548482, 28266272, 6241122, 13633490 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2006-01-19"), "end-date": date("2011-03-25") } ] }
+{ "id": 11676574, "id-copy": 11676574, "alias": "Isidore", "name": "IsidoreCatlay", "user-since": datetime("2012-08-26T08:28:08.000"), "user-since-copy": datetime("2012-08-26T08:28:08.000"), "friend-ids": {{ 46189001 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2001-03-07") } ] }
+{ "id": 11698384, "id-copy": 11698384, "alias": "Bernetta", "name": "BernettaFiddler", "user-since": datetime("2012-06-20T20:05:46.000"), "user-since-copy": datetime("2012-06-20T20:05:46.000"), "friend-ids": {{ 12203676 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2000-03-06") } ] }
+{ "id": 11708152, "id-copy": 11708152, "alias": "Gil", "name": "GilElsas", "user-since": datetime("2009-04-08T15:40:59.000"), "user-since-copy": datetime("2009-04-08T15:40:59.000"), "friend-ids": {{ 14661698, 22657473, 28892770, 39654430, 46338819, 44974094, 38564659, 24819725, 21550883, 37711934, 37285158, 20050610, 19163447, 10974750, 47513067, 43771947, 23633824 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2002-09-21"), "end-date": date("2011-03-11") } ] }
+{ "id": 11709478, "id-copy": 11709478, "alias": "Jonty", "name": "JontyCurry", "user-since": datetime("2006-09-08T22:15:05.000"), "user-since-copy": datetime("2006-09-08T22:15:05.000"), "friend-ids": {{ 1684909, 3914449, 16704128, 11890093, 44073634, 24897496 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2006-03-01") } ] }
+{ "id": 11713315, "id-copy": 11713315, "alias": "Chung", "name": "ChungStroble", "user-since": datetime("2005-10-20T22:59:27.000"), "user-since-copy": datetime("2005-10-20T22:59:27.000"), "friend-ids": {{ 13105744, 9160760, 37104436, 33688116, 31455484, 44428287 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2002-12-03"), "end-date": date("2010-10-06") } ] }
+{ "id": 11720794, "id-copy": 11720794, "alias": "Alisha", "name": "AlishaTue", "user-since": datetime("2010-08-11T01:17:31.000"), "user-since-copy": datetime("2010-08-11T01:17:31.000"), "friend-ids": {{ 6380101, 43972052, 6557931, 42465959, 21268624, 35831867, 45839471, 37781645, 34750475, 35886124, 4491900 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2001-02-02") } ] }
+{ "id": 11725939, "id-copy": 11725939, "alias": "Clover", "name": "CloverAlice", "user-since": datetime("2007-07-12T05:17:52.000"), "user-since-copy": datetime("2007-07-12T05:17:52.000"), "friend-ids": {{ 24426905, 6647137, 25463555, 11443041, 10549599, 35925634, 4053835, 11813301, 6976204, 26680887, 29934690, 7935338, 45092791, 30510709 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2000-05-04"), "end-date": date("2000-08-24") } ] }
+{ "id": 11735830, "id-copy": 11735830, "alias": "Maryvonne", "name": "MaryvonneHarrold", "user-since": datetime("2007-12-03T06:30:43.000"), "user-since-copy": datetime("2007-12-03T06:30:43.000"), "friend-ids": {{ 27842540, 46624942, 21701969, 33750891, 28523702, 38840881, 1497785, 32357938, 19740312, 1880841, 41116687, 35621654, 46917268, 14610853, 33099367, 8710534 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2004-10-04") } ] }
+{ "id": 11758474, "id-copy": 11758474, "alias": "Xavier", "name": "XavierAtweeke", "user-since": datetime("2011-10-03T12:35:37.000"), "user-since-copy": datetime("2011-10-03T12:35:37.000"), "friend-ids": {{ 30110740, 41016650, 23732518, 14585316, 34474077, 47591093, 10803514, 8912354, 43455040, 21960801, 31978150, 40693811, 14585416, 36411476, 20556412, 44978412, 7266670, 506620, 7686872 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2004-03-07") } ] }
+{ "id": 11774587, "id-copy": 11774587, "alias": "Shari", "name": "ShariMortland", "user-since": datetime("2012-07-21T10:15:22.000"), "user-since-copy": datetime("2012-07-21T10:15:22.000"), "friend-ids": {{ 17661326, 29399532, 38328734, 38063295, 46008807, 29873254, 4407085, 27903240 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2005-05-18") } ] }
+{ "id": 11783038, "id-copy": 11783038, "alias": "Cecily", "name": "CecilyRamsey", "user-since": datetime("2011-01-20T23:39:28.000"), "user-since-copy": datetime("2011-01-20T23:39:28.000"), "friend-ids": {{ 30228589, 45494315, 36823967, 2965036, 37243358, 7140131, 8303981, 10041948, 41439178, 24261471, 16906521, 46190105, 45392996, 21067630, 26632248, 44955893 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2000-03-25"), "end-date": date("2010-06-25") } ] }
+{ "id": 11786815, "id-copy": 11786815, "alias": "Micheal", "name": "MichealTreeby", "user-since": datetime("2008-06-04T14:59:23.000"), "user-since-copy": datetime("2008-06-04T14:59:23.000"), "friend-ids": {{ 15590922, 1367468, 37464776, 21877607, 38646966, 46702919, 46771039, 4688915, 41827211, 6556380 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2003-09-17") } ] }
+{ "id": 11801005, "id-copy": 11801005, "alias": "Jacques", "name": "JacquesWhitling", "user-since": datetime("2007-05-20T05:42:21.000"), "user-since-copy": datetime("2007-05-20T05:42:21.000"), "friend-ids": {{ 45134681, 48016178 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2006-12-07") } ] }
+{ "id": 11804755, "id-copy": 11804755, "alias": "Humbert", "name": "HumbertArmitage", "user-since": datetime("2008-01-01T21:14:34.000"), "user-since-copy": datetime("2008-01-01T21:14:34.000"), "friend-ids": {{ 15498777, 1984479, 18672418, 13137212, 17931875, 10446256, 39250716, 9422828, 35469173, 35940705, 44217206 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2005-11-12") } ] }
+{ "id": 11818252, "id-copy": 11818252, "alias": "Sandee", "name": "SandeeBlair", "user-since": datetime("2008-12-22T20:09:56.000"), "user-since-copy": datetime("2008-12-22T20:09:56.000"), "friend-ids": {{ 35579096, 13690328, 19410347, 10601941, 13140634, 19728850 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2007-09-24") } ] }
+{ "id": 11821996, "id-copy": 11821996, "alias": "Latanya", "name": "LatanyaZalack", "user-since": datetime("2010-12-07T15:20:09.000"), "user-since-copy": datetime("2010-12-07T15:20:09.000"), "friend-ids": {{ 23521495, 43957220, 3823403, 34033770 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2008-04-17") } ] }
+{ "id": 11830663, "id-copy": 11830663, "alias": "Bettie", "name": "BettieKing", "user-since": datetime("2009-11-06T15:04:55.000"), "user-since-copy": datetime("2009-11-06T15:04:55.000"), "friend-ids": {{ 46068058, 35215092, 34850678, 9126970, 16472040, 20000261, 17610567, 37016763, 19830405, 38071058, 43961371, 13092410, 24867008, 12366628, 15539063, 15611017, 1343975, 43254018, 30838755, 30488641, 38027133, 5701592 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2003-04-10") } ] }
+{ "id": 11857618, "id-copy": 11857618, "alias": "Glenda", "name": "GlendaPyle", "user-since": datetime("2009-01-05T13:34:53.000"), "user-since-copy": datetime("2009-01-05T13:34:53.000"), "friend-ids": {{ 31083833, 39371819, 38336556, 7590988, 17022330, 8016611, 41444367, 13194826, 1589028, 37076285, 33481940, 22093098, 9959371, 35262849, 20744580, 33226729, 35025566, 46396680, 30247311, 6884899, 35691024, 40965552, 46106170 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2000-02-19") } ] }
+{ "id": 11886532, "id-copy": 11886532, "alias": "Tel", "name": "TelGardner", "user-since": datetime("2009-10-06T10:33:32.000"), "user-since-copy": datetime("2009-10-06T10:33:32.000"), "friend-ids": {{ 37243107, 36561786, 3939621, 13531917, 7768514, 31689833, 27145019, 9462172, 40579935, 32184519, 8668855, 26137893, 5582080, 4847233, 10244448, 42634758, 34911290, 10834989, 34800551, 14109743 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2010-07-24") } ] }
+{ "id": 11894854, "id-copy": 11894854, "alias": "Connor", "name": "ConnorWilliamson", "user-since": datetime("2011-09-16T22:24:17.000"), "user-since-copy": datetime("2011-09-16T22:24:17.000"), "friend-ids": {{ 19318451, 47946991, 1913830, 45324890, 47189256, 39211392, 6998884, 4344587, 24720830, 4355756, 19102058, 34241496, 39408673, 1360498, 7695088, 25754984, 21796436 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2007-09-19"), "end-date": date("2010-07-22") } ] }
+{ "id": 11899576, "id-copy": 11899576, "alias": "Raven", "name": "RavenAdams", "user-since": datetime("2011-12-02T12:46:45.000"), "user-since-copy": datetime("2011-12-02T12:46:45.000"), "friend-ids": {{ 33232775, 8985272, 34257645, 15577012, 3749136, 36721837, 17368752, 36931534, 30688133, 36202643, 8373322, 34639728, 10776563, 5758944, 19414939, 46764976, 29704238, 38970621, 9462886, 46724087, 29191126, 9001393 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2003-03-02") } ] }
+{ "id": 11899861, "id-copy": 11899861, "alias": "Jacki", "name": "JackiLeach", "user-since": datetime("2009-01-07T13:33:40.000"), "user-since-copy": datetime("2009-01-07T13:33:40.000"), "friend-ids": {{ 17554995, 17598007, 2855045, 4108843, 47202404, 42565398, 45821410, 32619673, 7988594, 7631349, 20552170, 13116128, 14526615, 17916951, 43018507, 18114607 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-06-24") } ] }
+{ "id": 11920078, "id-copy": 11920078, "alias": "Alane", "name": "AlaneRichter", "user-since": datetime("2005-04-12T04:06:03.000"), "user-since-copy": datetime("2005-04-12T04:06:03.000"), "friend-ids": {{ 18326190, 34366549, 13047472, 29553920, 6210406, 41865352, 26108964, 15042193, 33225025, 7014329, 11051157, 37032436, 8025322, 21902099, 22953955, 42645725, 29144585 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2006-04-24") } ] }
+{ "id": 11920375, "id-copy": 11920375, "alias": "Terance", "name": "TeranceSaylor", "user-since": datetime("2005-02-09T10:33:47.000"), "user-since-copy": datetime("2005-02-09T10:33:47.000"), "friend-ids": {{ 17869677, 39051840, 6852335, 6153367, 1318628, 9983745, 5401091, 32798056, 42870494, 10337793, 43570623, 3233493, 38297525, 43712104, 15430099, 36703995, 25022620, 3681464, 21499719, 33737350, 6602331, 35391438, 47011233 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2005-11-05"), "end-date": date("2011-04-20") } ] }
+{ "id": 11943412, "id-copy": 11943412, "alias": "Kizzie", "name": "KizzieBillimek", "user-since": datetime("2011-08-25T09:24:43.000"), "user-since-copy": datetime("2011-08-25T09:24:43.000"), "friend-ids": {{ 47433684, 41380366, 5933545, 6348490, 24429719, 22579519, 21550752, 4653838, 44131628, 7980571, 3208666, 35631166, 13693250, 41263305, 29172668, 24656473, 31110672, 11323134, 23674731, 37422602, 20327470, 13419973 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2012-03-18"), "end-date": date("2012-06-09") } ] }
+{ "id": 11972860, "id-copy": 11972860, "alias": "Isador", "name": "IsadorCattley", "user-since": datetime("2005-04-10T23:37:49.000"), "user-since-copy": datetime("2005-04-10T23:37:49.000"), "friend-ids": {{ 39841874, 9405322, 3110197, 39455453, 11331432, 31809217, 45852118, 12899824, 19561127, 3413313, 19872192, 13427579, 140732, 6913603 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2006-01-01"), "end-date": date("2009-11-22") } ] }
+{ "id": 11989228, "id-copy": 11989228, "alias": "Jaden", "name": "JadenKelley", "user-since": datetime("2006-11-12T15:45:55.000"), "user-since-copy": datetime("2006-11-12T15:45:55.000"), "friend-ids": {{ 39881086, 47143027, 9394301, 17338199, 16961896, 6602092, 46708527, 24050942, 20543677, 13309656 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2003-10-19") } ] }
+{ "id": 11990740, "id-copy": 11990740, "alias": "Vernon", "name": "VernonBarnes", "user-since": datetime("2005-05-25T09:07:06.000"), "user-since-copy": datetime("2005-05-25T09:07:06.000"), "friend-ids": {{ 44677447, 20354746, 30157224, 29686873, 9413456, 11656099, 25404439, 24706566, 45005726, 22096097, 29868918, 12109246, 38948331, 2643312, 41565707, 17566751, 8045341, 25358960, 43614095, 28262168, 14405467, 22519550 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2012-01-23") } ] }
+{ "id": 11996683, "id-copy": 11996683, "alias": "Ivy", "name": "IvyReddish", "user-since": datetime("2008-10-09T09:54:46.000"), "user-since-copy": datetime("2008-10-09T09:54:46.000"), "friend-ids": {{ 42344158, 40312093, 15782003 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2003-04-16") } ] }
+{ "id": 9004354, "id-copy": 9004354, "alias": "Deshawn", "name": "DeshawnGarneys", "user-since": datetime("2010-07-21T12:45:03.000"), "user-since-copy": datetime("2010-07-21T12:45:03.000"), "friend-ids": {{ 46096495, 1526403 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2011-07-08") } ] }
+{ "id": 9025786, "id-copy": 9025786, "alias": "Terrance", "name": "TerranceFinlay", "user-since": datetime("2009-12-28T02:19:23.000"), "user-since-copy": datetime("2009-12-28T02:19:23.000"), "friend-ids": {{ 45324679, 13507068, 46678304, 37010727, 44866157, 12584675, 34305776, 14467180, 37751377, 2448873, 32584169, 14120838, 8902593, 31955437, 13436805 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2012-02-19"), "end-date": date("2012-07-25") } ] }
+{ "id": 9029377, "id-copy": 9029377, "alias": "Boyce", "name": "BoyceAnderson", "user-since": datetime("2010-12-18T14:17:12.000"), "user-since-copy": datetime("2010-12-18T14:17:12.000"), "friend-ids": {{ 19260027, 21449100, 35898407, 34501982 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2007-06-25") } ] }
+{ "id": 9041578, "id-copy": 9041578, "alias": "Kristia", "name": "KristiaWillcox", "user-since": datetime("2012-01-09T10:29:02.000"), "user-since-copy": datetime("2012-01-09T10:29:02.000"), "friend-ids": {{ 29794000, 34515675, 3759231, 14418948, 35788028, 34225561, 20821065, 27582458, 4424723 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2005-06-04"), "end-date": date("2008-01-13") } ] }
+{ "id": 9041992, "id-copy": 9041992, "alias": "Royston", "name": "RoystonBatten", "user-since": datetime("2009-06-27T08:09:45.000"), "user-since-copy": datetime("2009-06-27T08:09:45.000"), "friend-ids": {{ 35666317, 30439304, 35405688, 2079220, 5996407, 40490306, 33188983, 24455609, 4293738, 29028817, 32566429, 10186823 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2002-04-04"), "end-date": date("2010-06-28") } ] }
+{ "id": 9046852, "id-copy": 9046852, "alias": "Mauro", "name": "MauroChase", "user-since": datetime("2011-04-18T20:18:58.000"), "user-since-copy": datetime("2011-04-18T20:18:58.000"), "friend-ids": {{ 28268506, 13880377, 18637778, 27129860, 47146036, 23136396, 34534506, 23274864, 38781071, 9644011, 34754620, 45178277, 33832472, 31871984, 47201051, 42153557, 12418584, 37615805, 35474951, 29273401, 4845352, 18687033 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2012-05-14"), "end-date": date("2012-06-25") } ] }
+{ "id": 9067279, "id-copy": 9067279, "alias": "Jeanine", "name": "JeanineEmrick", "user-since": datetime("2011-06-25T09:43:07.000"), "user-since-copy": datetime("2011-06-25T09:43:07.000"), "friend-ids": {{ 12884712, 45104617, 41134568, 15844605, 645264, 33182092, 16884335, 46281324, 3977219, 5682848, 441588, 26025738, 3165091, 21821928, 23073877 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2012-01-02") } ] }
+{ "id": 9069397, "id-copy": 9069397, "alias": "Manuel", "name": "ManuelTrevithick", "user-since": datetime("2009-01-25T00:11:22.000"), "user-since-copy": datetime("2009-01-25T00:11:22.000"), "friend-ids": {{ 1121944, 14645273, 16100117, 45331540, 17901062, 7344920, 22533580, 16386626, 4267586, 34975914, 28841442, 38737330, 31607047, 11785331, 9617022, 44328180, 30996836, 14315445, 18464409, 21038654, 14409120, 12230754, 25856707 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2011-10-12"), "end-date": date("2011-03-28") } ] }
+{ "id": 9077020, "id-copy": 9077020, "alias": "Marquis", "name": "MarquisBunten", "user-since": datetime("2008-08-23T04:31:07.000"), "user-since-copy": datetime("2008-08-23T04:31:07.000"), "friend-ids": {{ 16894897, 21101342, 27872737, 14878739, 47969914, 38986368, 20779589, 4491084, 21066166, 40159242, 25290828, 43152855, 41928030, 2282944 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2001-07-16") } ] }
+{ "id": 9082201, "id-copy": 9082201, "alias": "Alberic", "name": "AlbericCrawford", "user-since": datetime("2005-02-11T07:41:05.000"), "user-since-copy": datetime("2005-02-11T07:41:05.000"), "friend-ids": {{ 26925567, 6108069, 30484049, 4903722, 4579631, 21166966, 3892344, 6259030, 32887933, 7183018, 46041497, 23448710, 47887528, 3679587, 7140571, 47671072, 4554470, 23481403, 16738975, 4885244 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2006-10-10") } ] }
+{ "id": 9083791, "id-copy": 9083791, "alias": "Lashay", "name": "LashayLeonard", "user-since": datetime("2008-07-03T04:52:06.000"), "user-since-copy": datetime("2008-07-03T04:52:06.000"), "friend-ids": {{ 16762687, 32021842, 851915, 36102981, 3553783, 30756474, 12043049, 16852621, 35699568, 4425852, 35227725, 25748188, 9140215, 24886626, 1945167, 12733697, 20761965 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2003-05-19"), "end-date": date("2006-10-16") } ] }
+{ "id": 9087292, "id-copy": 9087292, "alias": "Kiersten", "name": "KierstenRawls", "user-since": datetime("2005-03-21T08:42:24.000"), "user-since-copy": datetime("2005-03-21T08:42:24.000"), "friend-ids": {{ 5551555, 2958358, 17900476, 23956783, 31634897, 12573318, 32475113, 47343698, 40929064, 39881831, 38067700, 3519291, 19229024, 4383684, 13932328, 16414275, 8654888, 16145374, 26880764 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-03-10") } ] }
+{ "id": 9098314, "id-copy": 9098314, "alias": "Terrance", "name": "TerranceWilkerson", "user-since": datetime("2010-07-01T06:01:32.000"), "user-since-copy": datetime("2010-07-01T06:01:32.000"), "friend-ids": {{ 32477103, 38306013, 36022406, 25594192, 10966661, 28537611, 5444323, 16012053, 43228208, 30344050, 22600011, 42820310, 37103995, 6359985 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2012-01-24") } ] }
+{ "id": 9112336, "id-copy": 9112336, "alias": "Marlin", "name": "MarlinRosenstiehl", "user-since": datetime("2010-09-26T04:27:50.000"), "user-since-copy": datetime("2010-09-26T04:27:50.000"), "friend-ids": {{ 10225686, 16259250, 11552542, 28661586, 8900635, 27988260 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2011-12-05") } ] }
+{ "id": 9133714, "id-copy": 9133714, "alias": "Wil", "name": "WilDale", "user-since": datetime("2009-12-04T18:40:04.000"), "user-since-copy": datetime("2009-12-04T18:40:04.000"), "friend-ids": {{ 40400811, 26528322 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2005-10-08"), "end-date": date("2007-03-23") } ] }
+{ "id": 9139966, "id-copy": 9139966, "alias": "Elwood", "name": "ElwoodDavis", "user-since": datetime("2009-04-25T20:38:07.000"), "user-since-copy": datetime("2009-04-25T20:38:07.000"), "friend-ids": {{ 28327906, 35534034, 3278109, 20721373, 40303614, 22594044, 3292862, 42117489, 18133788, 31771270, 43837818, 36567026 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2004-09-03"), "end-date": date("2011-07-03") } ] }
+{ "id": 9151357, "id-copy": 9151357, "alias": "Clover", "name": "CloverTedrow", "user-since": datetime("2012-04-04T22:46:03.000"), "user-since-copy": datetime("2012-04-04T22:46:03.000"), "friend-ids": {{ 47959325, 11808875, 46311157, 33138600, 15486362, 27921017, 32586367, 24379643, 14793815, 5841252, 22249573, 2147304, 47811082, 40329394, 4601822, 27977744, 45733056 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2002-09-07"), "end-date": date("2006-08-04") } ] }
+{ "id": 9160906, "id-copy": 9160906, "alias": "Cathryn", "name": "CathrynReamer", "user-since": datetime("2010-10-08T06:24:51.000"), "user-since-copy": datetime("2010-10-08T06:24:51.000"), "friend-ids": {{ 30962953 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2006-10-28"), "end-date": date("2010-03-14") } ] }
+{ "id": 9174313, "id-copy": 9174313, "alias": "Hal", "name": "HalHasely", "user-since": datetime("2008-01-28T17:01:16.000"), "user-since-copy": datetime("2008-01-28T17:01:16.000"), "friend-ids": {{ 9058102, 40616538, 45706325, 991699, 37832260, 4793008, 36372035, 23272432, 36685642, 2621984, 9576806, 14325601, 41449409, 16499609, 20610820, 1564035, 20738111, 19735088, 31942764, 34813086 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-10-16") } ] }
+{ "id": 9179122, "id-copy": 9179122, "alias": "Zach", "name": "ZachMilliron", "user-since": datetime("2011-07-28T01:09:04.000"), "user-since-copy": datetime("2011-07-28T01:09:04.000"), "friend-ids": {{ 40552138, 19204406, 46806031, 18794200, 45071131, 40119114, 23955279, 11126709, 37101358, 23332998, 1172034, 41496458, 32278235, 30949991, 148070, 6360227, 7378339, 33611217 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2000-06-26") } ] }
+{ "id": 9187549, "id-copy": 9187549, "alias": "Lenny", "name": "LennyField", "user-since": datetime("2008-09-11T10:50:10.000"), "user-since-copy": datetime("2008-09-11T10:50:10.000"), "friend-ids": {{ 26505249, 4392946, 32062169, 45628101, 22865593, 4982483, 13425537, 18846467, 36122039, 2998293, 19787439, 22246499, 43133873, 30573462, 36272473, 41691126, 43929640, 43759980, 25546305 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2004-08-13"), "end-date": date("2010-03-08") } ] }
+{ "id": 9203731, "id-copy": 9203731, "alias": "Phoebe", "name": "PhoebeCoates", "user-since": datetime("2008-04-27T01:42:34.000"), "user-since-copy": datetime("2008-04-27T01:42:34.000"), "friend-ids": {{ 25611465, 519838, 22814080, 46015954, 7805914, 12757618, 36785422, 25727822, 32042543 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2003-11-11"), "end-date": date("2005-08-19") } ] }
+{ "id": 9216376, "id-copy": 9216376, "alias": "Stanford", "name": "StanfordBurney", "user-since": datetime("2010-04-24T23:03:06.000"), "user-since-copy": datetime("2010-04-24T23:03:06.000"), "friend-ids": {{ 15567770, 24839882, 163708, 45725879, 43621238, 27363995, 46782727, 21660511, 37585197, 17426559, 47247057, 41831246, 23944363, 1608826, 25831838, 41140458, 37108898, 19739056, 7475981, 17807472, 3126856, 40257768, 44873566 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2004-10-04") } ] }
+{ "id": 9219955, "id-copy": 9219955, "alias": "Audrey", "name": "AudreyOmara", "user-since": datetime("2011-06-04T15:31:25.000"), "user-since-copy": datetime("2011-06-04T15:31:25.000"), "friend-ids": {{ 28209595, 29907721, 18295175, 18631813, 3380755, 20244076, 43026452, 42394327, 10914853, 27224999 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2003-03-24") } ] }
+{ "id": 9232504, "id-copy": 9232504, "alias": "Lesley", "name": "LesleyHujsak", "user-since": datetime("2008-07-07T13:30:22.000"), "user-since-copy": datetime("2008-07-07T13:30:22.000"), "friend-ids": {{ 42058063, 24501683, 26865036, 180621 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2011-01-04"), "end-date": date("2011-02-07") } ] }
+{ "id": 9233794, "id-copy": 9233794, "alias": "Jeffrey", "name": "JeffreyThrockmorton", "user-since": datetime("2005-04-23T04:24:31.000"), "user-since-copy": datetime("2005-04-23T04:24:31.000"), "friend-ids": {{ 29565308, 29107229, 35495609, 27358360, 24507795, 18583779, 16799427, 3571959, 6539875, 32120867, 17248402, 12227155, 37995559, 29425657, 20855502 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2000-04-22"), "end-date": date("2010-05-28") } ] }
+{ "id": 9234529, "id-copy": 9234529, "alias": "Xavior", "name": "XaviorBarnes", "user-since": datetime("2010-08-26T12:06:44.000"), "user-since-copy": datetime("2010-08-26T12:06:44.000"), "friend-ids": {{ 19552290, 24018104, 43285028, 33954718, 18084047, 18675363, 17369450, 36533551, 46779811, 46943171, 17609996, 14171942, 10468121, 33831228, 9905114, 11839935, 41387228 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2007-12-24") } ] }
+{ "id": 9239515, "id-copy": 9239515, "alias": "Precious", "name": "PreciousWeingarten", "user-since": datetime("2006-07-03T10:28:56.000"), "user-since-copy": datetime("2006-07-03T10:28:56.000"), "friend-ids": {{ 20459132, 9181399, 30604442, 45266959, 31805782, 8190732, 46444663, 46572075, 43980715, 42547186, 21087158, 38075989, 32228414, 25466991, 4929897, 33467622, 35742242, 7150399, 16997658, 18543557, 11799062 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2003-02-15") } ] }
+{ "id": 9269422, "id-copy": 9269422, "alias": "Roddy", "name": "RoddyFriedline", "user-since": datetime("2007-03-26T23:41:29.000"), "user-since-copy": datetime("2007-03-26T23:41:29.000"), "friend-ids": {{ 31923430, 19739952, 30983882, 10630795 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2001-06-03") } ] }
+{ "id": 9292738, "id-copy": 9292738, "alias": "Walter", "name": "WalterWain", "user-since": datetime("2012-05-03T10:41:22.000"), "user-since-copy": datetime("2012-05-03T10:41:22.000"), "friend-ids": {{ 1834068, 38777276, 43381631, 32380769, 23994313, 37459142, 21015234, 23788270, 33191448, 31111521, 21788604, 39349512, 20638072, 17300228, 4712935, 36205876, 27740958, 27236154 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2011-03-28") } ] }
+{ "id": 9332161, "id-copy": 9332161, "alias": "Lavinia", "name": "LaviniaLineman", "user-since": datetime("2006-02-07T20:39:55.000"), "user-since-copy": datetime("2006-02-07T20:39:55.000"), "friend-ids": {{ 21419337, 31581364 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-07-05") } ] }
+{ "id": 9341008, "id-copy": 9341008, "alias": "Gus", "name": "GusGearhart", "user-since": datetime("2012-05-23T13:19:57.000"), "user-since-copy": datetime("2012-05-23T13:19:57.000"), "friend-ids": {{ 20124243, 19722425, 20605718, 21833401, 18276801, 46184199, 40454562, 22828817, 44122338, 4485860, 34209581, 19783645, 44454238, 1353350, 37958534, 33547730, 2456119, 3023314, 44828467, 46655836, 33144170, 16864855, 41938662 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2002-07-07") } ] }
+{ "id": 9354127, "id-copy": 9354127, "alias": "Seymour", "name": "SeymourFlick", "user-since": datetime("2011-06-17T06:00:11.000"), "user-since-copy": datetime("2011-06-17T06:00:11.000"), "friend-ids": {{ 7662170, 25563062, 18178019, 32667220, 12254954, 7192061, 18829113, 8959008, 1692176, 28852587, 17130396, 12781461, 4083182, 11054115, 10558861, 13876198 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2007-11-23") } ] }
+{ "id": 9356098, "id-copy": 9356098, "alias": "Juliana", "name": "JulianaAnderson", "user-since": datetime("2007-04-26T20:13:07.000"), "user-since-copy": datetime("2007-04-26T20:13:07.000"), "friend-ids": {{ 3850702, 46858392, 20177889, 34485932, 20958453, 26839176, 23562562, 47962945, 43961803, 19857248, 29816714, 14695228, 35327929, 16196977, 11908428, 30035277, 23919929 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2009-06-04"), "end-date": date("2009-05-05") } ] }
+{ "id": 9369847, "id-copy": 9369847, "alias": "Jeffrey", "name": "JeffreyArchibald", "user-since": datetime("2011-07-11T23:43:52.000"), "user-since-copy": datetime("2011-07-11T23:43:52.000"), "friend-ids": {{ 44928062, 45653705 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2010-03-25") } ] }
+{ "id": 9407710, "id-copy": 9407710, "alias": "Todd", "name": "ToddStall", "user-since": datetime("2009-09-21T02:18:16.000"), "user-since-copy": datetime("2009-09-21T02:18:16.000"), "friend-ids": {{ 46998635, 14217621, 8062100, 47498395, 37234901, 41039045, 37635206, 42173831, 24149948 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2009-09-27"), "end-date": date("2009-07-21") } ] }
+{ "id": 9418882, "id-copy": 9418882, "alias": "Laurine", "name": "LaurineCowart", "user-since": datetime("2012-06-14T22:26:09.000"), "user-since-copy": datetime("2012-06-14T22:26:09.000"), "friend-ids": {{ 19430214, 17084414, 12678029, 1783933, 42580022, 26274674, 13661281, 31117329, 19971039, 43840305, 42672247, 17088417, 31128028, 41009670, 16020772 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2002-03-20") } ] }
+{ "id": 9421798, "id-copy": 9421798, "alias": "Jaqueline", "name": "JaquelineHasely", "user-since": datetime("2011-06-06T16:32:03.000"), "user-since-copy": datetime("2011-06-06T16:32:03.000"), "friend-ids": {{ 17911249, 45887650, 15916739, 42045244, 42824039, 4802136, 43709530, 41533233, 13714833, 33000412, 29627102, 43277560, 3727319, 19030370, 47600623, 27902511, 13460397, 34825938, 9726577, 10062858, 34721080, 6725312, 21572679 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2004-02-17") } ] }
+{ "id": 9426544, "id-copy": 9426544, "alias": "Joshawa", "name": "JoshawaHiles", "user-since": datetime("2012-04-28T09:48:20.000"), "user-since-copy": datetime("2012-04-28T09:48:20.000"), "friend-ids": {{ 16780903 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2002-07-01") } ] }
+{ "id": 9440452, "id-copy": 9440452, "alias": "Maria", "name": "MariaField", "user-since": datetime("2010-04-06T15:15:24.000"), "user-since-copy": datetime("2010-04-06T15:15:24.000"), "friend-ids": {{ 35137543, 24166956, 45255343, 10050289, 27769291, 40368984, 38146662, 43123957, 10442976, 46931482, 447566, 14148069, 39035817, 32169234, 35607837, 8648749, 3741547, 31840808, 3029722, 40917859 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-04-27"), "end-date": date("2012-05-11") } ] }
+{ "id": 9449881, "id-copy": 9449881, "alias": "Veola", "name": "VeolaSchaeffer", "user-since": datetime("2005-06-15T04:27:55.000"), "user-since-copy": datetime("2005-06-15T04:27:55.000"), "friend-ids": {{ 15932585, 16875491, 977001, 15650783, 30629770, 9735829, 35435062, 2023808, 21909452, 29327288, 24004438, 41780113, 10546865, 17514287, 16690971, 23762008, 21853049, 12673064, 35992661, 30579445, 21341455, 2338670 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2001-09-07") } ] }
+{ "id": 9450532, "id-copy": 9450532, "alias": "Troy", "name": "TroyKoepple", "user-since": datetime("2011-05-10T09:56:46.000"), "user-since-copy": datetime("2011-05-10T09:56:46.000"), "friend-ids": {{ 42029412, 18025243, 715282, 501115, 38550360, 39016114, 31451417, 38836992, 13665836, 17286159, 28850827, 17241066, 41893804, 39172781, 4523003, 28542863, 25386847, 44039032, 19593806, 607220, 26442265, 47847281 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-12-26"), "end-date": date("2004-04-05") } ] }
+{ "id": 9483769, "id-copy": 9483769, "alias": "Marketta", "name": "MarkettaSchere", "user-since": datetime("2006-04-02T05:48:16.000"), "user-since-copy": datetime("2006-04-02T05:48:16.000"), "friend-ids": {{ 15151816, 38432593, 14501842, 21508230, 20201815, 35434395, 46212890, 9387767, 35469959, 6671088, 38888798, 10719563, 36944652, 36703732, 9646545, 29287523, 24156038, 24502755 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-07-20"), "end-date": date("2006-03-10") } ] }
+{ "id": 9503761, "id-copy": 9503761, "alias": "Demelza", "name": "DemelzaLaw", "user-since": datetime("2010-12-17T06:40:19.000"), "user-since-copy": datetime("2010-12-17T06:40:19.000"), "friend-ids": {{ 34126746, 5537488, 609154, 35877951, 36237612 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2005-10-22") } ] }
+{ "id": 9516883, "id-copy": 9516883, "alias": "Delsie", "name": "DelsieKuster", "user-since": datetime("2005-11-20T06:18:01.000"), "user-since-copy": datetime("2005-11-20T06:18:01.000"), "friend-ids": {{ 7211399, 31355269, 10052966, 11255272, 11976144, 13036749, 28228775, 3501290, 35668522, 21064471, 47089958, 20725508, 16768149, 39282691, 44096922, 12469594, 8258231, 36373387, 14994345, 32022989, 28975684, 29840860 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2008-11-11"), "end-date": date("2008-03-06") } ] }
+{ "id": 9518128, "id-copy": 9518128, "alias": "Jerrie", "name": "JerrieFonblanque", "user-since": datetime("2008-06-08T02:51:53.000"), "user-since-copy": datetime("2008-06-08T02:51:53.000"), "friend-ids": {{ 41051692, 21547608, 41749297, 21528434, 28012731, 43579854, 9172140, 17908381, 10276804, 12277383, 38454166, 6950146, 11878198, 24415804, 46218827, 33013212, 44735001, 36395459, 38515534, 16015324, 21085620, 20338207 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-01-14") } ] }
+{ "id": 9522265, "id-copy": 9522265, "alias": "Brendon", "name": "BrendonLing", "user-since": datetime("2012-08-11T12:01:34.000"), "user-since-copy": datetime("2012-08-11T12:01:34.000"), "friend-ids": {{ 32770998, 43037450, 13481444, 36411834, 21704194 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2012-08-30") } ] }
+{ "id": 9543280, "id-copy": 9543280, "alias": "Isabell", "name": "IsabellGaskins", "user-since": datetime("2009-12-05T01:29:24.000"), "user-since-copy": datetime("2009-12-05T01:29:24.000"), "friend-ids": {{ 9815607, 43778761, 25835208, 40078303, 28971077, 9802833, 17822058, 12655680, 37398606, 11387722, 5483134, 11506312, 36341116, 13511812, 3504784, 11655484, 18350098, 15365006, 32814750 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-07-01"), "end-date": date("2007-08-14") } ] }
+{ "id": 9545626, "id-copy": 9545626, "alias": "Russell", "name": "RussellKeilbach", "user-since": datetime("2010-05-20T15:10:25.000"), "user-since-copy": datetime("2010-05-20T15:10:25.000"), "friend-ids": {{ 40592323, 28819303 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2003-04-18") } ] }
+{ "id": 9568750, "id-copy": 9568750, "alias": "Daley", "name": "DaleyHarshman", "user-since": datetime("2012-01-17T10:38:07.000"), "user-since-copy": datetime("2012-01-17T10:38:07.000"), "friend-ids": {{ 18932212, 37118057, 37586464, 12686041, 21083532, 27598912 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2007-11-07") } ] }
+{ "id": 9596080, "id-copy": 9596080, "alias": "Yolonda", "name": "YolondaUlery", "user-since": datetime("2012-03-02T19:57:32.000"), "user-since-copy": datetime("2012-03-02T19:57:32.000"), "friend-ids": {{ 22382589, 22012001, 13142890, 44320162, 10358341, 14975, 43101433, 10324321, 14791134, 25984312, 11075173, 44140537, 40528755, 27384004, 40022140, 10650900, 37789740, 6928495, 22130557, 47679224, 40973393, 37190617, 35395183 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2012-03-03") } ] }
+{ "id": 9597526, "id-copy": 9597526, "alias": "Emory", "name": "EmoryThorley", "user-since": datetime("2006-01-19T22:44:03.000"), "user-since-copy": datetime("2006-01-19T22:44:03.000"), "friend-ids": {{ 420066, 8047878, 20510786, 1639671, 22923859, 27319995, 3624690, 18526424, 45857863, 2830065, 4588990, 25531572, 17878497, 47796172, 41309806, 34307425, 10084701, 1659934, 38218970, 44720636, 43501970, 610796, 35455526, 2080900 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2011-06-18"), "end-date": date("2011-09-10") } ] }
+{ "id": 9602242, "id-copy": 9602242, "alias": "Marc", "name": "MarcDimsdale", "user-since": datetime("2005-10-03T23:32:18.000"), "user-since-copy": datetime("2005-10-03T23:32:18.000"), "friend-ids": {{ 34004502, 24469994, 2140538, 1486939, 6895407, 13348535, 22384921, 11662871, 21398307, 33070732, 45602509, 26146770, 24148813, 45988030, 22184030, 855669, 36390708, 30883354, 26360628, 29836897, 28696575 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2004-05-15"), "end-date": date("2008-01-19") } ] }
+{ "id": 9638248, "id-copy": 9638248, "alias": "Azucena", "name": "AzucenaEmrick", "user-since": datetime("2005-12-04T00:15:40.000"), "user-since-copy": datetime("2005-12-04T00:15:40.000"), "friend-ids": {{ 37210744, 43097413, 2901403, 24492031, 7887583, 42518446, 28555003, 20402754, 5506767, 22982986, 21168589, 37638670, 30930177, 43662522, 45627167, 13450586, 36757137, 46663990 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2002-07-07"), "end-date": date("2006-06-11") } ] }
+{ "id": 9646474, "id-copy": 9646474, "alias": "Lilac", "name": "LilacWoodworth", "user-since": datetime("2009-12-17T02:42:51.000"), "user-since-copy": datetime("2009-12-17T02:42:51.000"), "friend-ids": {{ 47784123, 45348808, 36392712, 9381262, 10215254, 1461251, 23038092, 44549001, 39097217, 41152823, 31758517, 19401493, 39964393, 46307214, 41683224, 39011968, 5014398, 482179, 3789628, 46257340, 36041029, 10903757, 5980810, 31935548 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2004-10-25"), "end-date": date("2005-05-04") } ] }
+{ "id": 9665848, "id-copy": 9665848, "alias": "Shannah", "name": "ShannahDale", "user-since": datetime("2006-08-09T02:09:51.000"), "user-since-copy": datetime("2006-08-09T02:09:51.000"), "friend-ids": {{ 19512022, 25217933, 21742776, 35558948, 5893317, 2441637, 6907563, 36626257, 3366834, 25069218, 5753530, 45604388, 33908296, 1048890, 5720452, 7923351, 43424884, 43184720, 29744229, 10349400, 15273614, 15283237, 41997307 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2010-12-28"), "end-date": date("2010-09-17") } ] }
+{ "id": 9674677, "id-copy": 9674677, "alias": "Skye", "name": "SkyeTomlinson", "user-since": datetime("2006-02-02T19:15:10.000"), "user-since-copy": datetime("2006-02-02T19:15:10.000"), "friend-ids": {{ 24282798, 5600117, 33292938, 19518197, 11735189, 22867735, 8029689, 11269147, 7443311, 45905216, 12859442, 26944030 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2011-05-07"), "end-date": date("2011-04-19") } ] }
+{ "id": 9677293, "id-copy": 9677293, "alias": "Owen", "name": "OwenHoenshell", "user-since": datetime("2005-06-28T02:54:49.000"), "user-since-copy": datetime("2005-06-28T02:54:49.000"), "friend-ids": {{ 1016713, 4999321, 27107303, 15587298 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2010-01-11") } ] }
+{ "id": 9698980, "id-copy": 9698980, "alias": "Leland", "name": "LelandReiss", "user-since": datetime("2012-05-23T04:40:29.000"), "user-since-copy": datetime("2012-05-23T04:40:29.000"), "friend-ids": {{ 7623016, 12672253, 42612513, 44457047, 46981337, 1098470, 23122899, 15019916, 45345438, 30272843, 43546610 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-11-27") } ] }
+{ "id": 9733942, "id-copy": 9733942, "alias": "Andra", "name": "AndraConrad", "user-since": datetime("2007-01-23T01:20:01.000"), "user-since-copy": datetime("2007-01-23T01:20:01.000"), "friend-ids": {{ 42791827, 36987912, 12650269, 5310067, 33419819, 36880069, 1146970, 20314, 10762565, 20657888, 31871678, 42279496, 9831201, 4223369, 46820320, 21703772, 1326858, 21739453, 20082273, 12950360 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2002-03-21") } ] }
+{ "id": 9744016, "id-copy": 9744016, "alias": "Kasha", "name": "KashaMueller", "user-since": datetime("2011-03-16T17:17:31.000"), "user-since-copy": datetime("2011-03-16T17:17:31.000"), "friend-ids": {{ 15857660, 46791109, 10310040, 42863950, 19533508, 32561502, 4367358, 31952243, 7130063, 19536081, 19870534, 3642001, 910385, 28668446, 33204842, 13210089, 2805429 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2000-11-01") } ] }
+{ "id": 9748939, "id-copy": 9748939, "alias": "April", "name": "AprilCourtney", "user-since": datetime("2008-02-10T17:35:28.000"), "user-since-copy": datetime("2008-02-10T17:35:28.000"), "friend-ids": {{ 43018591, 38860193, 26524230, 23704979, 2293321, 18201469, 41569073, 26942967, 16348102, 20218840, 30888146, 7584389, 11355443, 3703344 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2001-08-26") } ] }
+{ "id": 9769501, "id-copy": 9769501, "alias": "Geffrey", "name": "GeffreyBurch", "user-since": datetime("2005-08-28T03:10:56.000"), "user-since-copy": datetime("2005-08-28T03:10:56.000"), "friend-ids": {{ 21060169, 45384418, 20564855, 24708101, 30231, 29383832, 9200835, 822161, 29674263, 619991, 38797966, 14299510, 13545166, 33027152 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2004-01-03"), "end-date": date("2006-04-13") } ] }
+{ "id": 9773836, "id-copy": 9773836, "alias": "Harris", "name": "HarrisAshmore", "user-since": datetime("2005-11-09T08:38:57.000"), "user-since-copy": datetime("2005-11-09T08:38:57.000"), "friend-ids": {{ 8138978, 18579002, 42663609, 12096643, 38992166, 36937135, 19634600, 2103929, 37072923, 25031081, 13379299, 11238246, 23166598, 19181943, 45382447, 8237252, 30986231, 29591835 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2000-02-12") } ] }
+{ "id": 9802330, "id-copy": 9802330, "alias": "Kirby", "name": "KirbyKnopsnider", "user-since": datetime("2011-12-18T01:10:12.000"), "user-since-copy": datetime("2011-12-18T01:10:12.000"), "friend-ids": {{ 3703876, 46564552, 9263120, 39930137, 36202804, 45164241, 7778394, 2527495, 2831079, 33834588, 42759211, 2766215, 36344152, 5218620, 1190357, 30615313, 25434877, 43958817, 23617510 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2008-02-01") } ] }
+{ "id": 9804196, "id-copy": 9804196, "alias": "Micheal", "name": "MichealEiford", "user-since": datetime("2009-05-21T02:55:17.000"), "user-since-copy": datetime("2009-05-21T02:55:17.000"), "friend-ids": {{ 31376257, 19749408, 5790154, 17891222, 15712036, 40911870, 40765983, 38804584, 24619388, 10957577, 35370581, 39352927, 6063001, 23702369, 14716580, 46589395, 35232946 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2003-07-27") } ] }
+{ "id": 9814867, "id-copy": 9814867, "alias": "Pacey", "name": "PaceyBranson", "user-since": datetime("2011-07-05T06:49:42.000"), "user-since-copy": datetime("2011-07-05T06:49:42.000"), "friend-ids": {{ 7196953 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2005-11-19"), "end-date": date("2007-12-03") } ] }
+{ "id": 9822973, "id-copy": 9822973, "alias": "Melia", "name": "MeliaWentzel", "user-since": datetime("2012-07-17T05:10:30.000"), "user-since-copy": datetime("2012-07-17T05:10:30.000"), "friend-ids": {{ 2563633, 27918474, 42233962, 40497985, 4437912, 43013491, 47283180, 20434605, 25309336, 11299381, 20584869, 15093618, 14273412, 46920368, 5868827, 40191100, 44286983, 11787568, 44551406 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2001-07-07") } ] }
+{ "id": 9856990, "id-copy": 9856990, "alias": "Claud", "name": "ClaudBaird", "user-since": datetime("2006-10-10T11:48:09.000"), "user-since-copy": datetime("2006-10-10T11:48:09.000"), "friend-ids": {{ 41756695, 15842897, 29797715, 13771892, 21179308, 42974840, 22223660, 35004748, 35597685, 45300254, 31116834, 42699991, 9704157, 23181215, 14806554, 8198556, 16256974, 16360634, 34736641 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2008-07-23") } ] }
+{ "id": 9878209, "id-copy": 9878209, "alias": "Duana", "name": "DuanaGettemy", "user-since": datetime("2007-03-05T19:06:27.000"), "user-since-copy": datetime("2007-03-05T19:06:27.000"), "friend-ids": {{ 5530171, 22409344, 22742046, 14418589, 27149252 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-07") } ] }
+{ "id": 9880603, "id-copy": 9880603, "alias": "Davis", "name": "DavisRitter", "user-since": datetime("2009-12-18T18:55:46.000"), "user-since-copy": datetime("2009-12-18T18:55:46.000"), "friend-ids": {{ 10790833, 43529865, 23457220, 6745186, 22333440, 39380793, 2096806, 44121543, 29345888, 46499780, 31896682, 35084540, 6060378, 27402271, 18954641 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2002-01-11") } ] }
+{ "id": 9886819, "id-copy": 9886819, "alias": "Phoebe", "name": "PhoebeBarnes", "user-since": datetime("2010-12-26T07:30:15.000"), "user-since-copy": datetime("2010-12-26T07:30:15.000"), "friend-ids": {{ 24361962, 43750816, 46566991, 4790101, 38827567, 6893116, 41555542, 35877264, 18479056, 22186674, 10954414, 43453344, 11903159, 12257863, 45299776 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2000-01-02"), "end-date": date("2008-05-24") } ] }
+{ "id": 9904822, "id-copy": 9904822, "alias": "Judith", "name": "JudithChristman", "user-since": datetime("2005-05-19T14:43:44.000"), "user-since-copy": datetime("2005-05-19T14:43:44.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "highfax", "start-date": date("2002-05-06") } ] }
+{ "id": 9917008, "id-copy": 9917008, "alias": "Clancy", "name": "ClancyHector", "user-since": datetime("2007-09-25T20:55:57.000"), "user-since-copy": datetime("2007-09-25T20:55:57.000"), "friend-ids": {{ 37754545, 37579706, 39121342, 28434988, 3927416, 3794736, 17107964, 20761621, 20497172, 28562441, 4310488, 35121288, 2380560, 32434056 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2011-06-24") } ] }
+{ "id": 9939937, "id-copy": 9939937, "alias": "Margeret", "name": "MargeretWhite", "user-since": datetime("2008-10-10T22:07:17.000"), "user-since-copy": datetime("2008-10-10T22:07:17.000"), "friend-ids": {{ 12369844, 34252449, 12412010, 16942281, 25231122, 42326296, 27054531, 8338820, 25466132, 10175756, 23763550, 40035149, 41030740, 36493305, 19615682, 30813330, 24869907, 6934392, 31309446, 2545800, 463498, 3089623, 12714051, 38317605 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2012-01-19") } ] }
+{ "id": 9945166, "id-copy": 9945166, "alias": "Lilly", "name": "LillyPirl", "user-since": datetime("2009-10-26T11:59:59.000"), "user-since-copy": datetime("2009-10-26T11:59:59.000"), "friend-ids": {{ 44569094, 5885974, 43165146, 40353390, 45117914, 35995608, 22535699, 46288114, 47171829, 14193764, 45832182, 4957844, 2623547, 37294528 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2009-12-15"), "end-date": date("2011-11-20") } ] }
+{ "id": 9952342, "id-copy": 9952342, "alias": "Christal", "name": "ChristalMcmichaels", "user-since": datetime("2008-02-13T13:25:45.000"), "user-since-copy": datetime("2008-02-13T13:25:45.000"), "friend-ids": {{ 12290348, 1563117, 10883525, 17285406, 3798829, 3734533, 13084348, 31001579, 23655942, 44480002, 11803789, 8240833, 42718608, 41919526, 37582304, 10494964, 10815416, 10676699, 9376307 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2011-05-16") } ] }
+{ "id": 9970132, "id-copy": 9970132, "alias": "Garrett", "name": "GarrettPery", "user-since": datetime("2007-03-03T11:19:29.000"), "user-since-copy": datetime("2007-03-03T11:19:29.000"), "friend-ids": {{ 25744707, 31991833, 37406793, 30461766, 24815522, 3640470, 13669903, 17663561, 19222132, 29107132, 42516393, 40032051, 24029037, 28047983, 45579233 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2001-01-12") } ] }
+{ "id": 9979750, "id-copy": 9979750, "alias": "Reginald", "name": "ReginaldAltman", "user-since": datetime("2007-04-04T08:51:58.000"), "user-since-copy": datetime("2007-04-04T08:51:58.000"), "friend-ids": {{ 2988287 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2002-01-28") } ] }
+{ "id": 9985393, "id-copy": 9985393, "alias": "Whitaker", "name": "WhitakerMang", "user-since": datetime("2007-11-28T09:34:34.000"), "user-since-copy": datetime("2007-11-28T09:34:34.000"), "friend-ids": {{ 24107735, 37165967, 31305236, 35313360, 9261860, 32724193, 34416346, 8143882, 9029425, 26723829, 4545824 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2000-08-23"), "end-date": date("2008-08-06") } ] }
+{ "id": 9986206, "id-copy": 9986206, "alias": "Tatiana", "name": "TatianaAlbright", "user-since": datetime("2006-03-21T10:00:55.000"), "user-since-copy": datetime("2006-03-21T10:00:55.000"), "friend-ids": {{ 42869099, 40178170, 13922993, 28844962, 26206785, 41293581, 17131809, 1583964, 47236558, 2656158, 11008100, 3994698, 23764118, 14275676, 4922979, 28466879, 16454954, 3620561, 42044685, 12665882, 18354684 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2012-04-24") } ] }
+{ "id": 10000456, "id-copy": 10000456, "alias": "Love", "name": "LoveHawker", "user-since": datetime("2011-03-01T20:42:05.000"), "user-since-copy": datetime("2011-03-01T20:42:05.000"), "friend-ids": {{ 33646270, 5736885, 35243769, 35528678, 43954964, 44975821, 1839952, 24025196, 1108928 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2010-11-23"), "end-date": date("2011-03-07") } ] }
+{ "id": 10001080, "id-copy": 10001080, "alias": "Garrett", "name": "GarrettBode", "user-since": datetime("2005-10-25T18:07:35.000"), "user-since-copy": datetime("2005-10-25T18:07:35.000"), "friend-ids": {{ 35858744, 16426061, 11473961, 4769664, 29038930, 33070686, 46271872, 42593454, 36202882, 46642640, 22243678, 20222041, 29014540, 7389258, 7172909, 12787979, 146736, 21081030, 21615179, 2936936, 44934891 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2007-06-24") } ] }
+{ "id": 10002907, "id-copy": 10002907, "alias": "Maegan", "name": "MaeganErschoff", "user-since": datetime("2011-10-15T18:08:56.000"), "user-since-copy": datetime("2011-10-15T18:08:56.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2009-02-06"), "end-date": date("2011-05-20") } ] }
+{ "id": 10045915, "id-copy": 10045915, "alias": "Mona", "name": "MonaMarshall", "user-since": datetime("2005-08-24T06:03:43.000"), "user-since-copy": datetime("2005-08-24T06:03:43.000"), "friend-ids": {{ 34157870, 1960568, 39038094, 2842182, 12353591, 44464974, 45836337, 4831806, 18179039, 21060089, 15776264, 41865218, 5999176, 18197780 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2006-07-28") } ] }
+{ "id": 10089976, "id-copy": 10089976, "alias": "Marion", "name": "MarionThomlinson", "user-since": datetime("2006-06-27T14:11:49.000"), "user-since-copy": datetime("2006-06-27T14:11:49.000"), "friend-ids": {{ 39404598, 46190974, 43413339, 41250692, 4194349, 5150083, 35574492, 30896673, 15969653, 41889132, 38801872, 17834003, 42587459, 42269051, 20206793, 46257713, 2735409, 28567746, 6641216, 3627253, 15945805, 33861471, 9997931, 38242090 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2011-11-22"), "end-date": date("2011-06-01") } ] }
+{ "id": 10108534, "id-copy": 10108534, "alias": "Moriah", "name": "MoriahMitchell", "user-since": datetime("2005-11-13T21:32:41.000"), "user-since-copy": datetime("2005-11-13T21:32:41.000"), "friend-ids": {{ 30372632 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-05-07") } ] }
+{ "id": 10114891, "id-copy": 10114891, "alias": "Destinee", "name": "DestineeLeech", "user-since": datetime("2006-06-05T09:32:17.000"), "user-since-copy": datetime("2006-06-05T09:32:17.000"), "friend-ids": {{ 9925448, 28685906, 3305693, 11131758, 10477741, 19058196, 25921997, 38543939, 20851041 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2001-09-24") } ] }
+{ "id": 10118077, "id-copy": 10118077, "alias": "Elizbeth", "name": "ElizbethPfeifer", "user-since": datetime("2011-09-08T11:58:48.000"), "user-since-copy": datetime("2011-09-08T11:58:48.000"), "friend-ids": {{ 18001251, 40309720, 10119557, 37766102, 22202316, 2805709, 693628, 5524288, 21415560, 45687644, 23912525, 25418741, 22816155, 26787291, 30518473, 27701649 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2002-03-15"), "end-date": date("2004-11-03") } ] }
+{ "id": 10133458, "id-copy": 10133458, "alias": "Kati", "name": "KatiPennington", "user-since": datetime("2011-01-28T10:51:37.000"), "user-since-copy": datetime("2011-01-28T10:51:37.000"), "friend-ids": {{ 41299906, 11523198, 8344474, 36086944, 34330342, 43585884, 6751565, 23415221, 32275829, 43645200 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2005-11-11") } ] }
+{ "id": 10136659, "id-copy": 10136659, "alias": "Robt", "name": "RobtKooser", "user-since": datetime("2008-11-08T19:22:49.000"), "user-since-copy": datetime("2008-11-08T19:22:49.000"), "friend-ids": {{ 22245145, 29285750, 9880896 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-02-07") } ] }
+{ "id": 10178182, "id-copy": 10178182, "alias": "Jen", "name": "JenOtis", "user-since": datetime("2007-08-09T09:42:29.000"), "user-since-copy": datetime("2007-08-09T09:42:29.000"), "friend-ids": {{ 26278603, 27983753, 13714345, 35452213, 27849291, 21838200, 1008530, 27777115, 27069057, 35804914, 34598070, 10076890, 12795361, 16653787, 2916026, 27047674, 8630755, 29822673 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2005-10-10") } ] }
+{ "id": 10185346, "id-copy": 10185346, "alias": "Noah", "name": "NoahAshmore", "user-since": datetime("2006-04-04T14:33:43.000"), "user-since-copy": datetime("2006-04-04T14:33:43.000"), "friend-ids": {{ 15819384, 46052301, 7102428, 7977240, 30337629, 31480307, 30013142, 4192580, 34814572, 6841517, 2253788, 31150059, 505825, 27897490, 11402219 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-06-06") } ] }
+{ "id": 10202302, "id-copy": 10202302, "alias": "Camila", "name": "CamilaKelley", "user-since": datetime("2010-04-17T06:57:52.000"), "user-since-copy": datetime("2010-04-17T06:57:52.000"), "friend-ids": {{ 21392718, 41703679, 41044232, 47307848, 13912958, 45329595, 33360889, 24572594, 23726460, 9181899, 42227287, 26565775, 12665691, 12244453, 26966326, 3189268, 41340076, 33904406, 38048631, 22870005 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2000-10-09") } ] }
+{ "id": 10205539, "id-copy": 10205539, "alias": "Raeburn", "name": "RaeburnWire", "user-since": datetime("2007-04-28T23:05:24.000"), "user-since-copy": datetime("2007-04-28T23:05:24.000"), "friend-ids": {{ 13609724, 40251506 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2000-08-09") } ] }
+{ "id": 10207636, "id-copy": 10207636, "alias": "Stewart", "name": "StewartHamilton", "user-since": datetime("2008-11-06T21:44:47.000"), "user-since-copy": datetime("2008-11-06T21:44:47.000"), "friend-ids": {{ 25417411, 7322723, 13495699, 47274757, 44964322, 4993843, 36429109, 11904558, 18759232, 45446850, 40537858, 40487724, 36200691, 6846408, 7421262, 2225424, 12997194 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2012-08-28"), "end-date": date("2012-08-29") } ] }
+{ "id": 10219465, "id-copy": 10219465, "alias": "Ros", "name": "RosSurrency", "user-since": datetime("2010-04-20T12:07:16.000"), "user-since-copy": datetime("2010-04-20T12:07:16.000"), "friend-ids": {{ 14365151, 47786936, 41386448, 10958072, 34068903, 28844652, 16749120, 16920092, 7474357, 35730197, 13732713, 26185093, 19486844, 13720196, 7483494, 16709415, 32998666, 31641404, 42939361, 20750447, 44343030, 17559252, 13810932 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-12-12"), "end-date": date("2010-05-04") } ] }
+{ "id": 10222144, "id-copy": 10222144, "alias": "Alvina", "name": "AlvinaTanner", "user-since": datetime("2007-10-15T04:24:14.000"), "user-since-copy": datetime("2007-10-15T04:24:14.000"), "friend-ids": {{ 44207447, 29837430, 407059, 4562324, 970458, 31348025, 16439061, 13011150, 23510630, 21529259, 8279487, 28052530, 36551405, 17492050, 17983056, 11834104, 242520, 9279232, 4179609, 28407763, 23038009, 36977762, 8779957, 15040402 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2006-05-27") } ] }
+{ "id": 10224400, "id-copy": 10224400, "alias": "Malvina", "name": "MalvinaPery", "user-since": datetime("2009-01-25T03:41:22.000"), "user-since-copy": datetime("2009-01-25T03:41:22.000"), "friend-ids": {{ 17095877, 17062955, 13129292, 31635980, 32747924, 902714, 32032985, 44944935, 30544897, 44429244 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2001-01-11"), "end-date": date("2011-04-10") } ] }
+{ "id": 10227844, "id-copy": 10227844, "alias": "Simon", "name": "SimonCoates", "user-since": datetime("2008-09-18T06:23:35.000"), "user-since-copy": datetime("2008-09-18T06:23:35.000"), "friend-ids": {{ 5847048, 15554997, 1367924, 17223026, 31605674, 38148868, 15521228, 37540102, 4103855, 39184726, 26130198, 43081715, 35929397, 28963043, 10703925 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2011-07-02") } ] }
+{ "id": 10230604, "id-copy": 10230604, "alias": "Courtney", "name": "CourtneyCountryman", "user-since": datetime("2012-03-05T08:49:56.000"), "user-since-copy": datetime("2012-03-05T08:49:56.000"), "friend-ids": {{ 28617094, 31170285, 26700577, 43586990, 12809105, 8131401, 15644912, 38127923, 7871621, 13276397, 41863539, 3715524, 13404150, 12834697, 237361, 41295097, 29471386, 19859329, 14312407, 79917, 42547367, 9661712, 30110962, 29137807 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2001-06-09"), "end-date": date("2004-06-04") } ] }
+{ "id": 10250857, "id-copy": 10250857, "alias": "Kandi", "name": "KandiFranks", "user-since": datetime("2010-11-24T19:47:41.000"), "user-since-copy": datetime("2010-11-24T19:47:41.000"), "friend-ids": {{ 44991748, 27655130, 7925482, 33419150, 18275478 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2002-04-08") } ] }
+{ "id": 10251805, "id-copy": 10251805, "alias": "Jericho", "name": "JerichoBaird", "user-since": datetime("2005-07-02T12:57:18.000"), "user-since-copy": datetime("2005-07-02T12:57:18.000"), "friend-ids": {{ 5748549, 47013396, 15858292, 458526, 28324553, 22401875, 21726858, 38878600, 29844738, 14547049, 11432495, 9227475 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2005-06-09"), "end-date": date("2011-11-01") } ] }
+{ "id": 10257028, "id-copy": 10257028, "alias": "Gary", "name": "GaryThompson", "user-since": datetime("2009-01-23T04:15:30.000"), "user-since-copy": datetime("2009-01-23T04:15:30.000"), "friend-ids": {{ 46006273, 33435458, 40976127, 42353737, 37166855, 14882549, 27357892, 31126471, 38151307, 38721200 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2008-05-03"), "end-date": date("2011-09-08") } ] }
+{ "id": 10258114, "id-copy": 10258114, "alias": "Chuck", "name": "ChuckGibson", "user-since": datetime("2012-07-20T03:48:15.000"), "user-since-copy": datetime("2012-07-20T03:48:15.000"), "friend-ids": {{ 32318205, 37049120, 26298456, 3281723, 14892306, 29998569, 29992020, 36383932, 15333422, 29670243 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2006-10-11"), "end-date": date("2011-09-02") } ] }
+{ "id": 10267057, "id-copy": 10267057, "alias": "Thomas", "name": "ThomasCook", "user-since": datetime("2008-03-02T23:04:31.000"), "user-since-copy": datetime("2008-03-02T23:04:31.000"), "friend-ids": {{ 23744020, 25995598, 40459051, 27658275, 10133202, 11434833, 29790727, 1672639, 19652058, 18554997, 37878642, 48016133, 46599310, 37105777, 36004129, 6402365, 9889815, 29589019, 1497208, 19269802, 43383394, 30936085 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2008-05-14"), "end-date": date("2008-07-10") } ] }
+{ "id": 10269349, "id-copy": 10269349, "alias": "Oneida", "name": "OneidaJube", "user-since": datetime("2010-11-18T02:17:28.000"), "user-since-copy": datetime("2010-11-18T02:17:28.000"), "friend-ids": {{ 12058841, 5816839, 33989309, 42710608, 27128355, 22765769, 30666197, 9009086, 7254731, 41783149, 10080163, 38431373, 35086196, 3607650 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2000-12-10") } ] }
+{ "id": 10271479, "id-copy": 10271479, "alias": "Leah", "name": "LeahKoepple", "user-since": datetime("2007-10-26T15:57:39.000"), "user-since-copy": datetime("2007-10-26T15:57:39.000"), "friend-ids": {{ 317362, 43304286, 35630504, 16014770, 43567734, 37946435, 7728583, 45620359, 43235478, 17133820, 22926471, 27438784, 43521614, 235789, 43107565, 21967424, 39119573, 1688079, 5463246, 10081045 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2012-06-14") } ] }
+{ "id": 10278607, "id-copy": 10278607, "alias": "Brenden", "name": "BrendenLombardi", "user-since": datetime("2012-02-13T05:59:40.000"), "user-since-copy": datetime("2012-02-13T05:59:40.000"), "friend-ids": {{ 2820692, 43529738, 38518064, 29672334, 24653037, 39717291, 14213502, 23982828, 47123006, 34213620, 5993185, 10068793, 47512414, 40682283, 26631237, 23442819, 9215972, 9003752, 31259126, 8467245, 32821220, 8582002, 42606040 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2001-06-16"), "end-date": date("2008-09-11") } ] }
+{ "id": 10280533, "id-copy": 10280533, "alias": "Normand", "name": "NormandAckerley", "user-since": datetime("2008-05-18T00:44:35.000"), "user-since-copy": datetime("2008-05-18T00:44:35.000"), "friend-ids": {{ 46908522, 2002203, 15632192, 3790633, 21300428, 15452344, 34478785, 18864214, 32842683, 10486268, 2496859 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2010-12-07") } ] }
+{ "id": 10300027, "id-copy": 10300027, "alias": "Cassie", "name": "CassieCarmichael", "user-since": datetime("2007-02-17T16:12:21.000"), "user-since-copy": datetime("2007-02-17T16:12:21.000"), "friend-ids": {{ 18690821, 9246387, 5425670, 8058755, 32156367, 29092478 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-03-06") } ] }
+{ "id": 10301008, "id-copy": 10301008, "alias": "Edgardo", "name": "EdgardoWheeler", "user-since": datetime("2012-04-27T03:11:16.000"), "user-since-copy": datetime("2012-04-27T03:11:16.000"), "friend-ids": {{ 44525957, 2368018 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2004-07-02"), "end-date": date("2009-04-13") } ] }
+{ "id": 10320979, "id-copy": 10320979, "alias": "Giuseppe", "name": "GiuseppePorter", "user-since": datetime("2006-10-21T21:56:23.000"), "user-since-copy": datetime("2006-10-21T21:56:23.000"), "friend-ids": {{ 34102109, 41585396, 8170669, 7376463, 11841426, 6745396, 35637670, 38513040, 26085708, 7577827, 4793535, 31185038, 9126, 502656, 18672743, 27688404, 19846788, 47731814, 42609593 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2012-06-08") } ] }
+{ "id": 10323868, "id-copy": 10323868, "alias": "Floyd", "name": "FloydCostello", "user-since": datetime("2007-12-17T05:45:55.000"), "user-since-copy": datetime("2007-12-17T05:45:55.000"), "friend-ids": {{ 16296950, 29360254, 19980961, 43395913, 46984972, 2696536, 9715184, 10851075, 25657111, 46730259, 9182621, 31950695, 46717390, 16664917, 38439464, 6987406, 28167105, 10608129, 11375117, 4306430, 31737185, 29321535, 7420588 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2000-08-21") } ] }
+{ "id": 10337950, "id-copy": 10337950, "alias": "Bibi", "name": "BibiCattley", "user-since": datetime("2007-11-16T11:08:34.000"), "user-since-copy": datetime("2007-11-16T11:08:34.000"), "friend-ids": {{ 24399247, 18391359, 18215808, 36042641, 19360937, 2039633, 17280287, 22159187, 31245932, 4767019, 3299881, 12321916, 22533524, 18760130, 31303729, 39565694, 21606207, 8226305, 16276064 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2003-02-25"), "end-date": date("2008-08-20") } ] }
+{ "id": 10365688, "id-copy": 10365688, "alias": "Innocent", "name": "InnocentBlatenberger", "user-since": datetime("2008-11-09T13:57:34.000"), "user-since-copy": datetime("2008-11-09T13:57:34.000"), "friend-ids": {{ 27902413, 27226238, 35017422, 28154221 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2011-09-13"), "end-date": date("2011-02-05") } ] }
+{ "id": 10367503, "id-copy": 10367503, "alias": "Tory", "name": "ToryBender", "user-since": datetime("2012-01-17T03:20:23.000"), "user-since-copy": datetime("2012-01-17T03:20:23.000"), "friend-ids": {{ 12035968, 32370161, 7506904, 40525754, 44978940, 28927429, 47139832, 9164811, 29534171, 3789973 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2009-02-05") } ] }
+{ "id": 10380031, "id-copy": 10380031, "alias": "Otha", "name": "OthaHaines", "user-since": datetime("2005-08-08T04:10:50.000"), "user-since-copy": datetime("2005-08-08T04:10:50.000"), "friend-ids": {{ 2710866, 28894512, 36379679, 32545673, 38671874, 16746916, 39103475, 19783615, 17514492, 42617267, 7461114, 17712393, 43474200, 3806350, 5065542, 35722940 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2011-05-07") } ] }
+{ "id": 10384705, "id-copy": 10384705, "alias": "Santos", "name": "SantosJames", "user-since": datetime("2011-05-07T11:54:13.000"), "user-since-copy": datetime("2011-05-07T11:54:13.000"), "friend-ids": {{ 43937179, 34015979, 7417213, 14660995, 19725400, 3931428, 7318379, 48016396, 44068471, 4577462, 38302695, 16520658, 40487183, 31181305, 11750148, 42688348, 42071075, 10641987, 28860865, 27686448, 40844612, 10817134 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2007-12-18") } ] }
+{ "id": 10391179, "id-copy": 10391179, "alias": "Raymond", "name": "RaymondHoopengarner", "user-since": datetime("2006-04-06T18:32:20.000"), "user-since-copy": datetime("2006-04-06T18:32:20.000"), "friend-ids": {{ 35664656, 36940003, 35836359, 25322876, 45895708, 14553421 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2005-08-05"), "end-date": date("2007-01-09") } ] }
+{ "id": 10394632, "id-copy": 10394632, "alias": "Marlin", "name": "MarlinLogue", "user-since": datetime("2011-08-28T14:57:40.000"), "user-since-copy": datetime("2011-08-28T14:57:40.000"), "friend-ids": {{ 45667126 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2004-07-03"), "end-date": date("2009-05-09") } ] }
+{ "id": 10400386, "id-copy": 10400386, "alias": "Marion", "name": "MarionBuck", "user-since": datetime("2006-06-22T03:35:25.000"), "user-since-copy": datetime("2006-06-22T03:35:25.000"), "friend-ids": {{ 35854700, 8766966, 41860546, 25745457, 12225165, 15412904, 39841282, 5879215, 24965438, 4636142, 43652954, 36414405, 34931848, 38550959, 30395999, 44263220, 8167212, 35555246, 11177002, 29078503 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2000-08-28") } ] }
+{ "id": 10413097, "id-copy": 10413097, "alias": "Lindsay", "name": "LindsayDoverspike", "user-since": datetime("2005-03-24T22:42:49.000"), "user-since-copy": datetime("2005-03-24T22:42:49.000"), "friend-ids": {{ 773762, 43764188, 23133486, 27099138, 38010544, 38283504, 38432745, 32450505, 34499948, 38200436, 44093983, 41684052, 41353940, 29027114, 2947798, 25212070, 9522627, 18680730, 13060818, 41586559 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2010-01-23"), "end-date": date("2011-01-14") } ] }
+{ "id": 10415575, "id-copy": 10415575, "alias": "Amabel", "name": "AmabelRoose", "user-since": datetime("2011-05-28T10:47:28.000"), "user-since-copy": datetime("2011-05-28T10:47:28.000"), "friend-ids": {{ 22120342, 22881927, 39043768, 27695122, 8669783, 25973892 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2000-03-14") } ] }
+{ "id": 10417531, "id-copy": 10417531, "alias": "Eileen", "name": "EileenCrissman", "user-since": datetime("2009-10-13T21:36:38.000"), "user-since-copy": datetime("2009-10-13T21:36:38.000"), "friend-ids": {{ 911579, 3590209, 15646563, 31960066, 14495212, 44915460, 42713118, 1962949, 44935091, 6578467, 21896024, 41455809, 25543039, 28884330, 44289305, 15569750, 32580470, 46016098, 9828368 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2003-06-11"), "end-date": date("2005-10-02") } ] }
+{ "id": 10451932, "id-copy": 10451932, "alias": "Kory", "name": "KoryRomanoff", "user-since": datetime("2008-09-27T13:29:18.000"), "user-since-copy": datetime("2008-09-27T13:29:18.000"), "friend-ids": {{ 21328124, 47569968, 22569123, 34316877, 36016117, 19944396, 34862141, 14875173, 3888684, 25235679, 7930355, 24991146, 2862320, 9552488, 23394143, 6292732, 23109993 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2008-04-25"), "end-date": date("2010-03-18") } ] }
+{ "id": 10458316, "id-copy": 10458316, "alias": "Nivek", "name": "NivekHarper", "user-since": datetime("2009-06-27T16:14:07.000"), "user-since-copy": datetime("2009-06-27T16:14:07.000"), "friend-ids": {{ 28377255, 40295259, 41434117, 37075748, 12913111, 1533923, 393103, 31161713, 13106373, 924904, 14927212, 7552938, 8299772, 28404911, 45464821, 34440085, 36216015, 2915789, 13470222, 34755494, 29250423 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2002-03-28"), "end-date": date("2010-12-09") } ] }
+{ "id": 10479073, "id-copy": 10479073, "alias": "Rhianna", "name": "RhiannaWerry", "user-since": datetime("2009-09-17T19:42:47.000"), "user-since-copy": datetime("2009-09-17T19:42:47.000"), "friend-ids": {{ 30293616, 42971604, 8411318, 37648744, 27412687, 17821200, 45008072 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2004-11-02"), "end-date": date("2011-06-24") } ] }
+{ "id": 10479190, "id-copy": 10479190, "alias": "Carmine", "name": "CarmineMortland", "user-since": datetime("2011-06-18T02:57:13.000"), "user-since-copy": datetime("2011-06-18T02:57:13.000"), "friend-ids": {{ 36090597, 35550849, 19614765, 34665409, 7740163, 12824683, 12997403, 32586142, 10137983, 44900811, 30392212, 43177710, 47792212 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2002-02-23") } ] }
+{ "id": 10492168, "id-copy": 10492168, "alias": "Savannah", "name": "SavannahRobinson", "user-since": datetime("2008-05-02T04:19:01.000"), "user-since-copy": datetime("2008-05-02T04:19:01.000"), "friend-ids": {{ 40126719, 38171650, 1474355, 6983398, 7918678, 45578368, 3210188, 29374863, 37758187, 2415003, 13746140, 44168763, 45798029, 17203664, 46309082, 21338452, 17217009, 24916114 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-07-20"), "end-date": date("2009-03-01") } ] }
+{ "id": 10493269, "id-copy": 10493269, "alias": "Anya", "name": "AnyaWoodward", "user-since": datetime("2009-03-08T07:08:04.000"), "user-since-copy": datetime("2009-03-08T07:08:04.000"), "friend-ids": {{ 2357333 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2001-05-04") } ] }
+{ "id": 10494370, "id-copy": 10494370, "alias": "Maria", "name": "MariaToke", "user-since": datetime("2009-12-06T17:40:38.000"), "user-since-copy": datetime("2009-12-06T17:40:38.000"), "friend-ids": {{ 28240347, 34042532 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2001-08-08"), "end-date": date("2008-07-09") } ] }
+{ "id": 10503262, "id-copy": 10503262, "alias": "Suzanne", "name": "SuzanneFonblanque", "user-since": datetime("2012-03-16T20:22:06.000"), "user-since-copy": datetime("2012-03-16T20:22:06.000"), "friend-ids": {{ 17868500, 500991, 7701699, 45401842, 16746916, 24217608, 46250003, 17567888, 28186634 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2001-01-18"), "end-date": date("2005-08-07") } ] }
+{ "id": 10509676, "id-copy": 10509676, "alias": "Dinorah", "name": "DinorahRopes", "user-since": datetime("2009-12-05T06:00:03.000"), "user-since-copy": datetime("2009-12-05T06:00:03.000"), "friend-ids": {{ 13297859, 17139775, 6500776, 46867326, 18510471, 20417055, 39500392, 2482383, 3361807, 14184772, 24928547, 14390842, 40519232, 14991589, 21242930, 24964529, 38160860, 25523267, 4709228, 13473948, 15850888, 30150938, 5984402, 26343874 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2010-08-11") } ] }
+{ "id": 10514095, "id-copy": 10514095, "alias": "Chantelle", "name": "ChantelleCatleay", "user-since": datetime("2008-10-23T00:05:15.000"), "user-since-copy": datetime("2008-10-23T00:05:15.000"), "friend-ids": {{ 11871759, 1505524, 45483061, 31479407, 15112731, 41816114, 22650998 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2008-06-14") } ] }
+{ "id": 10540825, "id-copy": 10540825, "alias": "Jayna", "name": "JaynaRowe", "user-since": datetime("2008-01-09T12:09:19.000"), "user-since-copy": datetime("2008-01-09T12:09:19.000"), "friend-ids": {{ 20315422, 9358699, 6204561, 40594838, 46678685, 34224970, 47262413, 42477325, 7591560, 39986319, 9438124, 30292072, 11187685, 27885, 47428887, 9535830, 36979072, 14613793 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2005-11-10") } ] }
+{ "id": 10567702, "id-copy": 10567702, "alias": "Zelda", "name": "ZeldaRitter", "user-since": datetime("2010-09-27T12:52:54.000"), "user-since-copy": datetime("2010-09-27T12:52:54.000"), "friend-ids": {{ 28336161, 20248788, 24723848, 8856879, 16831898, 7643547, 42868543, 23023606, 7531861, 36186817, 29113040, 995506, 32607538, 18755505, 44683178, 24627205, 39736850, 43535271, 385416, 40525568 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-11-27"), "end-date": date("2011-08-16") } ] }
+{ "id": 10582339, "id-copy": 10582339, "alias": "Randall", "name": "RandallDrabble", "user-since": datetime("2006-09-08T10:08:58.000"), "user-since-copy": datetime("2006-09-08T10:08:58.000"), "friend-ids": {{ 32686522, 24466673, 14026712, 31573032, 14639819, 19975138, 30208386, 24174917, 7234882, 9431452, 18256175, 18934583, 31539286, 46107937, 32747992, 28900739, 40079932, 40674667, 33527888, 45927633, 22350243, 14260823, 19696930, 17970296 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2008-12-13") } ] }
+{ "id": 10602166, "id-copy": 10602166, "alias": "Karine", "name": "KarineAdams", "user-since": datetime("2006-03-03T20:36:12.000"), "user-since-copy": datetime("2006-03-03T20:36:12.000"), "friend-ids": {{ 4463206, 23962283, 34321170, 10546383, 39886106, 37478996 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2000-10-28"), "end-date": date("2010-04-26") } ] }
+{ "id": 10623790, "id-copy": 10623790, "alias": "Leon", "name": "LeonSouthern", "user-since": datetime("2006-08-26T12:47:17.000"), "user-since-copy": datetime("2006-08-26T12:47:17.000"), "friend-ids": {{ 15974929, 10054172, 9775689, 22060162, 41777649, 13548836, 10842789, 45455670, 32027368, 45268626, 40570545, 18214851, 47559589, 38267347, 41101925, 45749689, 29277572, 47828706, 45708476, 33769625 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2012-06-22"), "end-date": date("2012-06-05") } ] }
+{ "id": 10650526, "id-copy": 10650526, "alias": "Gertie", "name": "GertieWallace", "user-since": datetime("2010-07-16T05:33:07.000"), "user-since-copy": datetime("2010-07-16T05:33:07.000"), "friend-ids": {{ 35934417, 43053648, 35859770, 43704932, 35605486, 17212020, 21235775, 26783725, 17450538, 42996452, 15873053, 36331217, 18524993, 45483950, 1549676, 24801562, 46527491 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2003-08-16") } ] }
+{ "id": 10661566, "id-copy": 10661566, "alias": "Cathy", "name": "CathyKight", "user-since": datetime("2007-07-17T18:53:31.000"), "user-since-copy": datetime("2007-07-17T18:53:31.000"), "friend-ids": {{ 19477294, 31919442, 6947933, 16858850, 21921187, 21214480, 19616226, 2133662, 42362248, 7534944, 12953803, 41148200, 30043772, 38130157, 36623612, 45371575, 25019205, 10260656 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2008-12-09"), "end-date": date("2008-01-04") } ] }
+{ "id": 10662082, "id-copy": 10662082, "alias": "Colbert", "name": "ColbertFylbrigg", "user-since": datetime("2005-04-09T18:04:54.000"), "user-since-copy": datetime("2005-04-09T18:04:54.000"), "friend-ids": {{ 25358191, 27442450, 16828484, 16821866, 7010321, 35271072, 32519925, 15521808, 35168957, 36812363, 18888093, 45727757, 30009499, 31505405, 27925036, 47549214, 20290733, 18290760, 36238437, 32377676 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-04-02") } ] }
+{ "id": 10663741, "id-copy": 10663741, "alias": "Gaylord", "name": "GaylordWynne", "user-since": datetime("2007-09-07T09:15:35.000"), "user-since-copy": datetime("2007-09-07T09:15:35.000"), "friend-ids": {{ 34508923, 28228552, 7714885, 16525247, 30914675, 8152699, 26553788, 8070452, 45739728 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2006-05-18"), "end-date": date("2008-04-07") } ] }
+{ "id": 10700431, "id-copy": 10700431, "alias": "Lessie", "name": "LessieRobinson", "user-since": datetime("2011-02-03T18:31:41.000"), "user-since-copy": datetime("2011-02-03T18:31:41.000"), "friend-ids": {{ 8174251, 46379649, 3507858, 13269282, 38334885, 12074283, 34128956, 46802811, 37285621, 15203773, 17611824, 47823053, 28609781, 31377970, 11077457, 3771375, 27529933, 170454, 38682017 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2002-01-08"), "end-date": date("2006-06-08") } ] }
+{ "id": 10712731, "id-copy": 10712731, "alias": "Abigail", "name": "AbigailKunkle", "user-since": datetime("2011-07-20T07:10:43.000"), "user-since-copy": datetime("2011-07-20T07:10:43.000"), "friend-ids": {{ 35920648, 38798778, 17160209, 16674423, 44247736, 45731986, 29605307, 123608, 46926535, 41274265, 36397206, 16900492, 19895463, 10043680, 42549381, 21006240, 13037274, 25867242, 34428167, 953419, 2284340, 32321044, 2351589, 30797666 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2002-08-11"), "end-date": date("2002-12-01") } ] }
+{ "id": 10735369, "id-copy": 10735369, "alias": "Cody", "name": "CodySchaeffer", "user-since": datetime("2008-07-03T05:27:24.000"), "user-since-copy": datetime("2008-07-03T05:27:24.000"), "friend-ids": {{ 15534779, 12333665, 10468027, 3865324, 39537208, 16999101, 9009757, 318331, 30123714, 10137427, 16481424 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2009-01-19") } ] }
+{ "id": 10739446, "id-copy": 10739446, "alias": "Urban", "name": "UrbanHair", "user-since": datetime("2010-12-28T02:29:19.000"), "user-since-copy": datetime("2010-12-28T02:29:19.000"), "friend-ids": {{ 31947556, 39058269, 43315882, 40575729, 4079275, 40689246, 22639555, 1422452, 28051313, 41854009, 30810426, 37406811, 20834349, 46933622, 28218698, 17239481, 33458180 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2006-05-21") } ] }
+{ "id": 10749553, "id-copy": 10749553, "alias": "Rolland", "name": "RollandMunshower", "user-since": datetime("2005-12-26T19:26:32.000"), "user-since-copy": datetime("2005-12-26T19:26:32.000"), "friend-ids": {{ 27080985, 4355429, 17027260, 30203290, 37292858, 1935550, 467329, 24265915, 4926329, 28586308, 27299677, 25356918, 14171255, 319307, 15014794 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2011-04-21") } ] }
+{ "id": 10777072, "id-copy": 10777072, "alias": "Fairy", "name": "FairyAgg", "user-since": datetime("2011-08-22T17:08:52.000"), "user-since-copy": datetime("2011-08-22T17:08:52.000"), "friend-ids": {{ 30447177, 24535470, 1763903, 4456057, 35013322 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2009-02-19") } ] }
+{ "id": 10783822, "id-copy": 10783822, "alias": "Emerald", "name": "EmeraldMillard", "user-since": datetime("2008-08-07T16:33:44.000"), "user-since-copy": datetime("2008-08-07T16:33:44.000"), "friend-ids": {{ 22464360, 7890894, 18256597, 33659179, 24554534, 30962087, 29716339, 23689397, 45113518, 19997635 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2001-06-10"), "end-date": date("2006-12-02") } ] }
+{ "id": 10799674, "id-copy": 10799674, "alias": "Dolores", "name": "DoloresPolson", "user-since": datetime("2006-03-24T00:54:47.000"), "user-since-copy": datetime("2006-03-24T00:54:47.000"), "friend-ids": {{ 40482317, 21393644, 151122, 13958566, 6524741, 1269094, 34703787, 38215473, 20258639, 144407, 23903205, 46922014, 26741209, 34932062, 1043581, 14090176, 45243069, 19226320, 33271281, 20215000, 46383495, 42405679, 42360649 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2009-07-18") } ] }
+{ "id": 10804771, "id-copy": 10804771, "alias": "Delicia", "name": "DeliciaPittman", "user-since": datetime("2008-04-12T01:07:13.000"), "user-since-copy": datetime("2008-04-12T01:07:13.000"), "friend-ids": {{ 35228090 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2006-08-16") } ] }
+{ "id": 10836430, "id-copy": 10836430, "alias": "Kaycee", "name": "KayceeCatleay", "user-since": datetime("2007-05-18T07:19:02.000"), "user-since-copy": datetime("2007-05-18T07:19:02.000"), "friend-ids": {{ 40568633, 44667158, 18923311, 34987631, 29306332, 38711535, 43999451, 3179954, 9799980, 3451381, 23204288, 17797804, 2164448, 16697308, 24697554, 45250786, 10029328, 27871642 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-04-25") } ] }
+{ "id": 10837876, "id-copy": 10837876, "alias": "Tianna", "name": "TiannaOppenheimer", "user-since": datetime("2006-05-14T01:19:23.000"), "user-since-copy": datetime("2006-05-14T01:19:23.000"), "friend-ids": {{ 8389212, 20540523, 37708985, 22298925, 5938365, 34705514, 39174355, 44283530, 44597508, 37912034, 45434053, 47086440, 6559664, 12451920, 47639456, 39030619, 24239344, 2566247, 27102794 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2008-04-03") } ] }
+{ "id": 10847359, "id-copy": 10847359, "alias": "Leone", "name": "LeoneWood", "user-since": datetime("2005-07-28T14:24:43.000"), "user-since-copy": datetime("2005-07-28T14:24:43.000"), "friend-ids": {{ 7650486, 39843416, 43272193, 47152762, 45218041, 45422234, 46812876, 18098636, 47174431, 19091549, 1405281, 46699360, 37961345, 43323551, 46824225, 30700451, 10188790, 16642374, 26570751 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2005-01-22") } ] }
+{ "id": 10851133, "id-copy": 10851133, "alias": "Wilbur", "name": "WilburDiegel", "user-since": datetime("2005-08-20T01:37:10.000"), "user-since-copy": datetime("2005-08-20T01:37:10.000"), "friend-ids": {{ 44811869, 15362002, 5320359, 4756538, 40097009, 905334, 44595717, 3685695, 35645656, 2090160, 35124514, 21715286, 26713020, 5816017, 15598653, 6425314, 10423130, 29593106, 14054734, 1780417, 38517315, 25570577, 5038946 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2004-05-04") } ] }
+{ "id": 10878553, "id-copy": 10878553, "alias": "Fido", "name": "FidoWillcox", "user-since": datetime("2007-01-10T01:06:54.000"), "user-since-copy": datetime("2007-01-10T01:06:54.000"), "friend-ids": {{ 28379360, 45087756, 15173549, 15693878, 23925453, 44178250, 26895550, 35260808, 9946110 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2012-02-09"), "end-date": date("2012-06-24") } ] }
+{ "id": 10896556, "id-copy": 10896556, "alias": "Kimberleigh", "name": "KimberleighWoolery", "user-since": datetime("2005-05-12T17:22:37.000"), "user-since-copy": datetime("2005-05-12T17:22:37.000"), "friend-ids": {{ 6300953, 46149018, 25478406, 577782, 38073266, 11461118, 10240145, 686269, 37990652, 26865957 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2007-05-03") } ] }
+{ "id": 10904125, "id-copy": 10904125, "alias": "Jarred", "name": "JarredRopes", "user-since": datetime("2005-11-09T09:53:06.000"), "user-since-copy": datetime("2005-11-09T09:53:06.000"), "friend-ids": {{ 26810, 23763346, 5064508, 26124598 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2007-12-28"), "end-date": date("2009-04-23") } ] }
+{ "id": 10905721, "id-copy": 10905721, "alias": "Tibby", "name": "TibbyPriebe", "user-since": datetime("2010-04-09T18:32:02.000"), "user-since-copy": datetime("2010-04-09T18:32:02.000"), "friend-ids": {{ 18406663, 1072532, 16897765 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2007-05-05"), "end-date": date("2007-03-06") } ] }
+{ "id": 10913971, "id-copy": 10913971, "alias": "Marylyn", "name": "MarylynBuehler", "user-since": datetime("2008-03-02T11:14:18.000"), "user-since-copy": datetime("2008-03-02T11:14:18.000"), "friend-ids": {{ 36555710, 21041383, 37895483, 11392039, 5195346, 12022072, 5206222, 37834919, 434970, 4441054, 39212196, 12773393 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2000-01-11") } ] }
+{ "id": 10936798, "id-copy": 10936798, "alias": "Chang", "name": "ChangBriner", "user-since": datetime("2011-01-21T02:58:13.000"), "user-since-copy": datetime("2011-01-21T02:58:13.000"), "friend-ids": {{ 44173597, 3293094, 47813131, 8981206, 36324479, 16594808, 20038389, 11223092, 7224123, 10682354, 7270314, 5170866, 10241023, 43090387, 21910381, 36504407, 18319458, 19534667, 14493618, 11394344, 5990164, 35322441 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2004-12-09"), "end-date": date("2006-08-28") } ] }
+{ "id": 10948315, "id-copy": 10948315, "alias": "Munro", "name": "MunroDiegel", "user-since": datetime("2006-11-24T10:55:36.000"), "user-since-copy": datetime("2006-11-24T10:55:36.000"), "friend-ids": {{ 46912879, 47760999, 8438850, 12005776, 7286415, 41598308, 42462653, 2040525, 8432844, 39644931 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2011-08-08"), "end-date": date("2011-09-27") } ] }
+{ "id": 10955896, "id-copy": 10955896, "alias": "Felton", "name": "FeltonRiggle", "user-since": datetime("2010-08-18T08:55:19.000"), "user-since-copy": datetime("2010-08-18T08:55:19.000"), "friend-ids": {{ 9250996, 46302470, 16921353, 21053478, 40274566, 25492381, 7743899 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-09-10"), "end-date": date("2009-01-22") } ] }
+{ "id": 10967305, "id-copy": 10967305, "alias": "Harrietta", "name": "HarriettaClewett", "user-since": datetime("2008-05-11T02:34:28.000"), "user-since-copy": datetime("2008-05-11T02:34:28.000"), "friend-ids": {{ 3346670, 25522849, 46919524, 22773543, 8985252, 43521041, 14951485, 45977993, 21285106, 17023357, 615364, 23079537, 23459313, 31663735, 24201883, 39321873, 47183802, 26870642, 34447310, 4848880, 17078809, 14119447, 39460378 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2012-05-12"), "end-date": date("2012-06-25") } ] }
+{ "id": 10975810, "id-copy": 10975810, "alias": "Davin", "name": "DavinKifer", "user-since": datetime("2005-08-19T20:23:07.000"), "user-since-copy": datetime("2005-08-19T20:23:07.000"), "friend-ids": {{ 20162027, 7842505, 3191764, 11487126, 44589086, 14959953, 18826364, 18917713, 37717273, 24319173, 1393081, 19608709, 47932966, 37681921, 47734310, 21616345, 21035793, 9650227, 43642280, 21890130, 17249802, 27944839 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2011-12-17"), "end-date": date("2011-12-01") } ] }
+{ "id": 10989949, "id-copy": 10989949, "alias": "Kaylyn", "name": "KaylynElder", "user-since": datetime("2011-01-13T12:02:13.000"), "user-since-copy": datetime("2011-01-13T12:02:13.000"), "friend-ids": {{ 22698118, 31639011, 11500577, 13007617, 26781164, 20827141, 9916306, 26415081, 14027605, 19305199, 45276489, 17632806, 42243983 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2006-01-05") } ] }
+{ "id": 10992421, "id-copy": 10992421, "alias": "Ashleigh", "name": "AshleighStroh", "user-since": datetime("2009-10-20T03:03:48.000"), "user-since-copy": datetime("2009-10-20T03:03:48.000"), "friend-ids": {{ 34581685, 36997971, 29555907, 34868441, 31092587, 9963667, 60170, 19708784, 26201942, 27806479, 40464656, 27628428, 5144660, 44794976, 9937339 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2001-11-04") } ] }
+{ "id": 11001610, "id-copy": 11001610, "alias": "Keven", "name": "KevenWildman", "user-since": datetime("2006-09-07T02:21:33.000"), "user-since-copy": datetime("2006-09-07T02:21:33.000"), "friend-ids": {{ 14316856, 4291050 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-06-20"), "end-date": date("2012-06-09") } ] }
+{ "id": 11003527, "id-copy": 11003527, "alias": "Clitus", "name": "ClitusDickinson", "user-since": datetime("2007-10-18T04:59:18.000"), "user-since-copy": datetime("2007-10-18T04:59:18.000"), "friend-ids": {{ 26264340, 47892511, 18715043, 43994375, 42874707, 44696774, 7281939 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2006-09-15") } ] }
+{ "id": 11010904, "id-copy": 11010904, "alias": "Chang", "name": "ChangSteele", "user-since": datetime("2009-02-24T01:43:56.000"), "user-since-copy": datetime("2009-02-24T01:43:56.000"), "friend-ids": {{ 19212881, 4019921, 24976558, 47613555, 26049623, 17656988, 24011085, 31763054, 21741933, 31356824, 9651386, 35034682, 5665574, 31306405, 38922156, 9837341, 31865250, 12415354 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2005-09-20"), "end-date": date("2005-05-28") } ] }
+{ "id": 11016238, "id-copy": 11016238, "alias": "Justy", "name": "JustyShaner", "user-since": datetime("2008-06-17T22:08:29.000"), "user-since-copy": datetime("2008-06-17T22:08:29.000"), "friend-ids": {{ 23689951, 17071721, 9194411, 34128749, 46316500, 31173605, 32802286, 26107462, 6561314, 9993897, 14746369, 7297148, 41466258 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2003-12-12"), "end-date": date("2007-04-12") } ] }
+{ "id": 11022826, "id-copy": 11022826, "alias": "Virgee", "name": "VirgeeHolts", "user-since": datetime("2012-01-17T22:54:54.000"), "user-since-copy": datetime("2012-01-17T22:54:54.000"), "friend-ids": {{ 40134062, 13624785, 23477090, 26708578, 18967215, 21325604, 15522457, 25873528 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2004-05-09"), "end-date": date("2010-06-15") } ] }
+{ "id": 11022889, "id-copy": 11022889, "alias": "Aubrey", "name": "AubreyMccallum", "user-since": datetime("2009-08-17T02:42:54.000"), "user-since-copy": datetime("2009-08-17T02:42:54.000"), "friend-ids": {{ 22265320, 4304911, 3403321, 20791603, 31499855, 22278594, 14580040, 31651270, 14509751, 13733306, 10947101, 7713960 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2001-01-11") } ] }
+{ "id": 11049715, "id-copy": 11049715, "alias": "Carlo", "name": "CarloBrooks", "user-since": datetime("2005-03-23T21:46:06.000"), "user-since-copy": datetime("2005-03-23T21:46:06.000"), "friend-ids": {{ 8214850, 7465603, 15385071, 32299168, 5993026, 3262895, 24995417, 25987462, 10230501, 12537459, 44597291, 33492282, 30758369, 15589085, 6799067, 23023304, 42597416, 10978280, 40668626, 25650335, 37336071 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-09-15"), "end-date": date("2011-09-03") } ] }
+{ "id": 11059435, "id-copy": 11059435, "alias": "Lucina", "name": "LucinaDurstine", "user-since": datetime("2007-04-14T19:19:23.000"), "user-since-copy": datetime("2007-04-14T19:19:23.000"), "friend-ids": {{ 18983436, 36225185, 42601602, 22134709, 20671612 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2009-06-15") } ] }
+{ "id": 11068231, "id-copy": 11068231, "alias": "Dinah", "name": "DinahSwink", "user-since": datetime("2012-05-02T04:24:33.000"), "user-since-copy": datetime("2012-05-02T04:24:33.000"), "friend-ids": {{ 31542440, 17451543, 32642661, 27867264, 32718667, 43042567, 7921827 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2003-04-10"), "end-date": date("2003-10-03") } ] }
+{ "id": 11072782, "id-copy": 11072782, "alias": "Jewel", "name": "JewelSchreckengost", "user-since": datetime("2012-06-04T18:20:29.000"), "user-since-copy": datetime("2012-06-04T18:20:29.000"), "friend-ids": {{ 47896348, 34649239, 38135221, 19731900, 14383059, 3639686, 28133949, 1326525, 415048, 34486382, 32809579, 31754806, 33563370 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-03-06") } ] }
+{ "id": 11089501, "id-copy": 11089501, "alias": "Antonette", "name": "AntonetteBrandenburg", "user-since": datetime("2010-01-02T05:42:44.000"), "user-since-copy": datetime("2010-01-02T05:42:44.000"), "friend-ids": {{ 18054329, 21707156, 1570987, 17610288, 32279976, 10880989, 37459189, 9057880, 46495123, 29331373, 20615029, 22282366, 22218648, 15950453, 30669615, 46097959, 16640911, 15896647 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2004-10-01"), "end-date": date("2009-02-20") } ] }
+{ "id": 11116465, "id-copy": 11116465, "alias": "Read", "name": "ReadOppenheimer", "user-since": datetime("2012-08-23T03:38:20.000"), "user-since-copy": datetime("2012-08-23T03:38:20.000"), "friend-ids": {{ 18679034, 12828526, 13510152, 28052139, 20367021, 30392195, 41580515, 2644015, 29573423, 22838698 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2009-11-05") } ] }
+{ "id": 11130676, "id-copy": 11130676, "alias": "Krystal", "name": "KrystalDavis", "user-since": datetime("2008-08-18T00:59:11.000"), "user-since-copy": datetime("2008-08-18T00:59:11.000"), "friend-ids": {{ 44775993, 31503397, 32012007, 16923302, 37099907, 14276165, 40040126, 38310068 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2003-11-21") } ] }
+{ "id": 11130781, "id-copy": 11130781, "alias": "Kenia", "name": "KeniaMiller", "user-since": datetime("2008-05-27T02:28:18.000"), "user-since-copy": datetime("2008-05-27T02:28:18.000"), "friend-ids": {{ 43139868, 16103105, 25352928, 23612973, 9645914, 20517323, 40438742, 47972276, 7395189, 44164898, 2805123, 33235701, 39846510, 21170026, 14223369, 14077979 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-06-24"), "end-date": date("2011-04-08") } ] }
+{ "id": 11131138, "id-copy": 11131138, "alias": "Maximillian", "name": "MaximillianSloan", "user-since": datetime("2009-12-26T13:02:42.000"), "user-since-copy": datetime("2009-12-26T13:02:42.000"), "friend-ids": {{ 4007900, 16474597, 36917058, 46709116, 35833748, 7074328, 6125321, 40646485, 23690629, 3251896, 3973740, 17863849, 9389737, 26501803, 4207105 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2010-10-16") } ] }
+{ "id": 11131756, "id-copy": 11131756, "alias": "Sharlene", "name": "SharleneFinlay", "user-since": datetime("2006-01-11T00:34:50.000"), "user-since-copy": datetime("2006-01-11T00:34:50.000"), "friend-ids": {{ 47024803, 17225785, 29871165, 14503159, 22992924, 38939801, 44563447, 101625, 40957129, 24838380, 7187619, 45283524, 31617405, 517806, 28714183, 32966332, 24006006 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2008-02-16"), "end-date": date("2011-09-12") } ] }
+{ "id": 11135899, "id-copy": 11135899, "alias": "Bailey", "name": "BaileyMoonshower", "user-since": datetime("2011-08-28T07:36:28.000"), "user-since-copy": datetime("2011-08-28T07:36:28.000"), "friend-ids": {{ 29802790, 16418079 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2010-05-17") } ] }
+{ "id": 11139106, "id-copy": 11139106, "alias": "Faith", "name": "FaithHicks", "user-since": datetime("2008-01-08T07:44:36.000"), "user-since-copy": datetime("2008-01-08T07:44:36.000"), "friend-ids": {{ 5409553, 11995627, 30724106, 17065157, 29513453, 38627025, 34382279, 36487812, 4292416, 19328709, 42169589, 18029462, 20202054, 8738011, 18339448, 2522742, 35366856, 10669527, 44287935, 47124982, 25912125, 38893810, 42212137, 22227146 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2000-11-15"), "end-date": date("2002-10-01") } ] }
+{ "id": 11145823, "id-copy": 11145823, "alias": "Rebeccah", "name": "RebeccahTodd", "user-since": datetime("2007-03-25T15:13:08.000"), "user-since-copy": datetime("2007-03-25T15:13:08.000"), "friend-ids": {{ 46132741, 11527757, 27573172, 45663865, 45572803, 30569464, 31892238 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-04-07") } ] }
+{ "id": 11147050, "id-copy": 11147050, "alias": "Karena", "name": "KarenaTanner", "user-since": datetime("2007-03-17T08:50:48.000"), "user-since-copy": datetime("2007-03-17T08:50:48.000"), "friend-ids": {{ 39952587, 2518830, 30305705, 21365609, 45914603, 2590495, 8595660 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2000-11-13"), "end-date": date("2009-01-10") } ] }
+{ "id": 11158711, "id-copy": 11158711, "alias": "Gwendolen", "name": "GwendolenBousum", "user-since": datetime("2007-07-06T10:35:24.000"), "user-since-copy": datetime("2007-07-06T10:35:24.000"), "friend-ids": {{ 22558162, 31443428, 22992355, 19452651, 23323540, 41272500, 17328954, 37489389, 35041092, 42476655 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2005-05-23") } ] }
+{ "id": 11179192, "id-copy": 11179192, "alias": "Derren", "name": "DerrenClose", "user-since": datetime("2008-04-28T09:18:19.000"), "user-since-copy": datetime("2008-04-28T09:18:19.000"), "friend-ids": {{ 43947479, 30154889, 10673575, 8056171, 28691242, 22881730, 15291446, 7331632, 32819016, 35194153 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-10-24"), "end-date": date("2006-08-12") } ] }
+{ "id": 11187373, "id-copy": 11187373, "alias": "Garfield", "name": "GarfieldWible", "user-since": datetime("2009-06-19T05:22:16.000"), "user-since-copy": datetime("2009-06-19T05:22:16.000"), "friend-ids": {{ 24453777, 20841948, 12224610, 30351943, 17826670, 36119836, 27850423, 4004658, 42610631, 25893845, 46022891, 33018964, 37844844, 1705377, 38811008, 36802000 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2000-02-22") } ] }
+{ "id": 11190361, "id-copy": 11190361, "alias": "Jancis", "name": "JancisFeufer", "user-since": datetime("2005-08-04T13:00:03.000"), "user-since-copy": datetime("2005-08-04T13:00:03.000"), "friend-ids": {{ 29421411, 15938833, 13248806, 1321174, 32401361, 34058563, 39735399, 35531531, 2631116, 1167996, 18366452, 45021961, 246133 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2003-08-27") } ] }
+{ "id": 11196118, "id-copy": 11196118, "alias": "Carson", "name": "CarsonBusk", "user-since": datetime("2006-07-23T07:08:34.000"), "user-since-copy": datetime("2006-07-23T07:08:34.000"), "friend-ids": {{ 36454884, 31755449, 44569587 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2007-08-13") } ] }
+{ "id": 11220541, "id-copy": 11220541, "alias": "Phyllida", "name": "PhyllidaRing", "user-since": datetime("2012-03-01T06:11:58.000"), "user-since-copy": datetime("2012-03-01T06:11:58.000"), "friend-ids": {{ 609357, 45820919, 17439004, 16790980, 27878958, 13930012, 20759108, 23987257, 29330180, 9298668, 10644382, 2596101, 29705735, 13371057, 41709459, 6973880, 41608321, 41344973, 9555209, 37508452, 26445359, 7693361, 12059348 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-12-05"), "end-date": date("2009-09-16") } ] }
+{ "id": 11230663, "id-copy": 11230663, "alias": "Caryl", "name": "CarylSmail", "user-since": datetime("2006-03-17T16:52:51.000"), "user-since-copy": datetime("2006-03-17T16:52:51.000"), "friend-ids": {{ 32153460, 21186863, 24199212, 25220508, 26590053, 42433121, 35372685 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-12-05") } ] }
+{ "id": 11233525, "id-copy": 11233525, "alias": "Syd", "name": "SydSauter", "user-since": datetime("2010-12-18T02:44:55.000"), "user-since-copy": datetime("2010-12-18T02:44:55.000"), "friend-ids": {{ 6312313, 17431246, 36729581, 3715101, 39534341, 10333995, 36042764, 14014852, 27375328, 17089631, 24066240, 42616402, 34049424, 29807262, 25669160, 43435752, 46702290, 27418631, 13587383, 14811241 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2010-06-21") } ] }
+{ "id": 11235340, "id-copy": 11235340, "alias": "Maurice", "name": "MauriceHayhurst", "user-since": datetime("2008-12-24T05:11:37.000"), "user-since-copy": datetime("2008-12-24T05:11:37.000"), "friend-ids": {{ 36045307, 37144109, 37142113, 38379399, 21011762, 30698208, 3185430, 24698099, 39750599, 1820110, 19740583, 5658727, 33165497, 27066109, 20299488, 26484094, 17984991, 9623240, 15287433, 32468842, 34023148, 16744372, 30389952, 40305465 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2011-05-19"), "end-date": date("2011-11-15") } ] }
+{ "id": 11244439, "id-copy": 11244439, "alias": "Francene", "name": "FranceneArmstrong", "user-since": datetime("2009-11-12T19:32:27.000"), "user-since-copy": datetime("2009-11-12T19:32:27.000"), "friend-ids": {{ 27784445, 37528954, 14014093, 18695376 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-06-26") } ] }
+{ "id": 11252185, "id-copy": 11252185, "alias": "Quintin", "name": "QuintinMcdonald", "user-since": datetime("2010-09-27T08:09:51.000"), "user-since-copy": datetime("2010-09-27T08:09:51.000"), "friend-ids": {{ 17231767, 1840658, 32389773, 31328720, 18446903, 48007173, 40417004, 41543048, 4774035, 43047815, 24232919, 936390, 20744224, 39536211, 34205950, 38429209, 399190, 38425767, 8776604, 10360244, 28414116, 15735235, 6431904 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-10-04") } ] }
+{ "id": 11262439, "id-copy": 11262439, "alias": "Alexandra", "name": "AlexandraStocker", "user-since": datetime("2010-08-28T03:48:52.000"), "user-since-copy": datetime("2010-08-28T03:48:52.000"), "friend-ids": {{ 16331707 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2010-12-04") } ] }
+{ "id": 11281576, "id-copy": 11281576, "alias": "Louisa", "name": "LouisaWheeler", "user-since": datetime("2005-01-19T05:34:26.000"), "user-since-copy": datetime("2005-01-19T05:34:26.000"), "friend-ids": {{ 29655724, 29204886, 24086191, 36260050, 502778, 368888, 42853595, 40434954, 46768026, 17096472, 33160972, 15621748, 46246949, 14174435, 99088, 44271646, 3676253, 11744063, 21957250, 34611796, 32735521, 45352911, 6097178, 3796892 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-03-05") } ] }
+{ "id": 11287666, "id-copy": 11287666, "alias": "Darian", "name": "DarianHurst", "user-since": datetime("2009-05-11T03:33:37.000"), "user-since-copy": datetime("2009-05-11T03:33:37.000"), "friend-ids": {{ 34901893, 38687373, 30369991, 44597588, 41413513, 24197212, 36791517, 19949174, 23092611, 29695794, 7024108, 25202811, 10231736, 3754404, 15863600, 30772236, 21615658 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2012-04-12"), "end-date": date("2012-05-07") } ] }
+{ "id": 11289733, "id-copy": 11289733, "alias": "Jettie", "name": "JettieElinor", "user-since": datetime("2006-03-02T09:44:17.000"), "user-since-copy": datetime("2006-03-02T09:44:17.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2002-07-25"), "end-date": date("2005-01-16") } ] }
+{ "id": 11302930, "id-copy": 11302930, "alias": "Eustace", "name": "EustaceKava", "user-since": datetime("2011-08-24T18:08:32.000"), "user-since-copy": datetime("2011-08-24T18:08:32.000"), "friend-ids": {{ 31173988, 7044500, 11649679, 34385410, 3097267, 24759223, 20452579, 7436501, 4500062, 765860, 14592959, 582267, 25586360, 6035361, 38333776, 47384154, 22158173 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2004-05-24") } ] }
+{ "id": 11313361, "id-copy": 11313361, "alias": "Lashawn", "name": "LashawnSchuth", "user-since": datetime("2006-08-24T02:37:43.000"), "user-since-copy": datetime("2006-08-24T02:37:43.000"), "friend-ids": {{ 3844342, 31605302, 11335667, 3890958 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2001-04-04"), "end-date": date("2006-12-03") } ] }
+{ "id": 11316178, "id-copy": 11316178, "alias": "Carlene", "name": "CarleneArchibald", "user-since": datetime("2007-09-02T16:24:57.000"), "user-since-copy": datetime("2007-09-02T16:24:57.000"), "friend-ids": {{ 45522809, 33213012, 2265630, 27087141, 7247502, 38659338, 33327692, 43927391, 41809132, 4738869, 9663680, 45809341, 38204579, 17145650, 23991333, 9915598, 28129675, 47406993, 37554697 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2007-12-15"), "end-date": date("2008-06-02") } ] }
+{ "id": 11321269, "id-copy": 11321269, "alias": "Wilford", "name": "WilfordFuhrer", "user-since": datetime("2012-01-25T14:53:32.000"), "user-since-copy": datetime("2012-01-25T14:53:32.000"), "friend-ids": {{ 6210425, 27216911, 3113058, 28094966, 119775, 805604, 43386400, 46812881, 22339620, 46498863, 26422270, 43219229, 40022359, 39446155 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2001-07-06") } ] }
+{ "id": 11357614, "id-copy": 11357614, "alias": "Denys", "name": "DenysMcintosh", "user-since": datetime("2006-01-15T22:32:48.000"), "user-since-copy": datetime("2006-01-15T22:32:48.000"), "friend-ids": {{ 10713170, 21699820, 14949046, 7935772, 21404351, 21078565, 15867691, 41676271, 2655928, 22987809, 16585582, 8318693, 46886662, 15081903, 47617713, 6317213, 32997127 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2008-08-28") } ] }
+{ "id": 11373598, "id-copy": 11373598, "alias": "Dina", "name": "DinaDriggers", "user-since": datetime("2010-01-06T22:56:18.000"), "user-since-copy": datetime("2010-01-06T22:56:18.000"), "friend-ids": {{ 8839886, 10146989, 10877857, 11710726, 5699142, 27984085, 12834284 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2012-07-25") } ] }
+{ "id": 11381089, "id-copy": 11381089, "alias": "Earlene", "name": "EarleneAmmons", "user-since": datetime("2010-03-24T05:25:35.000"), "user-since-copy": datetime("2010-03-24T05:25:35.000"), "friend-ids": {{ 25392364, 36996951, 16110083, 9799716, 22893553, 28551996, 7706432, 14225386, 15633254, 39395931, 46707062, 37226919, 8532306, 3765988, 20939685, 31136325, 45222021, 15355741, 8760941, 12045616, 6890610, 13560532, 44914868, 37744233 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2000-06-10") } ] }
+{ "id": 11427025, "id-copy": 11427025, "alias": "Kyran", "name": "KyranKlockman", "user-since": datetime("2007-11-24T11:35:40.000"), "user-since-copy": datetime("2007-11-24T11:35:40.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2004-06-10"), "end-date": date("2008-10-25") } ] }
+{ "id": 11435779, "id-copy": 11435779, "alias": "Jonty", "name": "JontyLarson", "user-since": datetime("2012-04-11T08:34:47.000"), "user-since-copy": datetime("2012-04-11T08:34:47.000"), "friend-ids": {{ 37343432, 9979565, 14647518, 32490112, 26673699, 22447290, 40923710, 47426439 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2010-08-24"), "end-date": date("2011-06-21") } ] }
+{ "id": 11454253, "id-copy": 11454253, "alias": "Fairy", "name": "FairyFoster", "user-since": datetime("2007-05-04T11:48:12.000"), "user-since-copy": datetime("2007-05-04T11:48:12.000"), "friend-ids": {{ 15077027, 13719617, 3663639, 16159577, 29937764, 11018999, 36883485, 35967804, 16558412, 19456409, 33156277, 8763694, 9279896 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2012-07-10") } ] }
+{ "id": 11455492, "id-copy": 11455492, "alias": "Cymbeline", "name": "CymbelineEliza", "user-since": datetime("2010-05-03T21:32:10.000"), "user-since-copy": datetime("2010-05-03T21:32:10.000"), "friend-ids": {{ 27738860, 21711920, 47805508, 33507501, 22648267, 1006513, 23617648, 20104970, 8132761, 14963107, 19477123 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2002-07-25") } ] }
+{ "id": 11515477, "id-copy": 11515477, "alias": "Kassandra", "name": "KassandraByers", "user-since": datetime("2005-05-24T10:27:06.000"), "user-since-copy": datetime("2005-05-24T10:27:06.000"), "friend-ids": {{ 23979652, 25789717, 7769765, 30747470, 30667193, 22447318, 42934938, 24601934, 31839813, 18960206, 30913033, 39059809, 18213877, 3731518, 10573130, 37902022 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-01-13") } ] }
+{ "id": 11515828, "id-copy": 11515828, "alias": "Christa", "name": "ChristaWain", "user-since": datetime("2007-05-01T13:32:18.000"), "user-since-copy": datetime("2007-05-01T13:32:18.000"), "friend-ids": {{ 9081871, 27897837, 47641133, 1224070, 41007475, 39553691, 10757036, 28663201, 44842180, 24894191, 42128523, 30703082, 27281648, 9786943 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2012-05-04") } ] }
+{ "id": 11529730, "id-copy": 11529730, "alias": "Linwood", "name": "LinwoodZadovsky", "user-since": datetime("2007-03-13T03:41:20.000"), "user-since-copy": datetime("2007-03-13T03:41:20.000"), "friend-ids": {{ 23516069, 24312236, 23750591, 36982495, 36483830 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2008-01-25") } ] }
+{ "id": 11534575, "id-copy": 11534575, "alias": "Sena", "name": "SenaWeidemann", "user-since": datetime("2008-05-25T01:11:53.000"), "user-since-copy": datetime("2008-05-25T01:11:53.000"), "friend-ids": {{ 8564372, 20258364, 35812476, 36877724, 30983504, 17757915, 42833517 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2003-04-22") } ] }
+{ "id": 11559613, "id-copy": 11559613, "alias": "Mick", "name": "MickWilkinson", "user-since": datetime("2005-12-23T15:11:33.000"), "user-since-copy": datetime("2005-12-23T15:11:33.000"), "friend-ids": {{ 4641355 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2000-06-03") } ] }
+{ "id": 11587057, "id-copy": 11587057, "alias": "Meagan", "name": "MeaganHays", "user-since": datetime("2012-08-15T21:45:05.000"), "user-since-copy": datetime("2012-08-15T21:45:05.000"), "friend-ids": {{ 26887765, 1940688, 10308941, 42037682, 1716669, 38995955, 17690888, 23227010, 4054166, 22275630, 6863237, 15140164, 38703696, 19044355, 43996569, 12255978, 28516070 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2003-02-26"), "end-date": date("2010-08-05") } ] }
+{ "id": 11588467, "id-copy": 11588467, "alias": "Soon", "name": "SoonHays", "user-since": datetime("2011-12-21T05:33:54.000"), "user-since-copy": datetime("2011-12-21T05:33:54.000"), "friend-ids": {{ 659930 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2009-05-20"), "end-date": date("2009-07-16") } ] }
+{ "id": 11591713, "id-copy": 11591713, "alias": "Nannie", "name": "NannieDiller", "user-since": datetime("2008-11-27T08:31:02.000"), "user-since-copy": datetime("2008-11-27T08:31:02.000"), "friend-ids": {{ 26059738, 32515289, 13702345, 16949001, 10188160, 30251286 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2000-11-27") } ] }
+{ "id": 11592799, "id-copy": 11592799, "alias": "Booker", "name": "BookerBurkett", "user-since": datetime("2008-07-19T14:13:28.000"), "user-since-copy": datetime("2008-07-19T14:13:28.000"), "friend-ids": {{ 8693431, 28970363, 8276536, 42506445, 20113337, 40761495 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2010-10-26"), "end-date": date("2010-11-15") } ] }
+{ "id": 11598403, "id-copy": 11598403, "alias": "Jo", "name": "JoCattley", "user-since": datetime("2008-01-04T03:33:03.000"), "user-since-copy": datetime("2008-01-04T03:33:03.000"), "friend-ids": {{ 28948698, 9851844, 31708351, 28418023, 33052184, 24995451, 2840550, 19426008, 3790086 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2006-09-15") } ] }
+{ "id": 11616502, "id-copy": 11616502, "alias": "Bernetta", "name": "BernettaMackendoerfer", "user-since": datetime("2005-04-22T03:41:17.000"), "user-since-copy": datetime("2005-04-22T03:41:17.000"), "friend-ids": {{ 18804036, 29570084, 43932411, 41492349, 46505981, 32524166, 5307968 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2004-08-14"), "end-date": date("2009-08-03") } ] }
+{ "id": 11616628, "id-copy": 11616628, "alias": "Jessamine", "name": "JessamineWolff", "user-since": datetime("2008-05-03T17:05:35.000"), "user-since-copy": datetime("2008-05-03T17:05:35.000"), "friend-ids": {{ 38285911, 42183685, 11422759, 25927239, 22771435, 47814309, 43146385, 39761181, 1670925, 15764683, 8068597, 3561105 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2007-07-26"), "end-date": date("2010-03-16") } ] }
+{ "id": 11625859, "id-copy": 11625859, "alias": "Zacharias", "name": "ZachariasSanner", "user-since": datetime("2007-06-12T21:21:21.000"), "user-since-copy": datetime("2007-06-12T21:21:21.000"), "friend-ids": {{ 13379571, 45822651, 39352555, 11549959, 24329960, 2142134, 15486962, 43011509, 46074449, 9322703 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2007-05-15") } ] }
+{ "id": 11626156, "id-copy": 11626156, "alias": "Laurine", "name": "LaurineBastion", "user-since": datetime("2012-05-14T21:34:43.000"), "user-since-copy": datetime("2012-05-14T21:34:43.000"), "friend-ids": {{ 13978691, 24432513, 41105156, 4981880 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-03-09") } ] }
+{ "id": 11633284, "id-copy": 11633284, "alias": "Quinn", "name": "QuinnMillhouse", "user-since": datetime("2006-08-06T07:42:49.000"), "user-since-copy": datetime("2006-08-06T07:42:49.000"), "friend-ids": {{ 15791690, 46827169, 41678324, 25101779, 24496106, 29442447, 29240215, 23819212, 11076551, 27248100, 1506119, 37415860 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2008-01-06") } ] }
+{ "id": 11633326, "id-copy": 11633326, "alias": "Jodi", "name": "JodiBrindle", "user-since": datetime("2009-01-02T19:57:58.000"), "user-since-copy": datetime("2009-01-02T19:57:58.000"), "friend-ids": {{ 5287281, 24414393, 31942570, 45025515, 35679462, 45244705, 4931287, 11590610, 39846242, 14999029, 38735562, 6275771, 33435194 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2009-11-25") } ] }
+{ "id": 11659237, "id-copy": 11659237, "alias": "Orlando", "name": "OrlandoMcloskey", "user-since": datetime("2006-09-15T00:02:58.000"), "user-since-copy": datetime("2006-09-15T00:02:58.000"), "friend-ids": {{ 18927260, 17411696, 20569511, 5242025, 18974872, 24923117, 42416784, 37339853, 42886763, 12241986, 40609114, 8814896, 30383771, 23631329, 41937811, 13354366, 40113344, 11968348, 23416173, 1546554, 46467044, 5542363, 32084191, 3049632 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2006-04-20") } ] }
+{ "id": 11681410, "id-copy": 11681410, "alias": "Wendell", "name": "WendellGarneys", "user-since": datetime("2007-07-23T13:10:29.000"), "user-since-copy": datetime("2007-07-23T13:10:29.000"), "friend-ids": {{ 11124106, 3438927, 28547601, 18074764, 35037765, 25438231, 8196141, 26000844, 6063826, 22981069, 31549929, 33158093, 40748728, 12245244, 2442169, 7879517, 877005, 24286984 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2008-02-10"), "end-date": date("2008-05-15") } ] }
+{ "id": 11721010, "id-copy": 11721010, "alias": "Eliot", "name": "EliotTennant", "user-since": datetime("2009-07-25T22:16:20.000"), "user-since-copy": datetime("2009-07-25T22:16:20.000"), "friend-ids": {{ 41972338, 13293762, 47012929, 13695904, 25235210, 39246961, 36832468, 26854695, 3046764, 17117110, 10902219, 36959080, 32665222 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-11-26") } ] }
+{ "id": 11723506, "id-copy": 11723506, "alias": "Odelia", "name": "OdeliaPaul", "user-since": datetime("2006-03-14T15:49:03.000"), "user-since-copy": datetime("2006-03-14T15:49:03.000"), "friend-ids": {{ 874326, 37021972, 27293893, 40453006, 44728117, 338941, 22832206, 11391929, 46420525 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2012-05-05") } ] }
+{ "id": 11729626, "id-copy": 11729626, "alias": "Kassandra", "name": "KassandraBaker", "user-since": datetime("2010-12-26T12:18:49.000"), "user-since-copy": datetime("2010-12-26T12:18:49.000"), "friend-ids": {{ 2336026, 15350108, 46098823, 35193308, 34644345, 45989141, 31179029, 15991657, 12863616, 18297246, 26571280, 16935684, 31339122, 10623785, 24666322, 23094237, 28117245, 40096052, 37538843, 8085609, 2437482, 8885815, 42016898, 4654048 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2007-07-10") } ] }
+{ "id": 11741821, "id-copy": 11741821, "alias": "Cal", "name": "CalHowe", "user-since": datetime("2005-12-27T20:26:31.000"), "user-since-copy": datetime("2005-12-27T20:26:31.000"), "friend-ids": {{ 45052138 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2006-12-10"), "end-date": date("2006-02-25") } ] }
+{ "id": 11762239, "id-copy": 11762239, "alias": "Guillermo", "name": "GuillermoCallison", "user-since": datetime("2009-02-12T13:46:40.000"), "user-since-copy": datetime("2009-02-12T13:46:40.000"), "friend-ids": {{ 3494924, 650832, 22099424, 11629223, 45581083, 206762, 27794516, 7639789, 31794781, 22985617, 17273963, 9120417, 9496942, 47474589, 47872578, 34639130, 37695869, 41346670, 7789418, 24870369, 31562430, 2414862, 41928569 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2008-07-26") } ] }
+{ "id": 11763463, "id-copy": 11763463, "alias": "Haven", "name": "HavenRaub", "user-since": datetime("2012-03-01T12:41:53.000"), "user-since-copy": datetime("2012-03-01T12:41:53.000"), "friend-ids": {{ 19981286 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2001-04-26") } ] }
+{ "id": 11788834, "id-copy": 11788834, "alias": "Benny", "name": "BennyAgg", "user-since": datetime("2011-12-19T14:28:16.000"), "user-since-copy": datetime("2011-12-19T14:28:16.000"), "friend-ids": {{ 6023130, 41817759, 15338300, 40598251, 38750529, 43646078, 9057658 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2006-09-16") } ] }
+{ "id": 11811196, "id-copy": 11811196, "alias": "Levi", "name": "LeviVeith", "user-since": datetime("2010-04-28T03:02:38.000"), "user-since-copy": datetime("2010-04-28T03:02:38.000"), "friend-ids": {{ 24907725, 35390929, 34837809, 5881290, 28179492, 44686412, 32544180, 20478414, 15685375, 8767940, 7295427 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2004-09-01") } ] }
+{ "id": 11840218, "id-copy": 11840218, "alias": "Deandre", "name": "DeandreMackendrick", "user-since": datetime("2012-07-03T08:22:13.000"), "user-since-copy": datetime("2012-07-03T08:22:13.000"), "friend-ids": {{ 36310775, 13455844, 1133499, 44183463, 28002311, 40758157, 33299342, 47526543, 9613784, 5698202, 1492720, 5663846 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2006-03-12"), "end-date": date("2009-08-08") } ] }
+{ "id": 11874358, "id-copy": 11874358, "alias": "Rachyl", "name": "RachylOmara", "user-since": datetime("2008-05-19T19:05:44.000"), "user-since-copy": datetime("2008-05-19T19:05:44.000"), "friend-ids": {{ 17070163, 39951748, 9940832, 6714785, 4963198, 17121038, 29997771, 21420071, 3672434, 37974288 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2001-11-11"), "end-date": date("2008-07-25") } ] }
+{ "id": 11919640, "id-copy": 11919640, "alias": "Blanch", "name": "BlanchHawkins", "user-since": datetime("2007-09-24T10:11:40.000"), "user-since-copy": datetime("2007-09-24T10:11:40.000"), "friend-ids": {{ 28731986, 7289796, 42121816, 33230171 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2007-09-17") } ] }
+{ "id": 11932807, "id-copy": 11932807, "alias": "Sheridan", "name": "SheridanCarr", "user-since": datetime("2009-05-17T01:39:53.000"), "user-since-copy": datetime("2009-05-17T01:39:53.000"), "friend-ids": {{ 12836351, 10066178, 40881248, 3744364, 18904729, 10238846, 27947251, 23407801, 39613208, 34468026, 20801656, 46114253, 26807188, 13084266, 27104805, 27016320, 25825154, 16782132, 29528918 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-09-04"), "end-date": date("2005-01-15") } ] }
+{ "id": 11954992, "id-copy": 11954992, "alias": "Caitlin", "name": "CaitlinLangston", "user-since": datetime("2007-01-02T01:50:34.000"), "user-since-copy": datetime("2007-01-02T01:50:34.000"), "friend-ids": {{ 23355687, 22474136, 28513847, 32515387, 44041844, 33706721, 10874992, 36341753, 34431157, 16146113, 15462591, 18188151, 29554174, 44940738, 25888018, 42795884, 14382632, 12734889, 11724519, 15830341, 25725320, 37580394, 24124411, 47984339 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2010-05-26"), "end-date": date("2010-03-28") } ] }
+{ "id": 11981266, "id-copy": 11981266, "alias": "Meghann", "name": "MeghannBatten", "user-since": datetime("2008-06-04T14:25:11.000"), "user-since-copy": datetime("2008-06-04T14:25:11.000"), "friend-ids": {{ 39206334, 28999157, 22813777 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2012-06-26") } ] }
+{ "id": 11988241, "id-copy": 11988241, "alias": "Cyrilla", "name": "CyrillaRohtin", "user-since": datetime("2005-02-10T08:24:14.000"), "user-since-copy": datetime("2005-02-10T08:24:14.000"), "friend-ids": {{ 32725541, 26677413, 29278988, 218049, 19833496, 20655804, 27991386, 5326490, 28583388, 41013948, 35541276, 41552165, 8526660 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2004-07-20"), "end-date": date("2004-08-19") } ] }
+{ "id": 9020338, "id-copy": 9020338, "alias": "Shenika", "name": "ShenikaColdsmith", "user-since": datetime("2011-02-22T08:03:05.000"), "user-since-copy": datetime("2011-02-22T08:03:05.000"), "friend-ids": {{ 28029790, 45719398, 12088661, 4134025, 27354070, 46504723, 23155578, 3370020, 26477155, 27314367, 7672726, 41117417 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2011-04-18") } ] }
+{ "id": 9035089, "id-copy": 9035089, "alias": "Marylyn", "name": "MarylynSteele", "user-since": datetime("2005-04-24T04:55:25.000"), "user-since-copy": datetime("2005-04-24T04:55:25.000"), "friend-ids": {{ 4250473, 16568038, 10872744, 32513859, 37267973, 2225211, 45148996, 1080441, 13013464, 10394988, 3316854, 8183563, 228753, 6849521, 8786964, 21073526 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2011-02-11"), "end-date": date("2011-10-08") } ] }
+{ "id": 9041443, "id-copy": 9041443, "alias": "Maria", "name": "MariaWard", "user-since": datetime("2006-12-25T01:24:40.000"), "user-since-copy": datetime("2006-12-25T01:24:40.000"), "friend-ids": {{ 10660010, 19103672, 11300656, 44383404, 36523093, 11434370, 34405687, 30889551, 4843181, 22025114, 26395363, 8607483, 25294309 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2005-06-13") } ] }
+{ "id": 9041689, "id-copy": 9041689, "alias": "Freeman", "name": "FreemanDriggers", "user-since": datetime("2011-05-23T03:51:13.000"), "user-since-copy": datetime("2011-05-23T03:51:13.000"), "friend-ids": {{ 29448942, 29196543, 22725448, 15145190, 11938396, 44028947, 18379392, 21813464, 7448397, 43717728, 10728731, 24177517, 29069798, 37056934, 27601399, 26867839, 16593922, 22247111 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2007-01-14") } ] }
+{ "id": 9043201, "id-copy": 9043201, "alias": "Eliseo", "name": "EliseoBagley", "user-since": datetime("2007-05-17T10:44:18.000"), "user-since-copy": datetime("2007-05-17T10:44:18.000"), "friend-ids": {{ 41250222, 28415639, 40825493, 11902499, 39161617, 16612650, 39102228, 46013732, 42664763, 20165539, 40891614, 2887877, 27999503, 5059039, 9617378, 16378780, 21987749 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2003-05-26") } ] }
+{ "id": 9045535, "id-copy": 9045535, "alias": "Ebenezer", "name": "EbenezerPery", "user-since": datetime("2008-06-05T17:48:45.000"), "user-since-copy": datetime("2008-06-05T17:48:45.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2012-04-07"), "end-date": date("2012-06-10") } ] }
+{ "id": 9050164, "id-copy": 9050164, "alias": "Haydee", "name": "HaydeeCook", "user-since": datetime("2005-08-28T12:13:59.000"), "user-since-copy": datetime("2005-08-28T12:13:59.000"), "friend-ids": {{ 26484166, 27686644, 42277018, 5893537, 34617524, 12158738, 41566344, 30653024, 23636324, 24072660, 1784294, 38620941, 40846838, 30303402, 27004887, 35907658, 42893556, 10118575, 47861482 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2007-11-23") } ] }
+{ "id": 9081124, "id-copy": 9081124, "alias": "Aureole", "name": "AureoleChappel", "user-since": datetime("2005-03-24T18:14:35.000"), "user-since-copy": datetime("2005-03-24T18:14:35.000"), "friend-ids": {{ 16199402, 2970920 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2005-07-19"), "end-date": date("2011-04-02") } ] }
+{ "id": 9107137, "id-copy": 9107137, "alias": "Woodrow", "name": "WoodrowMueller", "user-since": datetime("2012-06-15T04:53:52.000"), "user-since-copy": datetime("2012-06-15T04:53:52.000"), "friend-ids": {{ 39459662, 1343459, 16606290, 21443457, 29053037, 28244658, 27954195, 9411908, 2059678, 24579828, 40955404 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2002-02-11") } ] }
+{ "id": 9125827, "id-copy": 9125827, "alias": "Kary", "name": "KaryHildyard", "user-since": datetime("2006-03-17T23:21:33.000"), "user-since-copy": datetime("2006-03-17T23:21:33.000"), "friend-ids": {{ 5570026 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2007-09-27") } ] }
+{ "id": 9136882, "id-copy": 9136882, "alias": "Cassie", "name": "CassieGarratt", "user-since": datetime("2005-08-07T05:09:11.000"), "user-since-copy": datetime("2005-08-07T05:09:11.000"), "friend-ids": {{ 40916371, 42882703, 37748113, 45347468, 37653228, 15540626, 29276950, 31566687, 14600173, 12909057, 39561446, 41035377, 45987458, 43649639, 24488758, 25625568, 15566464, 584815, 35900688, 1079087, 46148561, 46404398 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2010-11-13"), "end-date": date("2010-09-04") } ] }
+{ "id": 9146107, "id-copy": 9146107, "alias": "Femie", "name": "FemieBurns", "user-since": datetime("2007-05-05T03:23:12.000"), "user-since-copy": datetime("2007-05-05T03:23:12.000"), "friend-ids": {{ 38688633, 2489245, 43502175, 34373436, 11854240, 23544813, 44263720, 20953878, 37021620, 16028559, 20673451, 46975172, 47409532, 44524395 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-07-26") } ] }
+{ "id": 9155080, "id-copy": 9155080, "alias": "Errol", "name": "ErrolLittle", "user-since": datetime("2011-12-20T07:09:25.000"), "user-since-copy": datetime("2011-12-20T07:09:25.000"), "friend-ids": {{ 17400275, 40794627, 12632163, 45365986, 7980045, 7368579, 40357205, 29279590, 258707, 38447445, 27048261, 19911849, 10768265, 24278809, 11940146, 33555290, 23286799, 40641141, 33877442 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-03-05") } ] }
+{ "id": 9168649, "id-copy": 9168649, "alias": "Harmony", "name": "HarmonyMackendoerfer", "user-since": datetime("2006-06-25T21:01:50.000"), "user-since-copy": datetime("2006-06-25T21:01:50.000"), "friend-ids": {{ 197057, 11973988, 2042364, 21282964, 25761405, 10180346, 39780287, 39243722, 2984620, 7756400, 21311572, 21013939, 16998045, 39135533, 47720897, 20316953 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2005-12-17"), "end-date": date("2009-07-11") } ] }
+{ "id": 9170767, "id-copy": 9170767, "alias": "Noble", "name": "NobleByers", "user-since": datetime("2012-04-19T03:21:33.000"), "user-since-copy": datetime("2012-04-19T03:21:33.000"), "friend-ids": {{ 17464807, 11911237, 31984348, 14323306, 21828766, 24212960, 3269277, 24648466, 30032203, 15837021, 12033801, 3899014, 6105665, 4416812, 33902540, 9640452, 3739829, 14414940, 36838129, 7327467, 35420130, 24031049 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2007-08-11") } ] }
+{ "id": 9179413, "id-copy": 9179413, "alias": "Benton", "name": "BentonMorland", "user-since": datetime("2006-02-08T13:43:03.000"), "user-since-copy": datetime("2006-02-08T13:43:03.000"), "friend-ids": {{ 25229017, 22411534, 46862190, 17238544, 10875646, 19572187, 9889710, 23940269, 24489112, 7997331, 8866147, 29705622, 35336434, 14756488, 40059408, 32606759, 37546068, 24168033, 20761302, 45465986, 27519909, 23920570, 3984052, 38799668 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2008-12-05") } ] }
+{ "id": 9205615, "id-copy": 9205615, "alias": "Eddie", "name": "EddieRosensteel", "user-since": datetime("2007-01-03T07:17:37.000"), "user-since-copy": datetime("2007-01-03T07:17:37.000"), "friend-ids": {{ 4208455, 19941893 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2005-11-15") } ] }
+{ "id": 9209866, "id-copy": 9209866, "alias": "Timothy", "name": "TimothyBuck", "user-since": datetime("2009-11-07T14:19:12.000"), "user-since-copy": datetime("2009-11-07T14:19:12.000"), "friend-ids": {{ 43082021, 25019103, 26061770, 7134151, 17663441, 35230064, 731481, 6719229, 23303796, 40777269 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2000-04-03"), "end-date": date("2000-04-20") } ] }
+{ "id": 9221836, "id-copy": 9221836, "alias": "Claud", "name": "ClaudPratt", "user-since": datetime("2008-01-01T04:10:02.000"), "user-since-copy": datetime("2008-01-01T04:10:02.000"), "friend-ids": {{ 35586361, 40548794, 7169299, 24675214, 21079165, 37323851, 16881366, 24433012, 38047831, 34495409, 33711705, 8957126, 38345318 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2002-12-05") } ] }
+{ "id": 9259234, "id-copy": 9259234, "alias": "Abigail", "name": "AbigailNicola", "user-since": datetime("2009-08-11T09:18:47.000"), "user-since-copy": datetime("2009-08-11T09:18:47.000"), "friend-ids": {{ 5465164, 47505082 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2006-02-22"), "end-date": date("2007-10-02") } ] }
+{ "id": 9267007, "id-copy": 9267007, "alias": "Perla", "name": "PerlaCox", "user-since": datetime("2009-04-14T20:56:37.000"), "user-since-copy": datetime("2009-04-14T20:56:37.000"), "friend-ids": {{ 8937408, 4640163, 41404266, 15668694, 21004833, 12635405, 40379208, 18641131, 14014264, 39008348, 36559306, 26261953, 3593955, 13559713, 34525259 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2003-07-02") } ] }
+{ "id": 9267397, "id-copy": 9267397, "alias": "Corbin", "name": "CorbinWhite", "user-since": datetime("2006-01-07T07:43:27.000"), "user-since-copy": datetime("2006-01-07T07:43:27.000"), "friend-ids": {{ 11772390, 16826538, 16103166, 3256508, 40044263, 44187580, 29521314, 46200384, 40192445, 1239869, 14257012, 21632509, 6292478, 38738535, 18136574, 8369661, 45672754 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2000-09-16"), "end-date": date("2003-07-12") } ] }
+{ "id": 9274378, "id-copy": 9274378, "alias": "Callista", "name": "CallistaCatleay", "user-since": datetime("2012-01-11T05:02:51.000"), "user-since-copy": datetime("2012-01-11T05:02:51.000"), "friend-ids": {{ 35709258, 45469345, 7683235, 10959232, 44123341, 35853639, 11693773, 39944820, 47667622, 42781782, 4756825, 23566535 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2002-04-15"), "end-date": date("2003-04-03") } ] }
+{ "id": 9286279, "id-copy": 9286279, "alias": "Barnaby", "name": "BarnabyAckerley", "user-since": datetime("2006-09-15T01:56:34.000"), "user-since-copy": datetime("2006-09-15T01:56:34.000"), "friend-ids": {{ 21236050, 22647474, 18898492, 22530993, 4332450, 38947319, 25882415, 47187086, 5810354, 18396369, 44918707, 9732196, 14821426, 148735 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2011-02-10") } ] }
+{ "id": 9291964, "id-copy": 9291964, "alias": "Ned", "name": "NedPullman", "user-since": datetime("2011-02-02T07:25:43.000"), "user-since-copy": datetime("2011-02-02T07:25:43.000"), "friend-ids": {{ 3168566, 3349059, 43400084, 26187570, 11222713, 9924690, 7250860, 9801843, 18856900, 3558502, 17237369, 20047877, 28454433, 12279948, 19319514, 36151797 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-08-11") } ] }
+{ "id": 9295696, "id-copy": 9295696, "alias": "Margaux", "name": "MargauxPerkins", "user-since": datetime("2012-05-23T04:28:13.000"), "user-since-copy": datetime("2012-05-23T04:28:13.000"), "friend-ids": {{ 23713491, 4271158, 27340057, 7815427, 14232017, 22868851, 2293397, 24147381, 11816307, 16597552, 47120663, 40746124, 9777479, 18134957, 39193317, 19755909, 42252346 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2006-02-17"), "end-date": date("2007-05-06") } ] }
+{ "id": 9313492, "id-copy": 9313492, "alias": "Tera", "name": "TeraWolfe", "user-since": datetime("2010-12-20T12:47:25.000"), "user-since-copy": datetime("2010-12-20T12:47:25.000"), "friend-ids": {{ 45424983, 18345704, 14849759, 31638064, 38670515, 48015953, 36114769 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2001-04-26"), "end-date": date("2004-12-06") } ] }
+{ "id": 9329272, "id-copy": 9329272, "alias": "Nonie", "name": "NonieStafford", "user-since": datetime("2005-10-01T21:12:24.000"), "user-since-copy": datetime("2005-10-01T21:12:24.000"), "friend-ids": {{ 42745071, 14744035, 37742648, 31179205, 28520118, 32828516, 2726599, 1667680 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2004-06-21") } ] }
+{ "id": 9341965, "id-copy": 9341965, "alias": "Stephania", "name": "StephaniaBriner", "user-since": datetime("2007-06-15T18:17:32.000"), "user-since-copy": datetime("2007-06-15T18:17:32.000"), "friend-ids": {{ 9361850, 12128362, 42864061, 6323327, 34867192, 32746507, 17493376, 17276666, 33869929, 20708786 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2004-03-23"), "end-date": date("2009-01-07") } ] }
+{ "id": 9345424, "id-copy": 9345424, "alias": "Jasmin", "name": "JasminGaskins", "user-since": datetime("2012-06-15T19:40:07.000"), "user-since-copy": datetime("2012-06-15T19:40:07.000"), "friend-ids": {{ 20837477, 42339634, 41136248, 24571549, 41060055, 18621328, 2057295, 41313707 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2012-05-27"), "end-date": date("2012-07-28") } ] }
+{ "id": 9361930, "id-copy": 9361930, "alias": "Leonard", "name": "LeonardAshbaugh", "user-since": datetime("2008-06-13T07:49:33.000"), "user-since-copy": datetime("2008-06-13T07:49:33.000"), "friend-ids": {{ 33929562, 22722370, 18562061, 44346144, 38834006, 1660309, 17690686, 8299074, 13219630, 42802095, 2203402, 47180979, 43715995, 24339545, 42132653, 32010945, 18200992, 5115504 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2012-04-15") } ] }
+{ "id": 9366253, "id-copy": 9366253, "alias": "Emma", "name": "EmmaKnisely", "user-since": datetime("2012-07-08T20:39:00.000"), "user-since-copy": datetime("2012-07-08T20:39:00.000"), "friend-ids": {{ 40874500, 35049897, 29559982, 42737582, 11405173, 38919458, 26268603, 38582942, 13758558, 16949073 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2009-11-18") } ] }
+{ "id": 9372871, "id-copy": 9372871, "alias": "Emerson", "name": "EmersonSell", "user-since": datetime("2010-01-25T11:12:56.000"), "user-since-copy": datetime("2010-01-25T11:12:56.000"), "friend-ids": {{ 13800934, 24493814 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2004-02-14"), "end-date": date("2005-11-07") } ] }
+{ "id": 9373819, "id-copy": 9373819, "alias": "Man", "name": "ManHarding", "user-since": datetime("2005-03-19T02:36:47.000"), "user-since-copy": datetime("2005-03-19T02:36:47.000"), "friend-ids": {{ 10687886, 6212430, 40098775, 8554409, 18917793, 9329327, 38361031, 27404932, 29083756, 28482636, 38832020, 7859160, 14175144, 3316105, 16742847, 8143105, 13049385, 22288103, 36693926, 26571195, 6536981, 32281681, 41798492, 36467563 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2009-02-08") } ] }
+{ "id": 9386794, "id-copy": 9386794, "alias": "Issac", "name": "IssacNickolson", "user-since": datetime("2009-12-11T08:40:10.000"), "user-since-copy": datetime("2009-12-11T08:40:10.000"), "friend-ids": {{ 4077760, 26197904, 22088648 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2002-05-17") } ] }
+{ "id": 9395638, "id-copy": 9395638, "alias": "Toby", "name": "TobyThomlinson", "user-since": datetime("2012-02-02T02:11:31.000"), "user-since-copy": datetime("2012-02-02T02:11:31.000"), "friend-ids": {{ 39086825, 14218540, 37526829, 46631432, 24407673, 19484977, 3657630 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2012-02-26") } ] }
+{ "id": 9403096, "id-copy": 9403096, "alias": "Clarita", "name": "ClaritaRitter", "user-since": datetime("2007-11-18T14:11:04.000"), "user-since-copy": datetime("2007-11-18T14:11:04.000"), "friend-ids": {{ 11967380, 17558867 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2011-01-28"), "end-date": date("2011-05-05") } ] }
+{ "id": 9408688, "id-copy": 9408688, "alias": "Goddard", "name": "GoddardWeisgarber", "user-since": datetime("2011-05-21T13:18:54.000"), "user-since-copy": datetime("2011-05-21T13:18:54.000"), "friend-ids": {{ 2820008, 31637633, 35026624, 544628, 2552858 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2007-11-26") } ] }
+{ "id": 9415921, "id-copy": 9415921, "alias": "Shad", "name": "ShadHaynes", "user-since": datetime("2010-01-19T22:19:28.000"), "user-since-copy": datetime("2010-01-19T22:19:28.000"), "friend-ids": {{ 4608515, 39839555, 31370710, 43278478, 731705, 26523982, 15560444, 10605444, 20229128, 41477079, 47960417, 1744587, 35477897, 10362849, 38394199, 24090076, 14390416 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2010-06-23") } ] }
+{ "id": 9420304, "id-copy": 9420304, "alias": "Alwyn", "name": "AlwynAkers", "user-since": datetime("2009-11-08T08:30:46.000"), "user-since-copy": datetime("2009-11-08T08:30:46.000"), "friend-ids": {{ 40384671, 13399303, 2163402 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-06-14"), "end-date": date("2012-07-17") } ] }
+{ "id": 9426244, "id-copy": 9426244, "alias": "Lamar", "name": "LamarMaugham", "user-since": datetime("2005-03-08T17:00:15.000"), "user-since-copy": datetime("2005-03-08T17:00:15.000"), "friend-ids": {{ 36168436, 20740167, 21922111, 32892152, 34608833, 28621520, 40818313, 23842558, 41275216, 36331147, 40737858, 45983619, 14033949, 23132425, 33634408 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2011-12-24") } ] }
+{ "id": 9434542, "id-copy": 9434542, "alias": "Alice", "name": "AliceRopes", "user-since": datetime("2011-09-10T10:32:17.000"), "user-since-copy": datetime("2011-09-10T10:32:17.000"), "friend-ids": {{ 30233815, 23593045, 243865, 46494768, 15852416, 2627657, 12253908, 11415849, 36381160, 25773586, 9952015, 20363967, 45499740, 15573031, 2939342, 24137982, 34026341, 34111551, 30963526, 7116453 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2008-07-09") } ] }
+{ "id": 9440818, "id-copy": 9440818, "alias": "Poppy", "name": "PoppyBoyer", "user-since": datetime("2007-06-09T08:15:05.000"), "user-since-copy": datetime("2007-06-09T08:15:05.000"), "friend-ids": {{ 10721272, 26882431, 45774996, 44725231, 34694934, 28877797, 12922671, 16078039, 43902220, 27311426, 34146150, 39285332, 7343219, 17482231, 15496713, 12439079, 18097780, 30046636, 16951144, 27968612 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2006-11-17") } ] }
+{ "id": 9446506, "id-copy": 9446506, "alias": "Deshawn", "name": "DeshawnBashline", "user-since": datetime("2009-03-11T18:09:19.000"), "user-since-copy": datetime("2009-03-11T18:09:19.000"), "friend-ids": {{ 22236205, 44669386, 5098679, 17631352, 40353783, 17155709 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-11-12"), "end-date": date("2003-04-22") } ] }
+{ "id": 9477919, "id-copy": 9477919, "alias": "Lilly", "name": "LillyLinton", "user-since": datetime("2005-01-09T12:24:01.000"), "user-since-copy": datetime("2005-01-09T12:24:01.000"), "friend-ids": {{ 19117935, 45208482, 36019625, 39146688, 15911832 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2011-03-03"), "end-date": date("2011-10-03") } ] }
+{ "id": 9480964, "id-copy": 9480964, "alias": "Ava", "name": "AvaCross", "user-since": datetime("2005-11-03T14:59:13.000"), "user-since-copy": datetime("2005-11-03T14:59:13.000"), "friend-ids": {{ 9693959, 3138151, 20631444, 8672727, 33701530, 14630539, 38539482, 3066915, 30934733, 38630163, 25673376 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2000-06-05"), "end-date": date("2000-10-06") } ] }
+{ "id": 9505936, "id-copy": 9505936, "alias": "Kerrie", "name": "KerrieGadow", "user-since": datetime("2005-06-26T08:47:14.000"), "user-since-copy": datetime("2005-06-26T08:47:14.000"), "friend-ids": {{ 46457424, 17421010, 11336465, 19785227 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2004-12-08"), "end-date": date("2010-04-11") } ] }
+{ "id": 9512989, "id-copy": 9512989, "alias": "Lilliana", "name": "LillianaAdams", "user-since": datetime("2007-06-01T16:54:29.000"), "user-since-copy": datetime("2007-06-01T16:54:29.000"), "friend-ids": {{ 14085316, 47471900, 24950195, 44416851, 6677091, 34188319, 1783776, 35860593, 29193624, 11999697, 13365419, 39452732, 14401842, 9087264, 15679216, 39424118, 45063958, 11967959, 29634503, 15763396 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2001-02-08"), "end-date": date("2008-03-23") } ] }
+{ "id": 9521401, "id-copy": 9521401, "alias": "Donnette", "name": "DonnetteFaust", "user-since": datetime("2012-03-22T09:38:14.000"), "user-since-copy": datetime("2012-03-22T09:38:14.000"), "friend-ids": {{ 25050925 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2008-08-20"), "end-date": date("2009-07-09") } ] }
+{ "id": 9545461, "id-copy": 9545461, "alias": "Sandra", "name": "SandraFea", "user-since": datetime("2005-12-09T14:40:28.000"), "user-since-copy": datetime("2005-12-09T14:40:28.000"), "friend-ids": {{ 28976045 }}, "employment": [ { "organization-name": "Voltbam", "start-date": date("2012-02-02") } ] }
+{ "id": 9555157, "id-copy": 9555157, "alias": "Alea", "name": "AleaWallick", "user-since": datetime("2009-11-12T19:32:16.000"), "user-since-copy": datetime("2009-11-12T19:32:16.000"), "friend-ids": {{ 9936033, 18972695, 22198051, 44425768, 37636218, 25373418, 17204473, 6543589, 23627204, 40204583, 18664982, 27647616, 43332268, 41812682 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2009-02-17") } ] }
+{ "id": 9563056, "id-copy": 9563056, "alias": "Iantha", "name": "IanthaHoward", "user-since": datetime("2009-03-09T10:16:12.000"), "user-since-copy": datetime("2009-03-09T10:16:12.000"), "friend-ids": {{ 31445918, 39207727, 45365035, 7861010, 28533268, 29009652, 40156013, 40416479, 42741676, 30221879, 30189614, 46450645, 30914117, 33681301, 19457868, 23309378, 15126664, 32913981, 5396205 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2000-03-18"), "end-date": date("2009-01-05") } ] }
+{ "id": 9574393, "id-copy": 9574393, "alias": "Ghislaine", "name": "GhislaineTaylor", "user-since": datetime("2005-01-23T07:49:26.000"), "user-since-copy": datetime("2005-01-23T07:49:26.000"), "friend-ids": {{ 23799181, 25411427, 3758740, 47542325, 41070945, 45261892, 23309481 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2003-04-15") } ] }
+{ "id": 9588427, "id-copy": 9588427, "alias": "Tiffany", "name": "TiffanyGeyer", "user-since": datetime("2007-09-10T11:20:53.000"), "user-since-copy": datetime("2007-09-10T11:20:53.000"), "friend-ids": {{ 31357437, 16305152, 39281885, 25249419, 434661, 13634747, 39812462, 25218908, 22362649, 41696008, 4523776, 40340358, 45330588, 299997, 11538141, 20972409, 25152923, 8627592, 33381524, 6226232 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2005-02-20") } ] }
+{ "id": 9594523, "id-copy": 9594523, "alias": "Tam", "name": "TamWillcox", "user-since": datetime("2011-12-23T11:41:58.000"), "user-since-copy": datetime("2011-12-23T11:41:58.000"), "friend-ids": {{ 27383896, 20745988, 10063024, 8241427, 40299998, 32408463, 25171835, 22380586, 15344194, 25951348, 28733234, 45421004, 2273747, 2229862, 6241144, 6704115, 8659430, 47431991, 47929530, 24393021 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2001-07-27") } ] }
+{ "id": 9629923, "id-copy": 9629923, "alias": "Adria", "name": "AdriaBoyer", "user-since": datetime("2005-08-12T16:31:38.000"), "user-since-copy": datetime("2005-08-12T16:31:38.000"), "friend-ids": {{ 43812176, 1271309, 1412045, 18793840, 40264072, 41525831, 25536841, 46110606, 40440782, 37228709, 37745315, 19025404, 13458371, 32475836, 30506186, 6860193, 44650222, 5924034 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2012-03-08") } ] }
+{ "id": 9664990, "id-copy": 9664990, "alias": "Travis", "name": "TravisJube", "user-since": datetime("2010-02-12T13:42:04.000"), "user-since-copy": datetime("2010-02-12T13:42:04.000"), "friend-ids": {{ 22627931, 5992593, 8208547, 37326819, 14939087, 18366709, 29043862, 45062025, 21360937, 19730114, 26779317, 46856921, 28406774, 40580511, 8062361, 2179206, 47765870, 14039643, 28857662, 42600706 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2007-01-14") } ] }
+{ "id": 9676201, "id-copy": 9676201, "alias": "Jessica", "name": "JessicaBeals", "user-since": datetime("2006-12-02T17:13:07.000"), "user-since-copy": datetime("2006-12-02T17:13:07.000"), "friend-ids": {{ 40180348, 5499689, 43937013, 12294744, 47607871, 15173594, 19403387, 30591667, 1488569, 11862843, 26230465, 15334606, 4397778, 8140277, 39859715, 25854759, 7216524, 41695061, 43036500, 15618315, 4503056, 23790965, 14510949, 34347866 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2011-05-15"), "end-date": date("2011-10-27") } ] }
+{ "id": 9682723, "id-copy": 9682723, "alias": "Rick", "name": "RickEisaman", "user-since": datetime("2011-01-04T04:42:13.000"), "user-since-copy": datetime("2011-01-04T04:42:13.000"), "friend-ids": {{ 843458, 40779817, 24515616, 9016765, 37332064, 2164822, 45832315, 27168757, 43771964, 46638388, 43667809 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2004-08-13"), "end-date": date("2011-04-11") } ] }
+{ "id": 9740476, "id-copy": 9740476, "alias": "Tucker", "name": "TuckerRogers", "user-since": datetime("2005-05-22T22:00:09.000"), "user-since-copy": datetime("2005-05-22T22:00:09.000"), "friend-ids": {{ 13095635, 36113924, 11767777, 15169454, 1692699, 19622409, 17110214 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2009-03-24"), "end-date": date("2011-02-13") } ] }
+{ "id": 9746482, "id-copy": 9746482, "alias": "Ava", "name": "AvaEndsley", "user-since": datetime("2005-07-05T11:34:59.000"), "user-since-copy": datetime("2005-07-05T11:34:59.000"), "friend-ids": {{ 38589612, 37168849, 27697487, 47869699, 7140447, 1195276, 25105593, 46071, 5222989, 39550451, 45838187, 8513498, 44093597, 25194162, 11534580, 37101502, 6417166, 23315276, 9854625 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2011-06-15") } ] }
+{ "id": 9767755, "id-copy": 9767755, "alias": "Joel", "name": "JoelHoopengarner", "user-since": datetime("2012-01-19T13:22:46.000"), "user-since-copy": datetime("2012-01-19T13:22:46.000"), "friend-ids": {{ 41934568, 20874721, 33807743 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2012-01-21"), "end-date": date("2012-06-09") } ] }
+{ "id": 9783310, "id-copy": 9783310, "alias": "Basil", "name": "BasilLangston", "user-since": datetime("2005-06-10T11:35:51.000"), "user-since-copy": datetime("2005-06-10T11:35:51.000"), "friend-ids": {{ 21087606, 17287729, 8132136, 17055542, 5795845, 41180261, 10977404, 29700430, 47047119, 358942, 29290990, 19557422, 35447157, 33135473, 36720866, 39510564 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2000-05-11"), "end-date": date("2000-03-09") } ] }
+{ "id": 9795463, "id-copy": 9795463, "alias": "Brunilda", "name": "BrunildaPheleps", "user-since": datetime("2007-04-21T01:56:02.000"), "user-since-copy": datetime("2007-04-21T01:56:02.000"), "friend-ids": {{ 39507879, 43296507, 45019669, 39481546, 16657717, 8707249, 47148318, 46560087, 42473978, 11974026, 40145543, 2127794, 19537942, 28159963, 21439105, 32578039, 24112998, 47853039, 6406099, 30697429 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2001-07-13") } ] }
+{ "id": 9805759, "id-copy": 9805759, "alias": "Emmie", "name": "EmmieJohns", "user-since": datetime("2008-11-01T15:15:13.000"), "user-since-copy": datetime("2008-11-01T15:15:13.000"), "friend-ids": {{ 47090234, 24484835, 11048702 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2008-02-26") } ] }
+{ "id": 9811513, "id-copy": 9811513, "alias": "Casie", "name": "CasieRose", "user-since": datetime("2011-11-25T11:32:36.000"), "user-since-copy": datetime("2011-11-25T11:32:36.000"), "friend-ids": {{ 8913855, 26924028, 19426899, 38037518, 39689117, 32691982, 6561788, 36463261, 31724455, 18356325, 23130893, 35227626, 13738524, 4700460, 6963740, 13255939, 12215189, 33593825, 34229322 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2003-11-22") } ] }
+{ "id": 9818617, "id-copy": 9818617, "alias": "Elwyn", "name": "ElwynEndsley", "user-since": datetime("2012-04-12T18:14:54.000"), "user-since-copy": datetime("2012-04-12T18:14:54.000"), "friend-ids": {{ 44007613, 15744997, 9366576, 44776374, 19082361, 9967101, 25247773, 20407697 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2003-09-09") } ] }
+{ "id": 9866572, "id-copy": 9866572, "alias": "Evelina", "name": "EvelinaBerry", "user-since": datetime("2006-12-16T03:56:00.000"), "user-since-copy": datetime("2006-12-16T03:56:00.000"), "friend-ids": {{ 13883615, 43198063, 30615747, 3228427, 23840450, 43443245, 17107485, 34691909, 44890462, 47992198, 46475465, 28790498, 7693182, 41338502, 6694688, 17592193, 9966336, 40899188, 16363000, 43996364 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2004-03-01"), "end-date": date("2008-08-21") } ] }
+{ "id": 9867190, "id-copy": 9867190, "alias": "Elvis", "name": "ElvisBasinger", "user-since": datetime("2009-01-16T11:48:43.000"), "user-since-copy": datetime("2009-01-16T11:48:43.000"), "friend-ids": {{ 31562017, 45465097, 29858836, 21720764, 37465930, 20639296, 7168709 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2011-01-11"), "end-date": date("2011-01-26") } ] }
+{ "id": 9879709, "id-copy": 9879709, "alias": "Winfred", "name": "WinfredCraig", "user-since": datetime("2005-08-03T19:34:00.000"), "user-since-copy": datetime("2005-08-03T19:34:00.000"), "friend-ids": {{ 22314477, 25116324, 22136373, 35942614, 21324680, 17967388, 29463891, 36125380, 20673052, 27353154, 25107580, 24689990, 17672337, 16922511, 26158336, 35966438, 26619840, 29808016, 12075922, 33292381, 17902188 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2010-02-04") } ] }
+{ "id": 9880696, "id-copy": 9880696, "alias": "Cynthia", "name": "CynthiaSeidner", "user-since": datetime("2006-03-17T01:36:33.000"), "user-since-copy": datetime("2006-03-17T01:36:33.000"), "friend-ids": {{ 47318799, 28282167 }}, "employment": [ { "organization-name": "Newfase", "start-date": date("2008-07-02"), "end-date": date("2010-11-25") } ] }
+{ "id": 9896473, "id-copy": 9896473, "alias": "Harlan", "name": "HarlanAnderson", "user-since": datetime("2012-06-03T22:40:33.000"), "user-since-copy": datetime("2012-06-03T22:40:33.000"), "friend-ids": {{ 28073049, 32365932, 23795268, 7563960, 47274822, 4907078, 8659018, 33480175, 3984139, 20631025, 26879093, 27168884, 20063035, 22192716, 18259756, 28904415, 28492528, 4140983, 12014021, 10959797, 38881978, 45835171, 6556552, 26372018 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2000-08-18") } ] }
+{ "id": 9950824, "id-copy": 9950824, "alias": "Maryann", "name": "MaryannCressman", "user-since": datetime("2011-02-25T17:51:21.000"), "user-since-copy": datetime("2011-02-25T17:51:21.000"), "friend-ids": {{ 30203965, 23348792, 19093409, 21079475 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2006-10-11"), "end-date": date("2006-10-09") } ] }
+{ "id": 9978190, "id-copy": 9978190, "alias": "Tatianna", "name": "TatiannaSchmidt", "user-since": datetime("2012-07-05T14:37:56.000"), "user-since-copy": datetime("2012-07-05T14:37:56.000"), "friend-ids": {{ 15128198 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2008-11-17") } ] }
+{ "id": 9997456, "id-copy": 9997456, "alias": "Micah", "name": "MicahRogers", "user-since": datetime("2008-03-01T05:53:42.000"), "user-since-copy": datetime("2008-03-01T05:53:42.000"), "friend-ids": {{ 17761154, 33509079, 36866187, 24618619, 7048673, 18747407, 31947241, 33710255, 40699565, 22334622, 24425777, 19450074, 39309621, 4464803, 15881946, 35888289, 10539684, 17175942, 20754578, 27045156, 14301629, 19478576 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2011-02-16") } ] }
+{ "id": 10001047, "id-copy": 10001047, "alias": "Rodger", "name": "RodgerRifler", "user-since": datetime("2009-12-08T18:34:21.000"), "user-since-copy": datetime("2009-12-08T18:34:21.000"), "friend-ids": {{ 41832587, 41015556, 17486735, 38428485, 29774516, 38574837, 2061546, 46972940, 25654449, 776023, 1164809, 34242171, 9752352, 1088591, 26406961, 7270316, 36371574, 24413303, 36287374, 43343719, 6830709, 2919772, 41313339 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2003-02-08") } ] }
+{ "id": 10059343, "id-copy": 10059343, "alias": "Randy", "name": "RandyQueer", "user-since": datetime("2005-06-01T02:30:35.000"), "user-since-copy": datetime("2005-06-01T02:30:35.000"), "friend-ids": {{ 8688755, 7077909, 41009273, 26932559, 29488059, 6408736, 6374592, 5042147, 21880854, 12704496, 28046022, 2384964, 20867794, 3990470, 7132171 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2006-07-07"), "end-date": date("2007-04-08") } ] }
+{ "id": 10065595, "id-copy": 10065595, "alias": "Zenobia", "name": "ZenobiaHiggens", "user-since": datetime("2009-11-06T11:19:47.000"), "user-since-copy": datetime("2009-11-06T11:19:47.000"), "friend-ids": {{ 19623415, 12770212, 30381171, 20436392, 33497094, 39556081, 22592010, 44832685, 35801007, 39682093, 26870566, 8667589, 43790411, 24760722, 8286108, 20709133 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2001-07-28"), "end-date": date("2004-12-26") } ] }
+{ "id": 10071475, "id-copy": 10071475, "alias": "Kyra", "name": "KyraWile", "user-since": datetime("2010-08-21T20:27:23.000"), "user-since-copy": datetime("2010-08-21T20:27:23.000"), "friend-ids": {{ 24326501, 3159228, 33973593, 47221189, 17474184, 17812891 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2010-04-14") } ] }
+{ "id": 10073632, "id-copy": 10073632, "alias": "Hadley", "name": "HadleyPainter", "user-since": datetime("2010-08-18T16:57:45.000"), "user-since-copy": datetime("2010-08-18T16:57:45.000"), "friend-ids": {{ 35310707, 40074121, 28614727, 29388510, 29966750, 45475518, 5989395, 9892960, 7137969, 5530675, 2278234, 9571067, 29644726, 30689189, 41083149 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2004-06-13"), "end-date": date("2004-11-28") } ] }
+{ "id": 10083103, "id-copy": 10083103, "alias": "Albertine", "name": "AlbertineShick", "user-since": datetime("2006-11-10T03:24:02.000"), "user-since-copy": datetime("2006-11-10T03:24:02.000"), "friend-ids": {{ 22979883, 41779991, 30340160, 44852777, 43786950, 33382165, 898482, 16427018, 1264379, 19925419, 10166319, 12658187, 38802346 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2010-03-05") } ] }
+{ "id": 10085446, "id-copy": 10085446, "alias": "Merla", "name": "MerlaWhitehead", "user-since": datetime("2006-12-08T11:13:30.000"), "user-since-copy": datetime("2006-12-08T11:13:30.000"), "friend-ids": {{ 44039547 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-03-16"), "end-date": date("2009-04-16") } ] }
+{ "id": 10086913, "id-copy": 10086913, "alias": "Margaretta", "name": "MargarettaPfeifer", "user-since": datetime("2012-03-04T14:47:18.000"), "user-since-copy": datetime("2012-03-04T14:47:18.000"), "friend-ids": {{ 9800482, 3761286, 34428154, 18082184, 14845214, 33053674, 46786785, 22235473, 23677556, 24819784, 47587008, 36939436, 14987278 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2003-07-08"), "end-date": date("2010-03-01") } ] }
+{ "id": 10090042, "id-copy": 10090042, "alias": "Gaye", "name": "GayeHayhurst", "user-since": datetime("2006-09-23T14:26:31.000"), "user-since-copy": datetime("2006-09-23T14:26:31.000"), "friend-ids": {{ 41099035, 16443590, 9899624, 2459064, 25428448, 1420220, 1487058, 13700561, 11008052, 36459693, 45632468, 30351729, 33053870, 26372759, 10801940, 37166367 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2005-07-15"), "end-date": date("2010-05-04") } ] }
+{ "id": 10116496, "id-copy": 10116496, "alias": "Gena", "name": "GenaJerome", "user-since": datetime("2005-03-04T21:38:41.000"), "user-since-copy": datetime("2005-03-04T21:38:41.000"), "friend-ids": {{ 11698908, 11838778, 10546816, 13504928, 25681727, 20198355, 28316946, 13835662, 16328293, 39540292, 43990464, 31393679, 34806990, 19167324, 8558031, 37794176, 14389975 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2003-10-01"), "end-date": date("2006-06-13") } ] }
+{ "id": 10122346, "id-copy": 10122346, "alias": "Salal", "name": "SalalPearson", "user-since": datetime("2011-11-14T10:42:11.000"), "user-since-copy": datetime("2011-11-14T10:42:11.000"), "friend-ids": {{ 44003884, 37124809, 7600567, 5158911, 31009406, 10708460 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2001-02-17"), "end-date": date("2010-06-23") } ] }
+{ "id": 10123051, "id-copy": 10123051, "alias": "Rowland", "name": "RowlandWaldron", "user-since": datetime("2011-08-01T17:20:14.000"), "user-since-copy": datetime("2011-08-01T17:20:14.000"), "friend-ids": {{ 7693849, 5416143, 10885197, 39771258, 41278769, 16236783, 18739058, 2293485, 32013369, 34882536, 14339467, 3680575, 4461977, 33715303, 26345760, 45729149, 17585375, 39496021 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2001-12-10"), "end-date": date("2006-04-07") } ] }
+{ "id": 10128076, "id-copy": 10128076, "alias": "Parker", "name": "ParkerHutton", "user-since": datetime("2011-06-05T03:46:01.000"), "user-since-copy": datetime("2011-06-05T03:46:01.000"), "friend-ids": {{ 24818185, 42512828, 22798434, 38901116, 12147430, 47942796, 34742031, 7142883, 11882526, 16055416, 3892909, 12824325, 13378363, 34281637, 15457426, 24092146, 27678834, 15804956 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2007-04-12"), "end-date": date("2009-05-09") } ] }
+{ "id": 10131352, "id-copy": 10131352, "alias": "Brett", "name": "BrettBullard", "user-since": datetime("2011-03-20T00:21:15.000"), "user-since-copy": datetime("2011-03-20T00:21:15.000"), "friend-ids": {{ 42102691, 34313392, 19476509, 40509353, 40764048, 32856149, 20306336, 18276288, 34284082, 32265145, 23912229, 7426729, 26377621, 43687843, 6140857, 4573908, 6840657, 18335864, 19868141, 6051525 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2005-11-09"), "end-date": date("2008-12-05") } ] }
+{ "id": 10132771, "id-copy": 10132771, "alias": "Gaenor", "name": "GaenorEvans", "user-since": datetime("2006-01-23T20:07:34.000"), "user-since-copy": datetime("2006-01-23T20:07:34.000"), "friend-ids": {{ 20344517, 47988409, 39449785, 16775663, 20200468 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-03-17") } ] }
+{ "id": 10138039, "id-copy": 10138039, "alias": "Farah", "name": "FarahAnn", "user-since": datetime("2008-05-10T19:04:28.000"), "user-since-copy": datetime("2008-05-10T19:04:28.000"), "friend-ids": {{ 32501277, 13715476, 10452566, 2652600, 16449577, 12508457, 30925424, 21595197, 26030962, 31683678 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2003-10-02") } ] }
+{ "id": 10150873, "id-copy": 10150873, "alias": "Shanice", "name": "ShaniceReiss", "user-since": datetime("2005-07-07T09:46:00.000"), "user-since-copy": datetime("2005-07-07T09:46:00.000"), "friend-ids": {{ 29208488, 6994033, 13074568, 31547206, 2547580, 15915539, 37448883, 38739687, 33246865, 28231547, 33861348, 44929557, 13977747, 44297013, 22367804 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2002-09-07"), "end-date": date("2006-04-23") } ] }
+{ "id": 10186180, "id-copy": 10186180, "alias": "Mina", "name": "MinaGist", "user-since": datetime("2012-07-05T21:56:14.000"), "user-since-copy": datetime("2012-07-05T21:56:14.000"), "friend-ids": {{ 12424234, 41863508, 44607839, 36984124, 3839840, 38458170, 41721653, 4785194, 20595881, 13515001 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2012-07-19") } ] }
+{ "id": 10188805, "id-copy": 10188805, "alias": "Margarita", "name": "MargaritaBrinigh", "user-since": datetime("2011-06-26T06:22:38.000"), "user-since-copy": datetime("2011-06-26T06:22:38.000"), "friend-ids": {{ 39275311, 42262790, 35041935, 12137373, 8507536 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2004-03-17") } ] }
+{ "id": 10193368, "id-copy": 10193368, "alias": "Oneida", "name": "OneidaEve", "user-since": datetime("2005-01-16T07:26:07.000"), "user-since-copy": datetime("2005-01-16T07:26:07.000"), "friend-ids": {{ 46396755, 39763353, 13661339, 5992749, 293256, 15572483, 16775625, 21543680 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2003-01-12"), "end-date": date("2008-03-22") } ] }
+{ "id": 10211827, "id-copy": 10211827, "alias": "Fanny", "name": "FannyHarrold", "user-since": datetime("2010-08-28T09:57:52.000"), "user-since-copy": datetime("2010-08-28T09:57:52.000"), "friend-ids": {{ 4061493, 30492642, 8550070, 34805906, 5798646, 39169853, 45190690, 34218456, 3758565, 18038216 }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2008-10-14"), "end-date": date("2008-05-18") } ] }
+{ "id": 10215280, "id-copy": 10215280, "alias": "Barbara", "name": "BarbaraEve", "user-since": datetime("2012-03-09T01:36:52.000"), "user-since-copy": datetime("2012-03-09T01:36:52.000"), "friend-ids": {{ 32562793, 33679771, 10306498, 37847497, 30180151, 3504698 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2011-12-14") } ] }
+{ "id": 10238749, "id-copy": 10238749, "alias": "Elspeth", "name": "ElspethFilby", "user-since": datetime("2010-02-08T22:55:13.000"), "user-since-copy": datetime("2010-02-08T22:55:13.000"), "friend-ids": {{ 307224, 16533888 }}, "employment": [ { "organization-name": "goldendexon", "start-date": date("2011-12-10") } ] }
+{ "id": 10269739, "id-copy": 10269739, "alias": "Shantel", "name": "ShantelEve", "user-since": datetime("2012-06-06T00:37:05.000"), "user-since-copy": datetime("2012-06-06T00:37:05.000"), "friend-ids": {{ 39436396, 20382971, 47821933, 28867521, 23217564, 40672635, 34693766, 4383592, 42534606, 23535312, 9112260, 4828073, 37429286, 27965200, 30257544, 47609429, 18527025, 33339218, 898986, 2817270, 6040099, 47802547 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2000-05-24") } ] }
+{ "id": 10283503, "id-copy": 10283503, "alias": "Terrilyn", "name": "TerrilynZadovsky", "user-since": datetime("2007-06-17T05:40:01.000"), "user-since-copy": datetime("2007-06-17T05:40:01.000"), "friend-ids": {{ 30185148, 22395650, 3212998, 41571861, 21336440, 41050091 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2009-12-14") } ] }
+{ "id": 10283941, "id-copy": 10283941, "alias": "Jeffie", "name": "JeffieChappel", "user-since": datetime("2012-06-17T10:07:53.000"), "user-since-copy": datetime("2012-06-17T10:07:53.000"), "friend-ids": {{ 37665650, 44995551, 8518132, 25975224, 22980129, 41720034, 42152946, 26671472, 25698917, 24270208, 36866555, 6728174, 46967331, 31563323, 1382901, 6764335, 35373496 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2005-01-26") } ] }
+{ "id": 10295389, "id-copy": 10295389, "alias": "Major", "name": "MajorDrabble", "user-since": datetime("2009-05-23T12:56:48.000"), "user-since-copy": datetime("2009-05-23T12:56:48.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2000-10-26") } ] }
+{ "id": 10307032, "id-copy": 10307032, "alias": "Quentin", "name": "QuentinSauter", "user-since": datetime("2012-07-11T07:16:43.000"), "user-since-copy": datetime("2012-07-11T07:16:43.000"), "friend-ids": {{ 1926278, 42211794, 1508832, 14973540, 6721046, 28872485, 5047722, 7805271, 31508326, 20891455, 38735410, 13190567, 18209753, 44468536, 34640135, 47290587, 25576626 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2012-02-13") } ] }
+{ "id": 10322023, "id-copy": 10322023, "alias": "Shanita", "name": "ShanitaBeedell", "user-since": datetime("2011-06-09T23:50:09.000"), "user-since-copy": datetime("2011-06-09T23:50:09.000"), "friend-ids": {{ 22628842, 2169935, 20656034, 9086684, 17234788, 11936164, 12465122, 2543006, 40067557, 36767662, 633930, 41805132, 13246529, 43801547, 44953975, 36902947, 34935791, 22923033, 28190533, 18230134, 9484458, 21184932 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2011-10-10") } ] }
+{ "id": 10322398, "id-copy": 10322398, "alias": "Alanna", "name": "AlannaBollinger", "user-since": datetime("2008-09-01T20:05:18.000"), "user-since-copy": datetime("2008-09-01T20:05:18.000"), "friend-ids": {{ 4294902, 42664964 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2009-10-08"), "end-date": date("2011-09-26") } ] }
+{ "id": 10346116, "id-copy": 10346116, "alias": "Breana", "name": "BreanaPainter", "user-since": datetime("2012-04-05T12:15:17.000"), "user-since-copy": datetime("2012-04-05T12:15:17.000"), "friend-ids": {{ 39999376, 5382299, 36254541, 16829210, 7084172, 13545656, 24681698, 34171417, 28514693, 8090159, 35046661, 44544921, 47754565, 28732689, 19680056, 21398367, 39260450 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2012-08-24"), "end-date": date("2012-08-24") } ] }
+{ "id": 10346338, "id-copy": 10346338, "alias": "Caelie", "name": "CaelieYates", "user-since": datetime("2011-11-10T19:17:38.000"), "user-since-copy": datetime("2011-11-10T19:17:38.000"), "friend-ids": {{ 3910270, 7940512, 32351319, 27966615, 33829964, 34529061, 19420019, 7423616, 22246488, 7284253, 8419860, 43330144 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2005-02-07"), "end-date": date("2011-09-05") } ] }
+{ "id": 10350421, "id-copy": 10350421, "alias": "Diane", "name": "DianeFisher", "user-since": datetime("2010-10-19T11:08:52.000"), "user-since-copy": datetime("2010-10-19T11:08:52.000"), "friend-ids": {{ 22455675, 20415125, 21917591, 44414352, 39158851, 3446534, 6627839, 28358200, 1176552, 37914774 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2008-06-04"), "end-date": date("2009-09-11") } ] }
+{ "id": 10353946, "id-copy": 10353946, "alias": "Cass", "name": "CassPirl", "user-since": datetime("2010-10-25T21:08:28.000"), "user-since-copy": datetime("2010-10-25T21:08:28.000"), "friend-ids": {{ 43117144, 29185875, 28524977, 4904289, 37353728, 30484159, 40114905, 18108320, 46098949, 30207639 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2006-06-10") } ] }
+{ "id": 10361965, "id-copy": 10361965, "alias": "Arlen", "name": "ArlenFlick", "user-since": datetime("2011-07-14T18:38:37.000"), "user-since-copy": datetime("2011-07-14T18:38:37.000"), "friend-ids": {{ 34249140, 2887282, 47622716, 3897801, 33692288, 14374380, 14183995, 41311739, 6378075, 17721901, 20807501, 8908974, 41080464, 26497672 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2008-05-18"), "end-date": date("2011-09-18") } ] }
+{ "id": 10364356, "id-copy": 10364356, "alias": "Katharine", "name": "KatharineHoward", "user-since": datetime("2012-03-04T04:40:32.000"), "user-since-copy": datetime("2012-03-04T04:40:32.000"), "friend-ids": {{ 38784, 9497194, 38432548, 30160971, 16843331, 36942612, 32507064, 41108421, 31761239, 20202472, 37170299, 39217222, 14201294, 46319310 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2011-11-09"), "end-date": date("2011-07-18") } ] }
+{ "id": 10367416, "id-copy": 10367416, "alias": "Damion", "name": "DamionDean", "user-since": datetime("2008-01-06T05:55:09.000"), "user-since-copy": datetime("2008-01-06T05:55:09.000"), "friend-ids": {{ 45804001, 13077962, 28346489, 25877214, 10164033, 42903493, 66753, 27961850, 41137249, 20490506 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2007-12-18") } ] }
+{ "id": 10391044, "id-copy": 10391044, "alias": "Kendrick", "name": "KendrickNabholz", "user-since": datetime("2007-10-11T19:49:13.000"), "user-since-copy": datetime("2007-10-11T19:49:13.000"), "friend-ids": {{ 39264696, 35794708, 222108, 29542536, 34470710, 16736694, 36282306, 12411530, 12507843, 30193842, 45764599, 32250152, 16472135, 26507230, 17443301, 16787960, 17651924, 37659951, 28610616, 12928071 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2007-05-07") } ] }
+{ "id": 10394488, "id-copy": 10394488, "alias": "Oswald", "name": "OswaldRay", "user-since": datetime("2006-02-12T17:39:23.000"), "user-since-copy": datetime("2006-02-12T17:39:23.000"), "friend-ids": {{ 14370372, 14174983, 7749259, 39375970, 1755409, 9056913 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2011-12-04"), "end-date": date("2011-06-08") } ] }
+{ "id": 10396831, "id-copy": 10396831, "alias": "Carman", "name": "CarmanElder", "user-since": datetime("2011-12-27T21:50:41.000"), "user-since-copy": datetime("2011-12-27T21:50:41.000"), "friend-ids": {{ 41782166, 39862540, 39100006, 45023958, 29253172, 31208143, 12637805, 5844876, 37296616, 20896053, 18358082, 11068853, 5350064, 14456765, 15758928 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2012-05-18"), "end-date": date("2012-07-26") } ] }
+{ "id": 10404706, "id-copy": 10404706, "alias": "Rylan", "name": "RylanEmrick", "user-since": datetime("2008-11-23T00:55:36.000"), "user-since-copy": datetime("2008-11-23T00:55:36.000"), "friend-ids": {{ 17936230, 20908773, 34834317, 26134774, 3534090, 7699389, 11743997, 37809096, 23228338, 19069026, 662582, 40839640, 26706968, 42711557, 28658968, 39161015, 29201879, 7516443, 21802464, 16456657, 32689464 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2012-02-16") } ] }
+{ "id": 10444585, "id-copy": 10444585, "alias": "Harrietta", "name": "HarriettaDunkle", "user-since": datetime("2012-01-26T16:14:19.000"), "user-since-copy": datetime("2012-01-26T16:14:19.000"), "friend-ids": {{ 9013750, 39577621, 40067238, 24177261, 41169182, 5939218, 13820152, 47741655 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2004-06-13") } ] }
+{ "id": 10453144, "id-copy": 10453144, "alias": "Jason", "name": "JasonSachse", "user-since": datetime("2009-01-25T10:27:17.000"), "user-since-copy": datetime("2009-01-25T10:27:17.000"), "friend-ids": {{ 12949882, 32048809, 23087453, 3994051, 20775019, 22184704, 38106058, 34520240, 13724092, 16309751, 25955640, 4812195, 40546554, 12695295, 16574455, 38615670, 43405164, 7997407, 12239790 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2005-08-01"), "end-date": date("2008-02-08") } ] }
+{ "id": 10469071, "id-copy": 10469071, "alias": "Apryl", "name": "AprylWatson", "user-since": datetime("2006-10-03T08:37:12.000"), "user-since-copy": datetime("2006-10-03T08:37:12.000"), "friend-ids": {{ 4517575, 34635569, 1199146 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2001-02-01"), "end-date": date("2007-09-01") } ] }
+{ "id": 10469980, "id-copy": 10469980, "alias": "Rosalynne", "name": "RosalynneZalack", "user-since": datetime("2012-03-07T10:12:20.000"), "user-since-copy": datetime("2012-03-07T10:12:20.000"), "friend-ids": {{ 46118617, 27264184, 8045697, 30832992, 47861079, 24266748, 10689886, 14799850, 1178687, 39540720, 17568852, 24394222, 10078451, 4748570, 47808632, 35277954, 8802885, 13747535, 22203533, 42065169, 19096770, 14087466, 45753492 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2006-04-15"), "end-date": date("2010-07-14") } ] }
+{ "id": 10486213, "id-copy": 10486213, "alias": "Modesto", "name": "ModestoCox", "user-since": datetime("2006-02-07T05:43:24.000"), "user-since-copy": datetime("2006-02-07T05:43:24.000"), "friend-ids": {{ 42665859, 12929499, 5618502, 24287766, 38722882, 5162913, 2978226, 37521984, 43144325, 3313029, 17680751, 726799 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2007-08-12") } ] }
+{ "id": 10487029, "id-copy": 10487029, "alias": "Fredericka", "name": "FrederickaShea", "user-since": datetime("2011-04-07T06:12:40.000"), "user-since-copy": datetime("2011-04-07T06:12:40.000"), "friend-ids": {{ 45223639, 1019151, 30626857, 10247171, 36952244, 36646177, 2396690, 26604216, 19215860, 20900949, 14160764 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2012-01-08") } ] }
+{ "id": 10495420, "id-copy": 10495420, "alias": "Wendy", "name": "WendyMcloskey", "user-since": datetime("2011-04-26T23:38:24.000"), "user-since-copy": datetime("2011-04-26T23:38:24.000"), "friend-ids": {{ 16762653, 46262691, 12313140, 20481262, 347993, 23105127, 1680519, 20880265, 45611347, 21907223, 46615281, 17188244, 44019800, 46943250, 28647738, 16792673, 29406270, 42714079 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2008-08-27") } ] }
+{ "id": 10498285, "id-copy": 10498285, "alias": "Kiley", "name": "KileyBridger", "user-since": datetime("2006-05-14T21:55:34.000"), "user-since-copy": datetime("2006-05-14T21:55:34.000"), "friend-ids": {{ 38780484, 46190003, 905670, 35609390, 46621151, 5099226, 24328595, 16340411, 13326485, 13872400, 35896828, 9196151, 8525875, 7461206, 28379538, 46461267, 45270205, 35718577, 5310596, 7080391 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2009-11-11"), "end-date": date("2009-06-23") } ] }
+{ "id": 10504084, "id-copy": 10504084, "alias": "Etsuko", "name": "EtsukoDealtry", "user-since": datetime("2012-05-11T00:35:22.000"), "user-since-copy": datetime("2012-05-11T00:35:22.000"), "friend-ids": {{ 27578969, 40308832, 15379566, 8664135, 21276773, 43659426, 28027401, 23264043, 23981731, 19124540, 36281456, 38766688, 37886842, 20522702, 28559857, 9838362, 30409517, 14237008, 41013610, 41586760, 37285778, 29427060, 45678692, 32255048 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-10-12"), "end-date": date("2011-12-04") } ] }
+{ "id": 10513507, "id-copy": 10513507, "alias": "Jasmin", "name": "JasminHatfield", "user-since": datetime("2009-06-25T22:45:16.000"), "user-since-copy": datetime("2009-06-25T22:45:16.000"), "friend-ids": {{ 31323261 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-05-12"), "end-date": date("2003-05-07") } ] }
+{ "id": 10515721, "id-copy": 10515721, "alias": "Mariano", "name": "MarianoTrout", "user-since": datetime("2007-08-27T09:33:28.000"), "user-since-copy": datetime("2007-08-27T09:33:28.000"), "friend-ids": {{ 18516004, 4847094, 31548989, 28302698, 18308169, 15068883, 33358074, 19739053, 34017693 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2004-10-18") } ] }
+{ "id": 10533343, "id-copy": 10533343, "alias": "Gwendolen", "name": "GwendolenHanseu", "user-since": datetime("2007-02-04T19:56:51.000"), "user-since-copy": datetime("2007-02-04T19:56:51.000"), "friend-ids": {{ 25281794, 21814505, 11684475, 5599252, 17261378, 11061422, 27392332, 47872606, 39198697, 17314413, 4034634, 42776559, 43885593, 24835625, 18150148, 4946129, 9288372, 5675162, 34976580 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2010-07-10") } ] }
+{ "id": 10540441, "id-copy": 10540441, "alias": "Albert", "name": "AlbertBasinger", "user-since": datetime("2007-05-12T06:03:38.000"), "user-since-copy": datetime("2007-05-12T06:03:38.000"), "friend-ids": {{ 36392592, 35815177, 22050314, 45279196, 15405747, 33802667, 44081359, 2027267, 47159697, 20007080 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2002-10-04"), "end-date": date("2005-08-17") } ] }
+{ "id": 10552405, "id-copy": 10552405, "alias": "Les", "name": "LesBarth", "user-since": datetime("2008-04-02T11:02:37.000"), "user-since-copy": datetime("2008-04-02T11:02:37.000"), "friend-ids": {{ 33645432, 43039707 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2000-09-18") } ] }
+{ "id": 10563310, "id-copy": 10563310, "alias": "Justina", "name": "JustinaHall", "user-since": datetime("2010-08-24T08:57:45.000"), "user-since-copy": datetime("2010-08-24T08:57:45.000"), "friend-ids": {{ 42796179, 25994871, 35439919, 28722419, 7189994, 41505357, 35095639, 14693797, 36519323, 32598167, 6323551, 14565174, 35997662, 9705559, 3996730 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2011-02-20"), "end-date": date("2011-05-05") } ] }
+{ "id": 10573795, "id-copy": 10573795, "alias": "Neil", "name": "NeilMilne", "user-since": datetime("2005-11-15T02:57:46.000"), "user-since-copy": datetime("2005-11-15T02:57:46.000"), "friend-ids": {{ 33469327, 4261514, 43412669, 17289131, 27535421, 15267017, 14005060 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2001-11-13"), "end-date": date("2001-10-28") } ] }
+{ "id": 10636498, "id-copy": 10636498, "alias": "Grahame", "name": "GrahameLeslie", "user-since": datetime("2006-01-17T16:17:07.000"), "user-since-copy": datetime("2006-01-17T16:17:07.000"), "friend-ids": {{ 3924169, 14543253, 19830425, 34696361, 26630699, 47664771 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2004-03-25") } ] }
+{ "id": 10642153, "id-copy": 10642153, "alias": "Wally", "name": "WallyRiggle", "user-since": datetime("2011-10-10T21:43:33.000"), "user-since-copy": datetime("2011-10-10T21:43:33.000"), "friend-ids": {{ 32910135, 45556839, 6526394, 13177451, 10588491, 40270322, 17438379, 21204776, 46036116, 44249789, 7375979, 43487252, 24858016, 3947997 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-10-10") } ] }
+{ "id": 10650265, "id-copy": 10650265, "alias": "Kristia", "name": "KristiaCowart", "user-since": datetime("2005-09-27T20:13:12.000"), "user-since-copy": datetime("2005-09-27T20:13:12.000"), "friend-ids": {{ 41553475, 45442923, 20846576, 6432869, 40830841 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2006-05-05") } ] }
+{ "id": 10659022, "id-copy": 10659022, "alias": "Cecelia", "name": "CeceliaHandyside", "user-since": datetime("2007-02-22T12:42:30.000"), "user-since-copy": datetime("2007-02-22T12:42:30.000"), "friend-ids": {{ 9051, 38746030, 6178049, 22068473, 25755202, 11577837, 28994476 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2004-07-09"), "end-date": date("2009-10-14") } ] }
+{ "id": 10668283, "id-copy": 10668283, "alias": "Dorian", "name": "DorianTomlinson", "user-since": datetime("2008-06-22T00:01:46.000"), "user-since-copy": datetime("2008-06-22T00:01:46.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Lexitechno", "start-date": date("2004-12-17") } ] }
+{ "id": 10674199, "id-copy": 10674199, "alias": "Dorothy", "name": "DorothyPritchard", "user-since": datetime("2007-09-19T04:32:05.000"), "user-since-copy": datetime("2007-09-19T04:32:05.000"), "friend-ids": {{ 11239155, 14468542, 8244419, 30563447, 2235193, 33015958, 11941749, 22198664, 41531114, 11614864, 43486312, 11394784, 46038310, 8248070, 12346192 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2000-10-03") } ] }
+{ "id": 10686646, "id-copy": 10686646, "alias": "Deborah", "name": "DeborahRosenstiehl", "user-since": datetime("2012-06-18T16:51:32.000"), "user-since-copy": datetime("2012-06-18T16:51:32.000"), "friend-ids": {{ 34005621, 6910583, 11226890, 1333457, 13615971, 15332838, 30484423, 38261521, 39526604, 12093262, 15397660, 29644860, 36715060, 16753181 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2005-07-01"), "end-date": date("2007-10-22") } ] }
+{ "id": 10703185, "id-copy": 10703185, "alias": "Sabina", "name": "SabinaHall", "user-since": datetime("2012-05-18T20:37:33.000"), "user-since-copy": datetime("2012-05-18T20:37:33.000"), "friend-ids": {{ 432154, 6472603, 35649237, 46598578, 35486135, 44354453 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2002-11-04"), "end-date": date("2011-10-12") } ] }
+{ "id": 10710526, "id-copy": 10710526, "alias": "Heike", "name": "HeikeReed", "user-since": datetime("2009-08-15T19:20:30.000"), "user-since-copy": datetime("2009-08-15T19:20:30.000"), "friend-ids": {{ 36253853, 35694929, 43324582, 24829816 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2011-02-12"), "end-date": date("2011-01-22") } ] }
+{ "id": 10734148, "id-copy": 10734148, "alias": "Allannah", "name": "AllannahHoffhants", "user-since": datetime("2005-11-18T00:54:25.000"), "user-since-copy": datetime("2005-11-18T00:54:25.000"), "friend-ids": {{ 26897353, 13343289, 1991130, 39024681, 21839148, 38693973, 19132058, 17589948, 13367008, 30389658, 21757614, 45618415, 23559236, 35669455, 22088928, 2531202, 120534, 867017, 8590987, 25956219, 21819960, 41918122, 31042839, 15019901 }}, "employment": [ { "organization-name": "Ranhotfan", "start-date": date("2004-10-25") } ] }
+{ "id": 10738096, "id-copy": 10738096, "alias": "Dori", "name": "DoriAlcocke", "user-since": datetime("2010-05-21T04:59:08.000"), "user-since-copy": datetime("2010-05-21T04:59:08.000"), "friend-ids": {{ 44039507, 40951102, 39132038, 31982600, 46848423, 43375356, 6188106, 3044041, 38421537, 18640387, 21639042, 11192576, 15659477, 360828, 26875197, 19433881 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2008-02-19"), "end-date": date("2011-03-24") } ] }
+{ "id": 10765090, "id-copy": 10765090, "alias": "Louiza", "name": "LouizaMcelroy", "user-since": datetime("2012-08-14T02:46:00.000"), "user-since-copy": datetime("2012-08-14T02:46:00.000"), "friend-ids": {{ 14365973, 9091111, 44279279, 45125689, 29955385, 23874606, 18142514, 24878700, 13928633, 47391704, 29729670, 35422059, 987030, 3200788, 7640346, 32947024, 32550247, 25746061, 34112521, 41193622, 2620213, 30090329, 5531715 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2002-06-16"), "end-date": date("2003-05-13") } ] }
+{ "id": 10768810, "id-copy": 10768810, "alias": "Gaston", "name": "GastonBender", "user-since": datetime("2008-05-24T17:27:14.000"), "user-since-copy": datetime("2008-05-24T17:27:14.000"), "friend-ids": {{ 29652235, 40180625, 34608178, 43814186, 9682855, 24692412, 33119254, 20480079, 35147289, 24629496, 1449575 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2010-04-06") } ] }
+{ "id": 10771030, "id-copy": 10771030, "alias": "Jen", "name": "JenZaun", "user-since": datetime("2006-12-02T14:42:43.000"), "user-since-copy": datetime("2006-12-02T14:42:43.000"), "friend-ids": {{ 38166077 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2003-12-01"), "end-date": date("2010-04-12") } ] }
+{ "id": 10772929, "id-copy": 10772929, "alias": "Hugh", "name": "HughTrout", "user-since": datetime("2008-01-24T03:16:55.000"), "user-since-copy": datetime("2008-01-24T03:16:55.000"), "friend-ids": {{ 39704817, 19656412, 37084896, 5219803, 23455492, 14248249, 26973609, 4607440, 25844255, 3032226, 45432192, 47011338, 41460367, 28779211, 31780563, 31808543, 29732190, 1264228, 7989711, 38397890, 7638694, 3002993, 8960147, 46258407 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2010-08-02"), "end-date": date("2010-05-08") } ] }
+{ "id": 10779373, "id-copy": 10779373, "alias": "Donya", "name": "DonyaWegley", "user-since": datetime("2012-03-28T01:26:06.000"), "user-since-copy": datetime("2012-03-28T01:26:06.000"), "friend-ids": {{ 24977052, 19856115, 36795249, 7875698, 23317261, 5916235, 17789989, 41932923 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2008-10-18") } ] }
+{ "id": 10786438, "id-copy": 10786438, "alias": "Sherika", "name": "SherikaShick", "user-since": datetime("2005-05-18T21:46:18.000"), "user-since-copy": datetime("2005-05-18T21:46:18.000"), "friend-ids": {{ 11188876, 12936787, 43459190, 40396919, 7166644, 20299758 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2002-06-09") } ] }
+{ "id": 10794448, "id-copy": 10794448, "alias": "Delmar", "name": "DelmarDowning", "user-since": datetime("2012-03-10T23:41:49.000"), "user-since-copy": datetime("2012-03-10T23:41:49.000"), "friend-ids": {{ 34002211, 41487, 45067426, 9754093, 23041928, 41378740, 4013550, 11584362, 46202858, 43273004, 35465505 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2005-09-12") } ] }
+{ "id": 10795960, "id-copy": 10795960, "alias": "Hallam", "name": "HallamBousum", "user-since": datetime("2010-04-23T14:02:10.000"), "user-since-copy": datetime("2010-04-23T14:02:10.000"), "friend-ids": {{ 23447883, 39605256, 41998325 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2007-04-18") } ] }
+{ "id": 10797166, "id-copy": 10797166, "alias": "Alethea", "name": "AletheaMills", "user-since": datetime("2011-01-10T03:06:16.000"), "user-since-copy": datetime("2011-01-10T03:06:16.000"), "friend-ids": {{ 25077851, 2396037, 25762626, 31358162, 41492027, 31211140, 38478662, 9688210, 16865534, 4209161, 19863828, 23760993, 36041139, 46184667 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2004-05-04") } ] }
+{ "id": 10803184, "id-copy": 10803184, "alias": "Daria", "name": "DariaPyle", "user-since": datetime("2010-11-22T05:29:27.000"), "user-since-copy": datetime("2010-11-22T05:29:27.000"), "friend-ids": {{ 26747755, 39431389, 24370112, 37832812, 20626868, 30614988, 38041392, 31908762, 47561829, 45121087, 24496373, 32944554, 16470795, 11915899, 29900938, 4003497, 38829225, 36390033, 36474051 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2009-10-03") } ] }
+{ "id": 10808284, "id-copy": 10808284, "alias": "Natalie", "name": "NatalieJewell", "user-since": datetime("2007-04-15T14:17:38.000"), "user-since-copy": datetime("2007-04-15T14:17:38.000"), "friend-ids": {{ 20839191, 18422391, 2571767, 39525211, 38867255, 13491856 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2005-09-10"), "end-date": date("2011-01-20") } ] }
+{ "id": 10809730, "id-copy": 10809730, "alias": "Algar", "name": "AlgarZaun", "user-since": datetime("2008-08-14T06:37:59.000"), "user-since-copy": datetime("2008-08-14T06:37:59.000"), "friend-ids": {{ 12676185, 26087426, 42241358, 47854149, 22179884, 34701736, 35541344, 46257087, 35091522, 10779069 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2010-09-13") } ] }
+{ "id": 10827610, "id-copy": 10827610, "alias": "Madelina", "name": "MadelinaCamp", "user-since": datetime("2010-06-08T13:22:59.000"), "user-since-copy": datetime("2010-06-08T13:22:59.000"), "friend-ids": {{ 35445385, 15924939, 7897517, 15573537, 7234891, 46098859, 877311, 40923818, 45519215, 27332107, 1693386, 21101894, 35225 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2002-10-04") } ] }
+{ "id": 10834579, "id-copy": 10834579, "alias": "Penni", "name": "PenniBlunt", "user-since": datetime("2010-05-20T20:29:16.000"), "user-since-copy": datetime("2010-05-20T20:29:16.000"), "friend-ids": {{ 25926886, 10263270, 4098530, 40765625, 16591278 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2003-03-22") } ] }
+{ "id": 10856059, "id-copy": 10856059, "alias": "Leland", "name": "LelandMcdonald", "user-since": datetime("2006-09-26T03:35:07.000"), "user-since-copy": datetime("2006-09-26T03:35:07.000"), "friend-ids": {{ 29735881, 7080599, 14172811, 24274797, 5773081, 2653240, 18151967, 34988676, 6599030, 46463015, 23254278, 37618443, 32396573 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2004-01-17") } ] }
+{ "id": 10858339, "id-copy": 10858339, "alias": "Eugenio", "name": "EugenioLangston", "user-since": datetime("2006-06-14T22:24:18.000"), "user-since-copy": datetime("2006-06-14T22:24:18.000"), "friend-ids": {{ 18107191, 19162062, 26048227, 16199255, 32644324, 3917262, 38994370, 36221435, 34919041 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2011-02-03") } ] }
+{ "id": 10858909, "id-copy": 10858909, "alias": "Kiley", "name": "KileyCoates", "user-since": datetime("2011-02-03T03:12:41.000"), "user-since-copy": datetime("2011-02-03T03:12:41.000"), "friend-ids": {{ 47990206, 29775839, 33872749, 38952297, 38802567, 38822660, 12420330, 18852873, 30468156, 29085185, 2660660, 28283210, 6711584, 35851765, 31124383, 39930865, 18329720 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2006-09-22") } ] }
+{ "id": 10860286, "id-copy": 10860286, "alias": "Albert", "name": "AlbertMills", "user-since": datetime("2005-01-04T04:39:49.000"), "user-since-copy": datetime("2005-01-04T04:39:49.000"), "friend-ids": {{ 45171802, 36246654, 30029601, 40155304, 4876814, 275363, 46427463, 5698619, 34383185, 47844520, 45026162, 33852471, 36744791, 40565586, 47142152, 42828565 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2012-02-20"), "end-date": date("2012-03-21") } ] }
+{ "id": 10865788, "id-copy": 10865788, "alias": "Ebba", "name": "EbbaSwartzbaugh", "user-since": datetime("2007-08-18T11:38:20.000"), "user-since-copy": datetime("2007-08-18T11:38:20.000"), "friend-ids": {{ 12850265, 19824056, 2754383, 43333892, 9287993, 14972999, 3729396, 20735424 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2001-10-07"), "end-date": date("2004-07-17") } ] }
+{ "id": 10888777, "id-copy": 10888777, "alias": "Bevis", "name": "BevisStall", "user-since": datetime("2007-04-05T02:35:27.000"), "user-since-copy": datetime("2007-04-05T02:35:27.000"), "friend-ids": {{ 1924847, 33036971, 5163765, 37816368, 15975671, 11388174, 38485519, 43186487, 30402693, 34350975, 24348537, 34349089, 22680019, 30625064, 23751465, 9072515, 15915109 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2005-03-11") } ] }
+{ "id": 10899544, "id-copy": 10899544, "alias": "Valentine", "name": "ValentineFisher", "user-since": datetime("2008-07-04T14:36:11.000"), "user-since-copy": datetime("2008-07-04T14:36:11.000"), "friend-ids": {{ 26471524, 781270, 17136010, 12943313, 42125653, 40372131 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2008-09-02"), "end-date": date("2008-01-21") } ] }
+{ "id": 10901332, "id-copy": 10901332, "alias": "Caelie", "name": "CaelieShafer", "user-since": datetime("2011-09-24T05:08:05.000"), "user-since-copy": datetime("2011-09-24T05:08:05.000"), "friend-ids": {{ 40761096, 31796928, 1066172, 21271172, 41179382, 46260705, 9287042, 37605846, 18083603, 23469027, 45497916, 10102434, 724885, 31794816, 44125905, 46373183, 28321712 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2012-07-04") } ] }
+{ "id": 10905802, "id-copy": 10905802, "alias": "Jamika", "name": "JamikaJowers", "user-since": datetime("2007-05-24T01:31:04.000"), "user-since-copy": datetime("2007-05-24T01:31:04.000"), "friend-ids": {{ 16476991, 9041491, 10867973, 18057276, 13716912, 184635, 47717267, 37995364 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-08-20") } ] }
+{ "id": 10930153, "id-copy": 10930153, "alias": "Liliana", "name": "LilianaGoodman", "user-since": datetime("2009-06-22T20:57:17.000"), "user-since-copy": datetime("2009-06-22T20:57:17.000"), "friend-ids": {{ 4302195, 1569986, 5108357, 40772631, 30372008, 36454077, 26878227, 10958250, 46069776, 4779188, 46627230, 47074148, 25489453, 24956443, 31679399, 21835639, 42097220, 35662047, 6354581, 34282348, 13473927 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2008-10-25") } ] }
+{ "id": 10937395, "id-copy": 10937395, "alias": "Madlyn", "name": "MadlynRader", "user-since": datetime("2010-11-11T02:19:12.000"), "user-since-copy": datetime("2010-11-11T02:19:12.000"), "friend-ids": {{ 8750346, 40237703, 11127018, 23810876, 33862918, 8179642 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2011-03-12"), "end-date": date("2011-12-06") } ] }
+{ "id": 10938328, "id-copy": 10938328, "alias": "Tyrese", "name": "TyreseStainforth", "user-since": datetime("2011-03-03T04:21:04.000"), "user-since-copy": datetime("2011-03-03T04:21:04.000"), "friend-ids": {{ 33557445, 27981614, 25595450, 31820772, 42028444, 31389097, 16332592, 3555278, 45113070, 5198333 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2000-06-04") } ] }
+{ "id": 10953628, "id-copy": 10953628, "alias": "Clement", "name": "ClementHoenshell", "user-since": datetime("2009-01-24T03:52:54.000"), "user-since-copy": datetime("2009-01-24T03:52:54.000"), "friend-ids": {{ 24684431, 16961296, 13566818 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2011-05-07") } ] }
+{ "id": 10957867, "id-copy": 10957867, "alias": "Zach", "name": "ZachOppenheimer", "user-since": datetime("2012-01-01T14:40:11.000"), "user-since-copy": datetime("2012-01-01T14:40:11.000"), "friend-ids": {{ 27759480, 2112389, 8560433, 10052851, 37714587, 16717012, 36648956, 44803993, 36030695, 5359496, 32302980, 27143894, 19287706 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-05-14"), "end-date": date("2004-02-23") } ] }
+{ "id": 10967254, "id-copy": 10967254, "alias": "Andre", "name": "AndreCowper", "user-since": datetime("2011-12-21T20:22:47.000"), "user-since-copy": datetime("2011-12-21T20:22:47.000"), "friend-ids": {{ 23645341, 16267661, 7660549, 24716202, 20945538, 10125828, 1712260, 5309070, 16802418, 18273953, 12670834 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2003-12-25"), "end-date": date("2004-04-09") } ] }
+{ "id": 10970950, "id-copy": 10970950, "alias": "Shana", "name": "ShanaRose", "user-since": datetime("2008-09-17T10:03:01.000"), "user-since-copy": datetime("2008-09-17T10:03:01.000"), "friend-ids": {{ 21025589, 17977659, 39920039, 44311386, 2634251 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2006-09-17") } ] }
+{ "id": 10985830, "id-copy": 10985830, "alias": "Spencer", "name": "SpencerWilo", "user-since": datetime("2010-03-02T07:41:59.000"), "user-since-copy": datetime("2010-03-02T07:41:59.000"), "friend-ids": {{ 5766878, 20551454, 27297902, 44757901, 7660518, 28072828, 6387548, 6276027, 40692560, 36168648, 24514885, 40791549, 15536640, 23757967, 19875372 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2004-04-14"), "end-date": date("2009-02-17") } ] }
+{ "id": 11012734, "id-copy": 11012734, "alias": "Jordan", "name": "JordanSadley", "user-since": datetime("2011-02-26T18:40:19.000"), "user-since-copy": datetime("2011-02-26T18:40:19.000"), "friend-ids": {{ 37319587, 37212468, 3023956, 43125609 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2007-07-03"), "end-date": date("2011-01-25") } ] }
+{ "id": 11049274, "id-copy": 11049274, "alias": "Fitz", "name": "FitzBeail", "user-since": datetime("2012-08-10T03:25:57.000"), "user-since-copy": datetime("2012-08-10T03:25:57.000"), "friend-ids": {{ 39403330, 13441324, 723509, 34025727, 23266816, 33898717, 11053310, 14582395, 38435153, 45855468, 45712821 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2004-02-16"), "end-date": date("2007-01-07") } ] }
+{ "id": 11061631, "id-copy": 11061631, "alias": "Maxene", "name": "MaxeneKellogg", "user-since": datetime("2005-11-13T01:09:31.000"), "user-since-copy": datetime("2005-11-13T01:09:31.000"), "friend-ids": {{ 31578394, 39466620, 35741359, 14244925, 3000582, 39031643, 5008430, 18315325, 30440631, 37868108, 12014032, 32314102, 42887702, 1853960, 28022174, 2024670, 38864358, 42073112, 16259942, 34693959, 25315399, 37475597, 33599283 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2008-05-13") } ] }
+{ "id": 11064301, "id-copy": 11064301, "alias": "Dave", "name": "DaveNicholas", "user-since": datetime("2007-01-09T09:19:57.000"), "user-since-copy": datetime("2007-01-09T09:19:57.000"), "friend-ids": {{ 19136340, 40809808, 18774928, 405329, 27436466, 35586548, 16671212, 44582715, 47932437, 22599645, 26281489, 39246487, 39088455, 43696576, 28175190 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2005-02-04") } ] }
+{ "id": 11109553, "id-copy": 11109553, "alias": "Walker", "name": "WalkerDrennan", "user-since": datetime("2007-05-03T02:10:46.000"), "user-since-copy": datetime("2007-05-03T02:10:46.000"), "friend-ids": {{ 38288636, 35385410, 24803705, 31461936, 34309407 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2010-05-20") } ] }
+{ "id": 11129635, "id-copy": 11129635, "alias": "Porter", "name": "PorterRohtin", "user-since": datetime("2005-08-07T05:18:16.000"), "user-since-copy": datetime("2005-08-07T05:18:16.000"), "friend-ids": {{ 15192554, 37509296, 35638203, 5517199, 3781940, 43497242, 28477558, 4325184, 34919156, 18037278, 36486191, 13966437, 16629611, 40623060 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2005-07-13") } ] }
+{ "id": 11140213, "id-copy": 11140213, "alias": "Montgomery", "name": "MontgomeryWhittier", "user-since": datetime("2007-06-19T17:46:13.000"), "user-since-copy": datetime("2007-06-19T17:46:13.000"), "friend-ids": {{ 32831460, 6030454, 30437362, 21866470, 17388602, 40815157, 20000967, 47555494, 5818137, 40634742, 21692148, 2365521, 33290069, 46471164, 9192561, 35768343, 7552168, 3577338, 5346012, 31129868 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2008-02-24") } ] }
+{ "id": 11162920, "id-copy": 11162920, "alias": "Michael", "name": "MichaelJohns", "user-since": datetime("2007-12-21T06:52:31.000"), "user-since-copy": datetime("2007-12-21T06:52:31.000"), "friend-ids": {{ 47587192, 5639113, 24042062, 26141562, 4128346, 25702038, 16421361, 44444678, 30940270, 16928219, 27816662, 37884076, 40854508, 21061894, 42850960, 42453718, 2763269, 16035171, 47650572, 26811622 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2003-02-24") } ] }
+{ "id": 11195221, "id-copy": 11195221, "alias": "Clement", "name": "ClementBriner", "user-since": datetime("2006-12-27T02:29:02.000"), "user-since-copy": datetime("2006-12-27T02:29:02.000"), "friend-ids": {{ 33023290 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2000-06-05") } ] }
+{ "id": 11203174, "id-copy": 11203174, "alias": "Lise", "name": "LiseRockwell", "user-since": datetime("2005-04-21T02:17:33.000"), "user-since-copy": datetime("2005-04-21T02:17:33.000"), "friend-ids": {{ 25322984, 687106, 15193641, 24397137, 34772763, 24725595, 30853266, 14933558, 36895249, 39451299, 2620397, 44594032, 3455415, 39921033, 21621070, 800967 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2006-01-13"), "end-date": date("2008-07-23") } ] }
+{ "id": 11221033, "id-copy": 11221033, "alias": "Vernon", "name": "VernonLear", "user-since": datetime("2006-04-19T13:02:26.000"), "user-since-copy": datetime("2006-04-19T13:02:26.000"), "friend-ids": {{ 45628776, 31762296, 22963223, 10079920, 20931037, 41768759, 25910794, 41882156, 36691498, 1652094, 25804751, 35757270, 40057670, 37961622, 7430384, 1498630, 7636920, 17109852, 12569850, 47366298, 22902730, 5889994, 21003934, 1929823 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2000-04-18") } ] }
+{ "id": 11223157, "id-copy": 11223157, "alias": "Lavina", "name": "LavinaPeters", "user-since": datetime("2007-11-08T11:13:48.000"), "user-since-copy": datetime("2007-11-08T11:13:48.000"), "friend-ids": {{ 45286302 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-01-13") } ] }
+{ "id": 11259028, "id-copy": 11259028, "alias": "Linsay", "name": "LinsayBranson", "user-since": datetime("2011-04-28T08:49:14.000"), "user-since-copy": datetime("2011-04-28T08:49:14.000"), "friend-ids": {{ 24222662, 814967, 16722114, 24161306, 31611, 2964110, 4912379 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2006-05-18"), "end-date": date("2006-12-16") } ] }
+{ "id": 11273239, "id-copy": 11273239, "alias": "Alanis", "name": "AlanisNeely", "user-since": datetime("2009-04-11T16:49:56.000"), "user-since-copy": datetime("2009-04-11T16:49:56.000"), "friend-ids": {{ 16788046, 3222185, 46272663, 16782006, 29597609, 9709951, 37694695, 39662749, 18430270, 38598018, 40033174, 34984089, 8435528, 2669100, 18469173, 25201258, 29975180, 16379939, 24603, 2573554, 16344157, 16880724, 2437581 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2001-10-01"), "end-date": date("2006-08-24") } ] }
+{ "id": 11280553, "id-copy": 11280553, "alias": "Wendy", "name": "WendyClarke", "user-since": datetime("2009-08-28T16:53:37.000"), "user-since-copy": datetime("2009-08-28T16:53:37.000"), "friend-ids": {{ 10802559, 42649709, 8824750, 19241403, 43339000, 23865070, 9842110, 7051904, 39440876, 16961992 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2004-11-15"), "end-date": date("2005-01-15") } ] }
+{ "id": 11287327, "id-copy": 11287327, "alias": "Vito", "name": "VitoMoffat", "user-since": datetime("2008-02-08T03:16:42.000"), "user-since-copy": datetime("2008-02-08T03:16:42.000"), "friend-ids": {{ 36850894, 16346016, 4072987, 36112362, 13277841, 24976604, 20216096, 36253616, 13624540, 39256929, 8411929, 13545093, 27563972, 4306316, 9819682, 21998450, 16647991, 1987261 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2001-07-08"), "end-date": date("2005-04-23") } ] }
+{ "id": 11290987, "id-copy": 11290987, "alias": "Ilana", "name": "IlanaTedrow", "user-since": datetime("2009-03-03T00:10:34.000"), "user-since-copy": datetime("2009-03-03T00:10:34.000"), "friend-ids": {{ 20902982, 27972021, 22354642, 32382609, 18711912, 17070293 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2005-11-28"), "end-date": date("2009-09-17") } ] }
+{ "id": 11293477, "id-copy": 11293477, "alias": "Tamzen", "name": "TamzenWheeler", "user-since": datetime("2006-02-25T23:55:58.000"), "user-since-copy": datetime("2006-02-25T23:55:58.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2002-05-19"), "end-date": date("2011-03-06") } ] }
+{ "id": 11297359, "id-copy": 11297359, "alias": "Perry", "name": "PerryLowe", "user-since": datetime("2005-12-28T02:16:57.000"), "user-since-copy": datetime("2005-12-28T02:16:57.000"), "friend-ids": {{ 33439767 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2007-08-11"), "end-date": date("2009-05-16") } ] }
+{ "id": 11307946, "id-copy": 11307946, "alias": "Helga", "name": "HelgaStough", "user-since": datetime("2007-01-12T21:50:11.000"), "user-since-copy": datetime("2007-01-12T21:50:11.000"), "friend-ids": {{ 22768365 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2007-01-04"), "end-date": date("2009-06-25") } ] }
+{ "id": 11318098, "id-copy": 11318098, "alias": "Lucilla", "name": "LucillaSteele", "user-since": datetime("2006-05-02T12:10:51.000"), "user-since-copy": datetime("2006-05-02T12:10:51.000"), "friend-ids": {{ 43202249, 11116520, 19404968, 23494384, 41664359, 2459832, 21895811, 29849475, 32963400, 24381723, 46790616, 10343240, 43849340, 16769526, 26104853 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2009-10-09") } ] }
+{ "id": 11318329, "id-copy": 11318329, "alias": "April", "name": "AprilSurrency", "user-since": datetime("2008-09-02T21:07:03.000"), "user-since-copy": datetime("2008-09-02T21:07:03.000"), "friend-ids": {{ 8646916, 27873471, 41336682, 42549624, 39851926, 29548550, 31209458, 40169445, 27695329, 20395537, 10311481, 47078664, 32368262, 6850643, 26890752 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2009-12-11") } ] }
+{ "id": 11327731, "id-copy": 11327731, "alias": "Duncan", "name": "DuncanPennington", "user-since": datetime("2007-09-08T05:38:28.000"), "user-since-copy": datetime("2007-09-08T05:38:28.000"), "friend-ids": {{ 7591038, 8046115, 16606742, 39494564, 32760725, 39036737, 9937167, 38968828, 32536611 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2003-12-06") } ] }
+{ "id": 11330215, "id-copy": 11330215, "alias": "Tilly", "name": "TillyMckinnon", "user-since": datetime("2011-04-13T10:13:13.000"), "user-since-copy": datetime("2011-04-13T10:13:13.000"), "friend-ids": {{ 5559510, 31907101, 45791333, 35002065, 1302921, 37193818, 32812039, 41322357, 20631502 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2001-03-22"), "end-date": date("2008-08-22") } ] }
+{ "id": 11333794, "id-copy": 11333794, "alias": "Yung", "name": "YungNash", "user-since": datetime("2010-06-08T17:32:35.000"), "user-since-copy": datetime("2010-06-08T17:32:35.000"), "friend-ids": {{ 11329358, 14452899, 15459758, 31785934, 15405998, 17431717, 36883854, 1230831, 17690420, 45243495, 31580409, 15264731, 10067263, 20381783, 41240146, 2883831, 29492394, 38409147, 35853447, 26151247 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2002-06-24"), "end-date": date("2010-03-23") } ] }
+{ "id": 11348449, "id-copy": 11348449, "alias": "Domitila", "name": "DomitilaPolson", "user-since": datetime("2009-09-24T21:31:17.000"), "user-since-copy": datetime("2009-09-24T21:31:17.000"), "friend-ids": {{ 46755392, 24913792, 47792230, 2451253, 10548653, 3083052, 20700516, 15133622, 17284439, 40871072, 6444103, 44749243, 45289097, 19631062, 8873017, 6262067, 4742977, 672148, 19303779 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2005-06-03") } ] }
+{ "id": 11355979, "id-copy": 11355979, "alias": "Sal", "name": "SalChapman", "user-since": datetime("2012-07-23T17:03:04.000"), "user-since-copy": datetime("2012-07-23T17:03:04.000"), "friend-ids": {{ 4959799, 33919735, 33624568, 9885012, 16788595, 39510500, 34856818, 22167281, 44317359, 45181449, 43901851, 42402339, 9573000, 16655168 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2006-12-10") } ] }
+{ "id": 11364871, "id-copy": 11364871, "alias": "Darrell", "name": "DarrellTaggart", "user-since": datetime("2007-02-14T07:06:21.000"), "user-since-copy": datetime("2007-02-14T07:06:21.000"), "friend-ids": {{ 42942141, 33727432, 32050372, 39330410, 38031970, 18321427, 4533038, 45054607, 34474798, 29859123, 17215101, 24811589, 12250229, 4712867, 23411515, 10287620, 37707941 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2006-11-26"), "end-date": date("2007-02-18") } ] }
+{ "id": 11366056, "id-copy": 11366056, "alias": "Devin", "name": "DevinUlery", "user-since": datetime("2011-05-03T13:27:51.000"), "user-since-copy": datetime("2011-05-03T13:27:51.000"), "friend-ids": {{ 25443767, 42385070, 31515075, 31340661, 25371541, 34378389, 40381786, 23698797, 40141450, 12814851, 41414503, 39733660, 27910438, 44106204, 18806338, 37909692, 12502759, 4270087, 5110443, 14347603, 19313129, 8826229 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2001-12-15") } ] }
+{ "id": 11370337, "id-copy": 11370337, "alias": "Devin", "name": "DevinWatson", "user-since": datetime("2009-07-19T11:47:07.000"), "user-since-copy": datetime("2009-07-19T11:47:07.000"), "friend-ids": {{ 25117468, 31957773, 46217915, 26169035, 34203342, 32134285, 10572760, 10974016, 33771064, 4177645, 4910095, 18301833, 15264956, 5806057, 37899843, 35459189, 4391801, 34940818 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2008-06-19") } ] }
+{ "id": 11390830, "id-copy": 11390830, "alias": "Luciano", "name": "LucianoHooker", "user-since": datetime("2006-08-16T08:17:56.000"), "user-since-copy": datetime("2006-08-16T08:17:56.000"), "friend-ids": {{ 42206490, 5533465, 32480435, 18058343 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2004-02-19") } ] }
+{ "id": 11400016, "id-copy": 11400016, "alias": "Beaumont", "name": "BeaumontMiller", "user-since": datetime("2008-05-12T07:13:22.000"), "user-since-copy": datetime("2008-05-12T07:13:22.000"), "friend-ids": {{ 41935126, 36767417, 10582797, 47501456, 43527117, 2821865, 27905409, 13531461, 16278289, 9565333, 15686197, 15195167, 29350985, 8804024, 31606110, 44124513, 15106563, 26509959, 47480296, 13623445, 17378703, 33568332, 19922072, 12746355 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2002-03-04") } ] }
+{ "id": 11416066, "id-copy": 11416066, "alias": "Janna", "name": "JannaBowchiew", "user-since": datetime("2010-12-06T10:53:56.000"), "user-since-copy": datetime("2010-12-06T10:53:56.000"), "friend-ids": {{ 43816151, 22032304, 27239988, 23813127, 34936097, 8817657, 39872787, 27628236, 38333824, 40879066 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-04-19"), "end-date": date("2008-01-09") } ] }
+{ "id": 11417764, "id-copy": 11417764, "alias": "Maren", "name": "MarenDickson", "user-since": datetime("2006-07-20T06:36:52.000"), "user-since-copy": datetime("2006-07-20T06:36:52.000"), "friend-ids": {{ 14573904, 11946003, 35291176, 25103717, 30010131, 886854, 46625000, 28533752, 46506784, 15300620, 40647607, 10249516, 27751123, 3883546, 41772148, 26655932, 39026036, 4416966, 15070564, 7052224, 10264392, 13650303, 30752174 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2012-08-26"), "end-date": date("2012-08-29") } ] }
+{ "id": 11425216, "id-copy": 11425216, "alias": "Levi", "name": "LeviEiford", "user-since": datetime("2010-04-10T23:37:26.000"), "user-since-copy": datetime("2010-04-10T23:37:26.000"), "friend-ids": {{ 39348801, 15029457, 33995161, 27782571, 16712478, 28987111 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-08-12") } ] }
+{ "id": 11426248, "id-copy": 11426248, "alias": "Chryssa", "name": "ChryssaHincken", "user-since": datetime("2005-06-16T01:11:36.000"), "user-since-copy": datetime("2005-06-16T01:11:36.000"), "friend-ids": {{ 47119545 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2003-11-20"), "end-date": date("2003-10-07") } ] }
+{ "id": 11441509, "id-copy": 11441509, "alias": "Franklyn", "name": "FranklynZimmer", "user-since": datetime("2012-03-22T13:12:29.000"), "user-since-copy": datetime("2012-03-22T13:12:29.000"), "friend-ids": {{ 12883110, 5637339, 42139368, 25533619, 19998291, 4231212, 40792266, 9689761, 7591603, 29088602, 40962884, 9432997, 29850101, 47563888, 10384431, 30557751, 9141240, 45176888, 40987369, 42808497, 37891546, 8520042, 12875368, 39706341 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2008-06-09") } ] }
+{ "id": 11447332, "id-copy": 11447332, "alias": "Sherisse", "name": "SherisseMaugham", "user-since": datetime("2012-02-09T14:21:08.000"), "user-since-copy": datetime("2012-02-09T14:21:08.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2011-09-16") } ] }
+{ "id": 11452525, "id-copy": 11452525, "alias": "Suzanna", "name": "SuzannaOlphert", "user-since": datetime("2005-10-22T04:41:20.000"), "user-since-copy": datetime("2005-10-22T04:41:20.000"), "friend-ids": {{ 44250347, 21517625, 10831891, 23365285, 2000581, 43387385, 40167252, 25288275, 6768341, 36116792, 10670805 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2001-10-21"), "end-date": date("2005-03-11") } ] }
+{ "id": 11456404, "id-copy": 11456404, "alias": "Lonny", "name": "LonnyUllman", "user-since": datetime("2008-10-19T03:05:07.000"), "user-since-copy": datetime("2008-10-19T03:05:07.000"), "friend-ids": {{ 30675414, 44654756, 8273748, 12998719, 20082930 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2009-11-02"), "end-date": date("2011-05-11") } ] }
+{ "id": 11458594, "id-copy": 11458594, "alias": "Rosaline", "name": "RosalineHawker", "user-since": datetime("2006-06-07T01:36:07.000"), "user-since-copy": datetime("2006-06-07T01:36:07.000"), "friend-ids": {{ 13674953, 43755185, 20151836, 40023637, 35564429, 45196295, 33392303, 2080473, 6786170, 42815553, 10811200, 5050190, 20987923, 32613675 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2002-06-05") } ] }
+{ "id": 11476339, "id-copy": 11476339, "alias": "Hopkin", "name": "HopkinNicholas", "user-since": datetime("2008-09-23T20:48:07.000"), "user-since-copy": datetime("2008-09-23T20:48:07.000"), "friend-ids": {{ 30021024, 29046949, 8412580, 10700657, 15739611, 36768609 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2004-01-02") } ] }
+{ "id": 11494930, "id-copy": 11494930, "alias": "Eleanor", "name": "EleanorAnderson", "user-since": datetime("2008-09-01T04:27:31.000"), "user-since-copy": datetime("2008-09-01T04:27:31.000"), "friend-ids": {{ 46834294, 32081711 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2008-01-19") } ] }
+{ "id": 11506045, "id-copy": 11506045, "alias": "Marci", "name": "MarciSaltser", "user-since": datetime("2011-08-05T00:36:14.000"), "user-since-copy": datetime("2011-08-05T00:36:14.000"), "friend-ids": {{ 44810951, 11599851, 4960763, 13454104, 22872317, 44594135, 15792938 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2002-06-22"), "end-date": date("2009-08-20") } ] }
+{ "id": 11507149, "id-copy": 11507149, "alias": "Kendal", "name": "KendalCourtney", "user-since": datetime("2006-06-22T04:28:09.000"), "user-since-copy": datetime("2006-06-22T04:28:09.000"), "friend-ids": {{ 9084267, 26163683, 15271756, 4229254, 5439809, 23992890, 23144677, 26584955, 29430424, 15196312, 19993838, 3665259, 15861241, 15197583, 15693177 }}, "employment": [ { "organization-name": "Streettax", "start-date": date("2010-08-06"), "end-date": date("2011-04-21") } ] }
+{ "id": 11515915, "id-copy": 11515915, "alias": "Hunter", "name": "HunterBash", "user-since": datetime("2011-03-05T16:16:17.000"), "user-since-copy": datetime("2011-03-05T16:16:17.000"), "friend-ids": {{ 14847122, 46314922, 14414318, 46374290, 45050391, 22617753 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2004-01-20") } ] }
+{ "id": 11525575, "id-copy": 11525575, "alias": "Zack", "name": "ZackMills", "user-since": datetime("2007-10-15T20:53:30.000"), "user-since-copy": datetime("2007-10-15T20:53:30.000"), "friend-ids": {{ 11119738, 47490530, 18951399, 24413247, 4019030, 39064308, 43279140, 11316225, 15383674, 40613636, 4793869, 21591307, 23561981, 3763992, 32892218, 34334911, 40693733 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2012-05-25"), "end-date": date("2012-07-09") } ] }
+{ "id": 11529364, "id-copy": 11529364, "alias": "Rufus", "name": "RufusGreen", "user-since": datetime("2009-04-14T15:51:24.000"), "user-since-copy": datetime("2009-04-14T15:51:24.000"), "friend-ids": {{ 5011595 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2000-09-25"), "end-date": date("2004-08-22") } ] }
+{ "id": 11536582, "id-copy": 11536582, "alias": "Deon", "name": "DeonBickerson", "user-since": datetime("2007-05-18T18:12:00.000"), "user-since-copy": datetime("2007-05-18T18:12:00.000"), "friend-ids": {{ 2848304, 6359671, 29695732, 42414044, 3277185, 17642866, 47064497, 32240400, 43486181, 5049864, 22831246, 9259974, 17502793, 29955647, 6928887, 19609966 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2009-01-20"), "end-date": date("2009-03-12") } ] }
+{ "id": 11547586, "id-copy": 11547586, "alias": "Rosanne", "name": "RosanneWatkins", "user-since": datetime("2008-03-02T16:07:45.000"), "user-since-copy": datetime("2008-03-02T16:07:45.000"), "friend-ids": {{ 47389452, 44553302, 30722503, 3892313, 9603884, 12058710, 18459884, 23971280, 39791340, 25400946, 25149383, 8391991, 6548649, 20662585, 34505551, 8352025 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2005-05-17") } ] }
+{ "id": 11551078, "id-copy": 11551078, "alias": "Percy", "name": "PercyStocker", "user-since": datetime("2012-01-12T15:14:02.000"), "user-since-copy": datetime("2012-01-12T15:14:02.000"), "friend-ids": {{ 8927010, 25565873, 1309019, 9736505, 27953053, 6619625, 45562540, 32022492, 1535156, 11343220, 40057278, 5452463, 36005348, 35072612, 31954888 }}, "employment": [ { "organization-name": "Sumlane", "start-date": date("2004-06-01"), "end-date": date("2010-03-09") } ] }
+{ "id": 11562148, "id-copy": 11562148, "alias": "Rexana", "name": "RexanaStange", "user-since": datetime("2012-08-13T20:11:05.000"), "user-since-copy": datetime("2012-08-13T20:11:05.000"), "friend-ids": {{ 22418981, 44892347, 43890424, 38530948, 33178064 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2004-11-21"), "end-date": date("2007-11-01") } ] }
+{ "id": 11582299, "id-copy": 11582299, "alias": "Seward", "name": "SewardReddish", "user-since": datetime("2007-11-07T11:10:00.000"), "user-since-copy": datetime("2007-11-07T11:10:00.000"), "friend-ids": {{ 14793773, 24447668, 30727802, 4757816, 26139324, 4433524, 15974482 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2012-02-10") } ] }
+{ "id": 11596522, "id-copy": 11596522, "alias": "Gena", "name": "GenaTurzanski", "user-since": datetime("2012-06-22T18:42:25.000"), "user-since-copy": datetime("2012-06-22T18:42:25.000"), "friend-ids": {{ 22525625, 22327219, 18520174, 38679685, 16561552, 1999972, 8066310, 24245231, 11682156, 31330371, 38780021, 46833789, 6710024, 38963740, 38984150, 33451484, 19022059, 36880540, 40003274 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2002-09-10") } ] }
+{ "id": 11619817, "id-copy": 11619817, "alias": "Conor", "name": "ConorIsaman", "user-since": datetime("2007-07-19T03:08:58.000"), "user-since-copy": datetime("2007-07-19T03:08:58.000"), "friend-ids": {{ 3118516, 11993690, 44936801, 20826732, 45978958, 5214526, 29651996, 39212065, 47935248, 13306157, 33084407, 537249, 42089040, 7553609, 42024531, 23482433, 45497814, 26865252, 42135224, 41353574, 28567135, 7898064 }}, "employment": [ { "organization-name": "Ronholdings", "start-date": date("2002-04-26") } ] }
+{ "id": 11626564, "id-copy": 11626564, "alias": "Gia", "name": "GiaNehling", "user-since": datetime("2007-05-04T02:40:35.000"), "user-since-copy": datetime("2007-05-04T02:40:35.000"), "friend-ids": {{ 14435544, 22982758, 14548448, 20359010, 43749230, 6484290, 43513351, 3652065, 1851524, 15523948, 1941233, 47031188, 12649571, 42789428, 10950757, 18325469, 24986924, 39948729, 29738829, 268135, 32952373, 29859037 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2007-06-13"), "end-date": date("2008-07-06") } ] }
+{ "id": 11626678, "id-copy": 11626678, "alias": "Reed", "name": "ReedHaile", "user-since": datetime("2011-05-28T09:52:04.000"), "user-since-copy": datetime("2011-05-28T09:52:04.000"), "friend-ids": {{ 38955792, 36648350, 7510300, 36168809, 41493759, 45265187, 1653351, 44881482, 44038304 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2012-03-08"), "end-date": date("2012-05-08") } ] }
+{ "id": 11627800, "id-copy": 11627800, "alias": "Andrina", "name": "AndrinaOrbell", "user-since": datetime("2005-01-07T13:18:15.000"), "user-since-copy": datetime("2005-01-07T13:18:15.000"), "friend-ids": {{ 14378125 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2001-07-27"), "end-date": date("2009-01-26") } ] }
+{ "id": 11630158, "id-copy": 11630158, "alias": "Jewel", "name": "JewelPrechtl", "user-since": datetime("2008-09-24T10:05:42.000"), "user-since-copy": datetime("2008-09-24T10:05:42.000"), "friend-ids": {{ 17110258, 26859370, 7070027, 19698792, 10087924, 31999744, 35694569, 10315290, 15006946, 25258889, 8036893, 20721778, 31250890, 31525573 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2002-10-09") } ] }
+{ "id": 11638618, "id-copy": 11638618, "alias": "Garfield", "name": "GarfieldHardie", "user-since": datetime("2007-07-05T04:44:27.000"), "user-since-copy": datetime("2007-07-05T04:44:27.000"), "friend-ids": {{ 47307628, 3109848, 30936899, 7173119, 33551634, 24239136, 11619168, 633835, 34791947, 12052833, 19798108, 3426648, 395456, 18555868, 18509839, 8340275, 14943912, 42330581, 313099, 25632353, 27912788, 20281899, 8961605, 13625222 }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2001-02-24") } ] }
+{ "id": 11666128, "id-copy": 11666128, "alias": "Mathilda", "name": "MathildaBurris", "user-since": datetime("2006-01-04T14:30:09.000"), "user-since-copy": datetime("2006-01-04T14:30:09.000"), "friend-ids": {{ 21229678, 40152290, 2867638, 27694777, 34054129, 47727334, 39805693, 9084777, 37744206, 47011794, 2190990, 19109454 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2007-09-14"), "end-date": date("2007-03-17") } ] }
+{ "id": 11675221, "id-copy": 11675221, "alias": "Calanthe", "name": "CalantheGearhart", "user-since": datetime("2007-06-08T02:44:20.000"), "user-since-copy": datetime("2007-06-08T02:44:20.000"), "friend-ids": {{ 19185575 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2010-05-21") } ] }
+{ "id": 11693350, "id-copy": 11693350, "alias": "Crystal", "name": "CrystalDickinson", "user-since": datetime("2007-02-08T08:05:12.000"), "user-since-copy": datetime("2007-02-08T08:05:12.000"), "friend-ids": {{ 32246301, 35277320, 38987334, 3391139, 30437594, 35314588, 32659406, 19055708, 5245289, 1155014, 9266846, 20085529, 27878886, 25128707, 46223557, 16459237, 41315912, 26681594 }}, "employment": [ { "organization-name": "Strongtone", "start-date": date("2011-07-03"), "end-date": date("2011-08-05") } ] }
+{ "id": 11694928, "id-copy": 11694928, "alias": "Anne", "name": "AnnePritchard", "user-since": datetime("2005-05-25T23:02:45.000"), "user-since-copy": datetime("2005-05-25T23:02:45.000"), "friend-ids": {{ 4000537, 32410978, 2682612, 1214946, 38250943, 36272447, 14182545, 27782322, 2714608, 38315875 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2011-02-22"), "end-date": date("2011-11-07") } ] }
+{ "id": 11695309, "id-copy": 11695309, "alias": "Petula", "name": "PetulaTanner", "user-since": datetime("2011-12-23T13:29:44.000"), "user-since-copy": datetime("2011-12-23T13:29:44.000"), "friend-ids": {{ 39411346, 33118908, 44553603 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2003-02-26"), "end-date": date("2007-11-12") } ] }
+{ "id": 11748019, "id-copy": 11748019, "alias": "Malinda", "name": "MalindaMoberly", "user-since": datetime("2005-06-21T22:34:38.000"), "user-since-copy": datetime("2005-06-21T22:34:38.000"), "friend-ids": {{ 46792750, 47197275, 45940765, 43931611, 33201251, 32508732, 23681521, 35069089, 43652710, 22676488, 5098654, 29592897, 18671070, 40200423 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2004-08-18") } ] }
+{ "id": 11779591, "id-copy": 11779591, "alias": "Galina", "name": "GalinaRoberts", "user-since": datetime("2007-03-18T12:09:38.000"), "user-since-copy": datetime("2007-03-18T12:09:38.000"), "friend-ids": {{ 16134690, 41543844 }}, "employment": [ { "organization-name": "Xx-technology", "start-date": date("2010-04-17") } ] }
+{ "id": 11781745, "id-copy": 11781745, "alias": "Merv", "name": "MervStocker", "user-since": datetime("2008-10-15T03:41:54.000"), "user-since-copy": datetime("2008-10-15T03:41:54.000"), "friend-ids": {{ 26394519, 2599602, 40237077, 43817129, 30392481, 43051494, 36128635, 35974184, 37237292, 7775912, 11569464, 9112021, 26837692, 11548106, 29331601, 11126182, 18076463, 33866145, 22408972, 42318835, 47199541, 26807788 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2005-01-15"), "end-date": date("2008-02-18") } ] }
+{ "id": 11782354, "id-copy": 11782354, "alias": "Glynda", "name": "GlyndaEnderly", "user-since": datetime("2007-11-25T06:01:45.000"), "user-since-copy": datetime("2007-11-25T06:01:45.000"), "friend-ids": {{ 16202981, 24035766, 10175614, 27353200, 26183740, 6084065, 31664832, 22446721, 2792685, 37521374, 1999182, 12494503, 18087992, 44433851 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2004-06-10") } ] }
+{ "id": 11788345, "id-copy": 11788345, "alias": "Mindy", "name": "MindyRockwell", "user-since": datetime("2011-02-20T23:55:16.000"), "user-since-copy": datetime("2011-02-20T23:55:16.000"), "friend-ids": {{ 7821092, 24614722, 27718237, 19686343, 43916267, 7882804, 34422272, 46273261, 658009, 42620170, 36177155, 3340224, 27157340, 20438623, 19694381, 15643415, 43465380, 17719224, 37073374, 42060457, 29532671, 3781069, 26121650 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2011-05-11") } ] }
+{ "id": 11793622, "id-copy": 11793622, "alias": "Leonard", "name": "LeonardAlice", "user-since": datetime("2011-03-02T21:42:07.000"), "user-since-copy": datetime("2011-03-02T21:42:07.000"), "friend-ids": {{ 38648452, 2302677, 713863, 2484976, 20706899, 6649310, 9952945, 1293945, 23188221, 43521816, 2398744, 28382427, 45933146, 27717079, 12894240, 8077643, 38945982, 12658937, 36047491, 42431984, 43626155 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2001-02-12"), "end-date": date("2001-06-02") } ] }
+{ "id": 11811079, "id-copy": 11811079, "alias": "Kenelm", "name": "KenelmKellogg", "user-since": datetime("2006-05-14T04:13:36.000"), "user-since-copy": datetime("2006-05-14T04:13:36.000"), "friend-ids": {{ 28287762, 45591894, 12026636, 34381293, 17018521, 37239852, 5735876, 8145944, 34171842, 32986088, 16537938, 20530369, 35161854, 1076550, 26081966, 35666231 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2001-02-03") } ] }
+{ "id": 11822506, "id-copy": 11822506, "alias": "Jerrold", "name": "JerroldEwing", "user-since": datetime("2010-08-27T22:34:36.000"), "user-since-copy": datetime("2010-08-27T22:34:36.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2007-03-21"), "end-date": date("2008-04-26") } ] }
+{ "id": 11839117, "id-copy": 11839117, "alias": "Kyra", "name": "KyraMcdonald", "user-since": datetime("2010-07-08T20:46:49.000"), "user-since-copy": datetime("2010-07-08T20:46:49.000"), "friend-ids": {{ 42933043, 41665211, 13075886, 36147059, 20127919, 31449381, 47427643, 24399833, 16541120, 38909218, 15609877, 46802599, 31772232, 46743670 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2005-06-08"), "end-date": date("2007-11-11") } ] }
+{ "id": 11862502, "id-copy": 11862502, "alias": "Innocent", "name": "InnocentWilliamson", "user-since": datetime("2005-06-09T18:44:51.000"), "user-since-copy": datetime("2005-06-09T18:44:51.000"), "friend-ids": {{ 14750408, 36287814, 21197416, 34246775, 18776860, 32777856, 46956112, 18578056, 13053407, 3282278, 29812571, 25299530, 47168979, 6027296, 10540009 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2010-05-20"), "end-date": date("2010-01-24") } ] }
+{ "id": 11872177, "id-copy": 11872177, "alias": "Lillie", "name": "LillieLineman", "user-since": datetime("2009-09-28T02:48:03.000"), "user-since-copy": datetime("2009-09-28T02:48:03.000"), "friend-ids": {{ 16078664, 22307944, 21464886, 40255882, 39090292, 32823112, 5748916, 46831442, 25498280, 268782, 22829744, 17001614 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2006-02-18") } ] }
+{ "id": 11878948, "id-copy": 11878948, "alias": "Corey", "name": "CoreyWarrick", "user-since": datetime("2005-05-28T15:18:23.000"), "user-since-copy": datetime("2005-05-28T15:18:23.000"), "friend-ids": {{ 17192577, 19646534, 44755348, 28653064, 30539369, 15001411, 11921646, 44450607, 33599896, 41984600, 2187246, 8785209, 28099595 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2010-12-07") } ] }
+{ "id": 11886709, "id-copy": 11886709, "alias": "Leigh", "name": "LeighBatten", "user-since": datetime("2005-06-18T21:25:13.000"), "user-since-copy": datetime("2005-06-18T21:25:13.000"), "friend-ids": {{ 161610, 3498914, 24173074, 33102324, 42213688, 44551300, 36373040, 30704767, 24224319, 5784194, 13092764, 38315503, 13246046, 2836280, 672136, 37021775 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2001-05-26"), "end-date": date("2001-05-11") } ] }
+{ "id": 11886856, "id-copy": 11886856, "alias": "Eldred", "name": "EldredArmstrong", "user-since": datetime("2012-02-20T10:08:40.000"), "user-since-copy": datetime("2012-02-20T10:08:40.000"), "friend-ids": {{ 5146204, 10549788, 40744824, 38277859 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-09-18") } ] }
+{ "id": 11888530, "id-copy": 11888530, "alias": "Louis", "name": "LouisRichards", "user-since": datetime("2011-10-26T02:27:49.000"), "user-since-copy": datetime("2011-10-26T02:27:49.000"), "friend-ids": {{ 40512993, 46289399 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2000-04-18"), "end-date": date("2002-08-03") } ] }
+{ "id": 11893462, "id-copy": 11893462, "alias": "Shonna", "name": "ShonnaDickson", "user-since": datetime("2007-06-12T09:36:50.000"), "user-since-copy": datetime("2007-06-12T09:36:50.000"), "friend-ids": {{ 30462288, 43630666, 35884473, 25217438, 3196051, 41844836, 8922622, 15388786, 33486563, 22739607, 42411271, 47936046, 8921955, 11314832, 13138669, 1057389, 45874085 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2010-06-18") } ] }
+{ "id": 11914129, "id-copy": 11914129, "alias": "Ebenezer", "name": "EbenezerMonahan", "user-since": datetime("2006-01-08T08:17:51.000"), "user-since-copy": datetime("2006-01-08T08:17:51.000"), "friend-ids": {{ 9692770 }}, "employment": [ { "organization-name": "Hatcom", "start-date": date("2002-10-22"), "end-date": date("2005-07-17") } ] }
+{ "id": 11921524, "id-copy": 11921524, "alias": "Mickey", "name": "MickeySybilla", "user-since": datetime("2012-03-28T17:05:25.000"), "user-since-copy": datetime("2012-03-28T17:05:25.000"), "friend-ids": {{ 40813978, 14172552, 40702786, 929262, 2220334, 33077762, 20716547, 11400385, 21916926, 38422356, 13378381, 32362984, 8162369, 8965084, 37823302, 3542211, 29294304, 37672739, 28359647 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2007-09-27") } ] }
+{ "id": 11937787, "id-copy": 11937787, "alias": "Addison", "name": "AddisonEckert", "user-since": datetime("2007-04-26T01:06:38.000"), "user-since-copy": datetime("2007-04-26T01:06:38.000"), "friend-ids": {{ 6446414, 23134374, 38952228, 25368200, 47868440, 29231397, 15672064, 2482344, 22824732, 13563448, 43826877 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2009-10-09") } ] }
+{ "id": 11951800, "id-copy": 11951800, "alias": "Camron", "name": "CamronBrooks", "user-since": datetime("2006-03-05T19:32:03.000"), "user-since-copy": datetime("2006-03-05T19:32:03.000"), "friend-ids": {{ 39430755, 45789857, 5352132, 34490450, 39117503, 2233039, 16387184 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2006-12-26"), "end-date": date("2007-11-16") } ] }
+{ "id": 11953306, "id-copy": 11953306, "alias": "Teale", "name": "TealeHoltzer", "user-since": datetime("2007-02-14T21:50:54.000"), "user-since-copy": datetime("2007-02-14T21:50:54.000"), "friend-ids": {{ 30902622, 26223630, 46832466, 32585590, 34005386, 23371032, 25984545, 7502619 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2010-02-14"), "end-date": date("2011-07-08") } ] }
+{ "id": 11969527, "id-copy": 11969527, "alias": "Adrian", "name": "AdrianTedrow", "user-since": datetime("2012-02-13T21:27:48.000"), "user-since-copy": datetime("2012-02-13T21:27:48.000"), "friend-ids": {{ 36940614, 29564878 }}, "employment": [ { "organization-name": "Ganjastrip", "start-date": date("2002-01-16") } ] }
+{ "id": 11978782, "id-copy": 11978782, "alias": "Louiza", "name": "LouizaLlora", "user-since": datetime("2012-06-24T06:19:05.000"), "user-since-copy": datetime("2012-06-24T06:19:05.000"), "friend-ids": {{ 36495107, 35125435, 30347420, 17703387, 40909002 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2008-05-25") } ] }
+{ "id": 11987626, "id-copy": 11987626, "alias": "Chassidy", "name": "ChassidyHector", "user-since": datetime("2008-07-23T16:16:55.000"), "user-since-copy": datetime("2008-07-23T16:16:55.000"), "friend-ids": {{ 29831103, 12411598, 20670552, 42569662 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2010-08-22") } ] }
+{ "id": 11989645, "id-copy": 11989645, "alias": "Weston", "name": "WestonPershing", "user-since": datetime("2010-04-02T17:25:31.000"), "user-since-copy": datetime("2010-04-02T17:25:31.000"), "friend-ids": {{ 11689127 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2006-03-27") } ] }
+{ "id": 11989660, "id-copy": 11989660, "alias": "Rolland", "name": "RollandGarneis", "user-since": datetime("2008-09-16T19:54:32.000"), "user-since-copy": datetime("2008-09-16T19:54:32.000"), "friend-ids": {{ 30959592, 6160903, 27316367, 6518756, 23008668, 36942525, 39489068, 8710310, 17726852, 72593, 15440937, 4901728, 28916846, 38257093, 28414859, 8857050 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-02-11") } ] }
+{ "id": 9001816, "id-copy": 9001816, "alias": "Concordia", "name": "ConcordiaThomlinson", "user-since": datetime("2006-04-13T03:30:17.000"), "user-since-copy": datetime("2006-04-13T03:30:17.000"), "friend-ids": {{ 31001079, 10620343, 29160614, 8991085, 45471665, 865015, 11592391, 33106281, 15448665, 29325047, 47814022, 4562661, 11895808, 41974900 }}, "employment": [ { "organization-name": "Vaiatech", "start-date": date("2002-03-25") } ] }
+{ "id": 9005248, "id-copy": 9005248, "alias": "Jervis", "name": "JervisWarrick", "user-since": datetime("2007-02-06T17:54:17.000"), "user-since-copy": datetime("2007-02-06T17:54:17.000"), "friend-ids": {{ 5038062, 15101135, 28136073, 10706469, 8706391, 10623870, 1759405, 37020186, 17173998, 14985805, 19308437, 43696985, 46650868, 25621415, 14252531, 44491166, 42536769, 33614525, 34665072, 640793 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2010-08-21") } ] }
+{ "id": 9039973, "id-copy": 9039973, "alias": "Desmond", "name": "DesmondRice", "user-since": datetime("2008-04-17T12:00:38.000"), "user-since-copy": datetime("2008-04-17T12:00:38.000"), "friend-ids": {{ 16128090, 28937536, 30905098, 25666304, 23272582, 29438991, 42040849, 42396891, 9345677, 9260055, 17415621, 31581557, 1249365, 20734436, 2341357, 36307325, 20347771, 23723655 }}, "employment": [ { "organization-name": "Zimcone", "start-date": date("2002-10-24"), "end-date": date("2008-02-24") } ] }
+{ "id": 9050866, "id-copy": 9050866, "alias": "Jimmie", "name": "JimmieBicknell", "user-since": datetime("2007-02-15T16:39:19.000"), "user-since-copy": datetime("2007-02-15T16:39:19.000"), "friend-ids": {{ 17248854, 13830961, 10571254, 637235, 18219702, 4541511, 42876025, 19679892, 14009802, 15312402, 20914286, 41969971, 39807443, 5990836, 1594551, 25853135, 25021671, 21604624, 47574478 }}, "employment": [ { "organization-name": "Ontohothex", "start-date": date("2001-04-09") } ] }
+{ "id": 9056494, "id-copy": 9056494, "alias": "Alvena", "name": "AlvenaPearsall", "user-since": datetime("2005-08-09T08:50:25.000"), "user-since-copy": datetime("2005-08-09T08:50:25.000"), "friend-ids": {{ 26263956, 80589, 37669623, 32875186, 42026139, 22169384, 47224581, 25632957, 28392334, 42393204, 15028714, 554526 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2005-02-19") } ] }
+{ "id": 9074290, "id-copy": 9074290, "alias": "Riley", "name": "RileyBode", "user-since": datetime("2010-11-20T01:12:36.000"), "user-since-copy": datetime("2010-11-20T01:12:36.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2004-10-12") } ] }
+{ "id": 9099376, "id-copy": 9099376, "alias": "Tena", "name": "TenaKline", "user-since": datetime("2011-10-20T14:46:29.000"), "user-since-copy": datetime("2011-10-20T14:46:29.000"), "friend-ids": {{ 28615752, 16589994, 24896126, 32768352, 40921310, 22643822, 39206554, 45652466, 17237997, 44705249, 30599864, 17750741, 14758376, 4842744 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2000-03-18") } ] }
+{ "id": 9139057, "id-copy": 9139057, "alias": "Esther", "name": "EstherUllman", "user-since": datetime("2010-01-05T19:25:44.000"), "user-since-copy": datetime("2010-01-05T19:25:44.000"), "friend-ids": {{ 25401186, 25915246, 33727208, 17431690, 24541706, 19998503, 42399029, 30405906, 20023918, 9788811, 32513474, 14919034, 10073867, 9309154, 1423378, 37386209, 16346279, 45167618, 34716280, 29023237, 20639001, 332097, 28344544 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2003-09-05"), "end-date": date("2009-10-17") } ] }
+{ "id": 9158293, "id-copy": 9158293, "alias": "Cortney", "name": "CortneyPainter", "user-since": datetime("2006-03-15T09:03:09.000"), "user-since-copy": datetime("2006-03-15T09:03:09.000"), "friend-ids": {{ 42832801, 24287760, 37934712, 43376751, 24673433, 14168792, 46862345, 46736573, 21181723, 2094484, 30254710, 45439521, 26589024, 45746175, 13898656, 13470143, 9669892 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2011-06-13") } ] }
+{ "id": 9190501, "id-copy": 9190501, "alias": "Leonardo", "name": "LeonardoBarr", "user-since": datetime("2008-02-23T14:20:45.000"), "user-since-copy": datetime("2008-02-23T14:20:45.000"), "friend-ids": {{ 24193096, 44367993, 10307197, 20420512, 36000544, 45069724, 42621729, 10863302, 21701700, 7110735, 6226449, 3269792, 12797617, 19460642, 7357145, 27051982, 31847212, 28691920, 382743, 11602175, 1787538, 42283089, 19610964 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2001-06-25") } ] }
+{ "id": 9201610, "id-copy": 9201610, "alias": "Elaine", "name": "ElaineMcclymonds", "user-since": datetime("2008-04-13T17:06:35.000"), "user-since-copy": datetime("2008-04-13T17:06:35.000"), "friend-ids": {{ 18934024, 5114594, 25593808 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-08-28") } ] }
+{ "id": 9205834, "id-copy": 9205834, "alias": "Tristin", "name": "TristinWalker", "user-since": datetime("2012-04-25T01:08:05.000"), "user-since-copy": datetime("2012-04-25T01:08:05.000"), "friend-ids": {{ 2222398, 15073251, 16222879, 24405969, 32651599, 44500557, 31699173, 41724026, 1745441, 9674348, 29594086, 26580583, 42258300, 36027050, 3204087, 2147469, 36519580 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2007-02-09") } ] }
+{ "id": 9210847, "id-copy": 9210847, "alias": "Kristeen", "name": "KristeenShaffer", "user-since": datetime("2008-01-04T12:31:50.000"), "user-since-copy": datetime("2008-01-04T12:31:50.000"), "friend-ids": {{ 662954, 18313322, 10737685, 5498351, 24795605, 4497605, 45729062, 31007969, 16211490, 19408104, 5882137, 12084923, 14143383, 31263672, 32404691, 8973685, 32756191, 3822704 }}, "employment": [ { "organization-name": "Rungozoom", "start-date": date("2009-12-07"), "end-date": date("2010-02-08") } ] }
+{ "id": 9223375, "id-copy": 9223375, "alias": "Anne", "name": "AnneMoore", "user-since": datetime("2010-07-16T22:06:20.000"), "user-since-copy": datetime("2010-07-16T22:06:20.000"), "friend-ids": {{ 45553359, 40589681, 9461257, 39253068, 14447226, 37656564, 37047377, 34855985 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2011-04-25") } ] }
+{ "id": 9226960, "id-copy": 9226960, "alias": "Irish", "name": "IrishJohnson", "user-since": datetime("2009-09-07T21:02:01.000"), "user-since-copy": datetime("2009-09-07T21:02:01.000"), "friend-ids": {{ 4920892, 15681759, 19110917, 26620361, 34712468, 40890326, 20312413 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2009-11-11") } ] }
+{ "id": 9262768, "id-copy": 9262768, "alias": "Graham", "name": "GrahamHunt", "user-since": datetime("2009-03-19T13:15:02.000"), "user-since-copy": datetime("2009-03-19T13:15:02.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2012-04-23"), "end-date": date("2012-04-15") } ] }
+{ "id": 9271291, "id-copy": 9271291, "alias": "Kaitlynn", "name": "KaitlynnPycroft", "user-since": datetime("2010-10-09T11:30:12.000"), "user-since-copy": datetime("2010-10-09T11:30:12.000"), "friend-ids": {{ 38067939, 25732262, 17076819, 19477302, 29794559 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2006-09-04") } ] }
+{ "id": 9275620, "id-copy": 9275620, "alias": "Jackie", "name": "JackieRumbaugh", "user-since": datetime("2011-10-11T07:30:25.000"), "user-since-copy": datetime("2011-10-11T07:30:25.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Technohow", "start-date": date("2004-01-14") } ] }
+{ "id": 9288154, "id-copy": 9288154, "alias": "Lauren", "name": "LaurenGraff", "user-since": datetime("2005-12-28T07:21:17.000"), "user-since-copy": datetime("2005-12-28T07:21:17.000"), "friend-ids": {{ 38658043, 4029859, 43671010, 20184796, 23429992, 3744331, 39377881, 1336305, 33712064, 36443 }}, "employment": [ { "organization-name": "strongex", "start-date": date("2009-04-06") } ] }
+{ "id": 9297361, "id-copy": 9297361, "alias": "Yasmine", "name": "YasmineBullard", "user-since": datetime("2006-07-11T23:54:23.000"), "user-since-copy": datetime("2006-07-11T23:54:23.000"), "friend-ids": {{ 27580636, 11448774, 32271178, 9627095, 11487349, 46595708 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2001-10-06"), "end-date": date("2003-03-05") } ] }
+{ "id": 9317395, "id-copy": 9317395, "alias": "Timothy", "name": "TimothyMays", "user-since": datetime("2007-05-23T15:42:26.000"), "user-since-copy": datetime("2007-05-23T15:42:26.000"), "friend-ids": {{ 38066468, 16126194, 20685050, 8542551, 36810930, 36333903, 31522960, 44908120, 45171970, 9212095, 16986466, 41689196, 22300874, 45983009, 30918582, 5896299, 2682406, 6649020, 33199300, 14523848 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2006-04-16"), "end-date": date("2008-02-21") } ] }
+{ "id": 9320062, "id-copy": 9320062, "alias": "Samantha", "name": "SamanthaTanner", "user-since": datetime("2010-06-25T14:13:49.000"), "user-since-copy": datetime("2010-06-25T14:13:49.000"), "friend-ids": {{ 19538026 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2009-02-24") } ] }
+{ "id": 9329746, "id-copy": 9329746, "alias": "Albert", "name": "AlbertZundel", "user-since": datetime("2005-11-01T23:41:02.000"), "user-since-copy": datetime("2005-11-01T23:41:02.000"), "friend-ids": {{ 44252308, 14483702, 27233282, 24263669, 35409140, 38591765, 42901786, 24502313, 6384822, 36359249, 36816246, 16578182, 530819, 29481837, 12698700, 6101521, 11990316, 35327955, 10435272 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2003-08-06"), "end-date": date("2010-09-22") } ] }
+{ "id": 9331075, "id-copy": 9331075, "alias": "Monday", "name": "MondayWarrick", "user-since": datetime("2012-01-13T06:13:30.000"), "user-since-copy": datetime("2012-01-13T06:13:30.000"), "friend-ids": {{ 27699724, 39094128, 11014820, 44605243, 20177679, 37579779, 35875781, 13713739, 8882475, 37427927, 28595578, 3788567, 31200715, 40590973, 7630783, 36856789, 22013865 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2000-04-08") } ] }
+{ "id": 9367306, "id-copy": 9367306, "alias": "Jacinth", "name": "JacinthBynum", "user-since": datetime("2012-03-08T11:26:04.000"), "user-since-copy": datetime("2012-03-08T11:26:04.000"), "friend-ids": {{ 35048012, 42620612, 39526901, 12673410, 16363143, 45509270, 47714729, 47902094, 12551745, 45510597, 31513255, 2848992, 16088751, 1953590, 32956014, 38607548, 15982103, 31161780, 7331812, 44977526, 15022020, 19905573 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2003-03-24") } ] }
+{ "id": 9373726, "id-copy": 9373726, "alias": "Joe", "name": "JoeRoche", "user-since": datetime("2005-07-09T16:42:53.000"), "user-since-copy": datetime("2005-07-09T16:42:53.000"), "friend-ids": {{ 16433644, 5532847, 743901, 2134179, 43053028, 36961668, 9731766, 45686582, 17084459, 27026683, 1687547, 6582422, 38798685, 9871595, 2677099, 42280963, 32191501, 4347234 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2009-09-16") } ] }
+{ "id": 9389254, "id-copy": 9389254, "alias": "Jon", "name": "JonShaw", "user-since": datetime("2006-12-10T11:28:23.000"), "user-since-copy": datetime("2006-12-10T11:28:23.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2010-07-24") } ] }
+{ "id": 9396193, "id-copy": 9396193, "alias": "Franklyn", "name": "FranklynVorrasi", "user-since": datetime("2007-06-27T09:38:03.000"), "user-since-copy": datetime("2007-06-27T09:38:03.000"), "friend-ids": {{ 12870114, 28811462, 19219273, 38745339, 22310708, 11419733, 21583164, 42276545, 1177024, 43617748, 11702666, 19332437, 1523883, 40265275, 41227772 }}, "employment": [ { "organization-name": "over-it", "start-date": date("2001-03-13"), "end-date": date("2009-02-07") } ] }
+{ "id": 9408427, "id-copy": 9408427, "alias": "Matt", "name": "MattPritchard", "user-since": datetime("2008-10-02T15:31:39.000"), "user-since-copy": datetime("2008-10-02T15:31:39.000"), "friend-ids": {{ 3596345, 15476624, 33857894, 13004846, 29332890, 23638145, 43402648, 14337754, 3290802, 10537283, 9989868, 33400736, 43952799, 34128983, 3090230, 12591428, 15051691, 7239629, 10295253, 23448932, 30507945 }}, "employment": [ { "organization-name": "Fixelectrics", "start-date": date("2012-02-05") } ] }
+{ "id": 9453925, "id-copy": 9453925, "alias": "Ritchie", "name": "RitchieJube", "user-since": datetime("2008-04-28T12:33:34.000"), "user-since-copy": datetime("2008-04-28T12:33:34.000"), "friend-ids": {{ 44327769, 45189889, 11098478, 41612069, 40647950, 638474, 21614810, 22273745, 6230791, 15120137, 18477729, 16895919, 5907839, 43993812, 31639138, 7966991, 11024409 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2012-07-22") } ] }
+{ "id": 9461770, "id-copy": 9461770, "alias": "Georgina", "name": "GeorginaPearson", "user-since": datetime("2005-02-04T09:47:21.000"), "user-since-copy": datetime("2005-02-04T09:47:21.000"), "friend-ids": {{ 26615251, 5874803, 5189465, 29564778, 1778424, 38706542, 38915757, 16819394, 3318129, 2166806, 30570432, 15192853, 4857015, 41673300, 23510020 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2003-06-06") } ] }
+{ "id": 9467614, "id-copy": 9467614, "alias": "Eloisa", "name": "EloisaEvans", "user-since": datetime("2012-01-20T01:00:51.000"), "user-since-copy": datetime("2012-01-20T01:00:51.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2000-11-03"), "end-date": date("2003-01-14") } ] }
+{ "id": 9478720, "id-copy": 9478720, "alias": "Angelia", "name": "AngeliaKettlewell", "user-since": datetime("2005-05-27T06:29:30.000"), "user-since-copy": datetime("2005-05-27T06:29:30.000"), "friend-ids": {{ 42556433, 20033025, 38112512, 19420757, 31822717, 7116081, 39544900, 19203395, 46787205, 32303456, 4509345, 45558040, 42616291, 6929369, 9272653, 37459048, 37113569, 38942369, 47741031, 46761451, 14163845 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2012-03-28"), "end-date": date("2012-03-04") } ] }
+{ "id": 9482569, "id-copy": 9482569, "alias": "Marty", "name": "MartyBurnett", "user-since": datetime("2006-03-21T10:10:40.000"), "user-since-copy": datetime("2006-03-21T10:10:40.000"), "friend-ids": {{ 5791578, 3884688, 7686005 }}, "employment": [ { "organization-name": "Solfix", "start-date": date("2009-02-01") } ] }
+{ "id": 9497698, "id-copy": 9497698, "alias": "Jenny", "name": "JennyBiery", "user-since": datetime("2007-07-24T17:20:06.000"), "user-since-copy": datetime("2007-07-24T17:20:06.000"), "friend-ids": {{ 37832227, 17148339, 38184683, 45775690, 17511050, 1866913, 30631091, 5996302, 3796747, 33135567, 5930972, 9509054, 44003369, 34299276, 16135297, 15435466, 42464299, 34961792, 47264306, 30734198, 26192613 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2007-10-02"), "end-date": date("2011-09-20") } ] }
+{ "id": 9502096, "id-copy": 9502096, "alias": "Hebe", "name": "HebeEndsley", "user-since": datetime("2012-08-08T18:55:28.000"), "user-since-copy": datetime("2012-08-08T18:55:28.000"), "friend-ids": {{ 34917916, 5530270, 12994124, 25113086, 28819142, 44228082 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2007-04-11") } ] }
+{ "id": 9503443, "id-copy": 9503443, "alias": "Ebenezer", "name": "EbenezerFulton", "user-since": datetime("2012-07-03T20:14:05.000"), "user-since-copy": datetime("2012-07-03T20:14:05.000"), "friend-ids": {{ 11155403, 7932344, 24822329, 19823943, 37496284 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2011-08-22") } ] }
+{ "id": 9510451, "id-copy": 9510451, "alias": "Chuck", "name": "ChuckFinck", "user-since": datetime("2011-09-10T08:27:31.000"), "user-since-copy": datetime("2011-09-10T08:27:31.000"), "friend-ids": {{ 5559039, 8997599, 8311284, 20478562, 13734713, 21511695, 30393493 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2001-10-12") } ] }
+{ "id": 9521683, "id-copy": 9521683, "alias": "Tennille", "name": "TennilleHamilton", "user-since": datetime("2009-04-21T20:56:25.000"), "user-since-copy": datetime("2009-04-21T20:56:25.000"), "friend-ids": {{ 32048407, 3619952, 41652292, 45570368, 31678290, 11241324 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2010-10-15") } ] }
+{ "id": 9549610, "id-copy": 9549610, "alias": "Blossom", "name": "BlossomGreif", "user-since": datetime("2010-05-03T21:08:56.000"), "user-since-copy": datetime("2010-05-03T21:08:56.000"), "friend-ids": {{ 47791115, 42952282 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2011-12-25"), "end-date": date("2011-11-27") } ] }
+{ "id": 9556570, "id-copy": 9556570, "alias": "Kassandra", "name": "KassandraKern", "user-since": datetime("2010-12-03T15:29:12.000"), "user-since-copy": datetime("2010-12-03T15:29:12.000"), "friend-ids": {{ 35944118, 3024691, 43927521, 44121317, 29834404, 18626717, 47095811, 38438153, 30557309, 37143411, 41634172, 23338449, 30455300, 12009022, 26366377, 36381324, 25084236, 36521163, 20063914, 11419154, 40243010, 9336807, 3544397, 20455720 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2005-02-12") } ] }
+{ "id": 9574261, "id-copy": 9574261, "alias": "Kalysta", "name": "KalystaBeedell", "user-since": datetime("2010-01-27T14:57:31.000"), "user-since-copy": datetime("2010-01-27T14:57:31.000"), "friend-ids": {{ 5811189, 22155580, 41736564, 27399656, 40013573, 28340467, 45690668, 16097604, 9655169, 44870593 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2009-12-16"), "end-date": date("2010-10-22") } ] }
+{ "id": 9575338, "id-copy": 9575338, "alias": "Isabell", "name": "IsabellWain", "user-since": datetime("2011-07-05T12:26:43.000"), "user-since-copy": datetime("2011-07-05T12:26:43.000"), "friend-ids": {{ 42651024, 15652966, 27390748, 19369775, 44130969, 45269514, 210916, 36228917, 31857984, 11676544, 42752689, 14021599, 31749945, 9405328, 37567152, 17083209, 32654328, 39607403, 18699149, 37082017, 6059914, 881724 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2003-06-04") } ] }
+{ "id": 9577867, "id-copy": 9577867, "alias": "Lavette", "name": "LavetteSnyder", "user-since": datetime("2007-02-22T10:01:04.000"), "user-since-copy": datetime("2007-02-22T10:01:04.000"), "friend-ids": {{ 25749553, 31379974, 15118772, 38725424, 26760226, 8908746, 20299291, 20288328, 19659485, 22400738, 477700, 20253845, 12753420, 46016251, 29518581, 21898853, 19015599, 3455762, 19350275, 2630122 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2011-04-22") } ] }
+{ "id": 9598486, "id-copy": 9598486, "alias": "Grover", "name": "GroverNewbern", "user-since": datetime("2012-01-06T20:50:38.000"), "user-since-copy": datetime("2012-01-06T20:50:38.000"), "friend-ids": {{ 8389292, 25521744, 23387036, 38008541, 43673600, 23656679, 1401712, 39164079, 1810015, 20625744, 15651316, 23441546, 24572830, 19077921 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2009-07-28"), "end-date": date("2010-06-09") } ] }
+{ "id": 9599647, "id-copy": 9599647, "alias": "Alexandria", "name": "AlexandriaWade", "user-since": datetime("2012-06-25T06:48:48.000"), "user-since-copy": datetime("2012-06-25T06:48:48.000"), "friend-ids": {{ 20910866, 20843338, 8182424, 21070448, 43548111, 39370893, 26760127, 11135506 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2011-06-02") } ] }
+{ "id": 9635563, "id-copy": 9635563, "alias": "Tamsen", "name": "TamsenCowart", "user-since": datetime("2010-10-07T05:11:20.000"), "user-since-copy": datetime("2010-10-07T05:11:20.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2007-01-07") } ] }
+{ "id": 9636802, "id-copy": 9636802, "alias": "Gage", "name": "GageHair", "user-since": datetime("2011-01-23T22:31:49.000"), "user-since-copy": datetime("2011-01-23T22:31:49.000"), "friend-ids": {{ 46795684, 38195763, 25882078, 28871879, 5178144, 17683475, 43441471, 5427133, 13936915, 2608474, 9513798, 31041524, 557454, 22452168, 12948004, 16835098, 1151241, 37188687 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2000-05-02"), "end-date": date("2010-02-13") } ] }
+{ "id": 9638626, "id-copy": 9638626, "alias": "Hisako", "name": "HisakoEisaman", "user-since": datetime("2008-05-26T23:34:43.000"), "user-since-copy": datetime("2008-05-26T23:34:43.000"), "friend-ids": {{ 17773563, 18434504, 1082020, 40557107, 43294701, 1982610, 8259201, 47490886, 20044705, 35882471, 7297053, 17276976, 38660830, 36435103, 29511457, 3474864, 17100964, 23978369, 6260698, 17616437, 1617227, 18325960, 42613056 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2009-07-12") } ] }
+{ "id": 9690049, "id-copy": 9690049, "alias": "Ahmed", "name": "AhmedVinsant", "user-since": datetime("2009-12-24T23:10:10.000"), "user-since-copy": datetime("2009-12-24T23:10:10.000"), "friend-ids": {{ 9425379, 24773026, 47645199, 12718095, 32145472, 30931581, 11512330, 46898742, 26190870, 38985851, 40692118, 34327720, 47432207 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2002-05-26") } ] }
+{ "id": 9693988, "id-copy": 9693988, "alias": "Geordie", "name": "GeordieBunten", "user-since": datetime("2006-08-03T15:00:25.000"), "user-since-copy": datetime("2006-08-03T15:00:25.000"), "friend-ids": {{ 31987089, 15556815, 3656365, 35713356, 9573642, 38459850, 44400137, 44882118, 44921684, 47393814, 7869122, 35085016, 43725704, 17602789, 9966406, 20936803, 26425879, 41666932 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2007-01-20") } ] }
+{ "id": 9696160, "id-copy": 9696160, "alias": "Lawerence", "name": "LawerenceLudwig", "user-since": datetime("2005-09-04T07:08:01.000"), "user-since-copy": datetime("2005-09-04T07:08:01.000"), "friend-ids": {{ 33125788, 14719007, 35434564 }}, "employment": [ { "organization-name": "U-ron", "start-date": date("2001-02-02") } ] }
+{ "id": 9699673, "id-copy": 9699673, "alias": "Jim", "name": "JimPycroft", "user-since": datetime("2012-07-25T20:20:38.000"), "user-since-copy": datetime("2012-07-25T20:20:38.000"), "friend-ids": {{ 14858146, 47543880, 3186927, 38198580, 2365336, 5255886, 11178580, 41188272, 17623582, 6422949, 4405751, 12128017, 32409443, 38861849, 16511892, 24515731, 46665640, 40644816, 19341995, 44288533, 26148671 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2007-01-24"), "end-date": date("2009-12-16") } ] }
+{ "id": 9709663, "id-copy": 9709663, "alias": "Trevor", "name": "TrevorSell", "user-since": datetime("2008-08-28T18:18:54.000"), "user-since-copy": datetime("2008-08-28T18:18:54.000"), "friend-ids": {{ 13788189, 27667188, 588943, 1574745, 5763893, 19661124, 45630528, 47078471, 42976078, 32943975 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2007-07-04") } ] }
+{ "id": 9719995, "id-copy": 9719995, "alias": "Hazel", "name": "HazelKnopsnider", "user-since": datetime("2007-04-05T01:11:42.000"), "user-since-copy": datetime("2007-04-05T01:11:42.000"), "friend-ids": {{ 38515770, 23212874, 6000594, 27957554, 28093880, 3726628, 22800428, 42313894, 23190476, 18537188, 22083915, 43478674, 33364444, 19158958, 1590605, 36792931, 42057988, 33286729, 29580197, 25232028 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2008-09-28") } ] }
+{ "id": 9736855, "id-copy": 9736855, "alias": "Sudie", "name": "SudieAlbright", "user-since": datetime("2011-10-08T08:46:27.000"), "user-since-copy": datetime("2011-10-08T08:46:27.000"), "friend-ids": {{ 20506190, 13537252, 46211902, 4320089 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2004-12-07"), "end-date": date("2010-07-02") } ] }
+{ "id": 9752227, "id-copy": 9752227, "alias": "Audley", "name": "AudleyPeters", "user-since": datetime("2006-07-27T01:15:35.000"), "user-since-copy": datetime("2006-07-27T01:15:35.000"), "friend-ids": {{ 877448, 29611844, 2844046, 42493473, 28216181, 353847, 44172105, 36184409, 44010617 }}, "employment": [ { "organization-name": "Hot-tech", "start-date": date("2002-12-17") } ] }
+{ "id": 9760834, "id-copy": 9760834, "alias": "Lavette", "name": "LavettePirl", "user-since": datetime("2006-02-12T07:28:53.000"), "user-since-copy": datetime("2006-02-12T07:28:53.000"), "friend-ids": {{ 27450797, 36415787 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2002-09-20") } ] }
+{ "id": 9765517, "id-copy": 9765517, "alias": "Alexia", "name": "AlexiaTownsend", "user-since": datetime("2006-02-23T13:26:33.000"), "user-since-copy": datetime("2006-02-23T13:26:33.000"), "friend-ids": {{ 39892441, 43413199, 45070224, 46877180, 24247279, 26450737, 29111107, 46768934, 11833332, 25913646, 43063781 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2003-02-08") } ] }
+{ "id": 9774613, "id-copy": 9774613, "alias": "Kaycee", "name": "KayceeGeyer", "user-since": datetime("2008-12-19T06:09:36.000"), "user-since-copy": datetime("2008-12-19T06:09:36.000"), "friend-ids": {{ 35485847, 33668074, 21309976, 40428525, 40450508, 30804358, 1365381, 5197688, 37844952, 4076960, 28446817, 20696590, 23896488, 33454126, 21411087, 9300550, 12986775, 36731809, 47850175, 9503217, 22481614, 29556396, 15013896, 14407126 }}, "employment": [ { "organization-name": "Lexicone", "start-date": date("2001-03-23"), "end-date": date("2003-01-16") } ] }
+{ "id": 9784687, "id-copy": 9784687, "alias": "Larrie", "name": "LarrieStroh", "user-since": datetime("2005-12-03T13:45:30.000"), "user-since-copy": datetime("2005-12-03T13:45:30.000"), "friend-ids": {{ 38055237, 43436653, 21194063, 30405058, 7754813, 14616686, 3434657, 24778389, 5653770, 8600235, 44560871, 4379727, 32140404, 35445864, 24133933, 21379278, 45626842, 25710375, 25970333, 16831917 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2006-09-18") } ] }
+{ "id": 9809977, "id-copy": 9809977, "alias": "Kassandra", "name": "KassandraHarding", "user-since": datetime("2007-05-01T06:22:22.000"), "user-since-copy": datetime("2007-05-01T06:22:22.000"), "friend-ids": {{ 29945374, 38811992, 41372042, 28714909, 16897620, 5020268, 24134801, 26310926, 32871167, 18787983, 47295432, 31873694, 36300817, 42779931, 27486692 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2001-08-26") } ] }
+{ "id": 9812020, "id-copy": 9812020, "alias": "Elias", "name": "EliasBuck", "user-since": datetime("2012-08-03T07:52:34.000"), "user-since-copy": datetime("2012-08-03T07:52:34.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2008-07-05"), "end-date": date("2008-12-18") } ] }
+{ "id": 9842389, "id-copy": 9842389, "alias": "Nicolas", "name": "NicolasHynes", "user-since": datetime("2005-08-10T23:35:18.000"), "user-since-copy": datetime("2005-08-10T23:35:18.000"), "friend-ids": {{ 40180500, 33396487, 26907885, 4321366, 10229201, 41118923 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2006-10-23"), "end-date": date("2010-03-11") } ] }
+{ "id": 9854788, "id-copy": 9854788, "alias": "Mathilda", "name": "MathildaVanleer", "user-since": datetime("2007-01-05T08:45:07.000"), "user-since-copy": datetime("2007-01-05T08:45:07.000"), "friend-ids": {{ 20510022, 1353061, 24801201, 11438611, 30281530, 15596343, 29404248, 2024925, 3425369, 18530400 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2002-07-22"), "end-date": date("2011-02-24") } ] }
+{ "id": 9882241, "id-copy": 9882241, "alias": "Dillon", "name": "DillonSimpson", "user-since": datetime("2006-03-20T13:21:16.000"), "user-since-copy": datetime("2006-03-20T13:21:16.000"), "friend-ids": {{ 22747996, 6266176, 22832223, 30880579, 35481343, 48005259, 381757, 27560756, 6053858, 42532723, 33355330, 40374460, 39019469, 35869327 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2007-06-13"), "end-date": date("2011-08-15") } ] }
+{ "id": 9883165, "id-copy": 9883165, "alias": "Dean", "name": "DeanKern", "user-since": datetime("2005-11-02T13:10:37.000"), "user-since-copy": datetime("2005-11-02T13:10:37.000"), "friend-ids": {{ 33343261, 27280204, 31345192, 723310, 11949431, 4787422, 28427922, 11974873, 24553234, 19067609, 12178905, 38171944, 26832701, 47422914, 47782561, 26391811, 28206950, 17135029, 37069726, 40613638, 11509775 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2003-02-16"), "end-date": date("2009-12-16") } ] }
+{ "id": 9885289, "id-copy": 9885289, "alias": "Kayla", "name": "KaylaDugger", "user-since": datetime("2007-10-20T12:55:38.000"), "user-since-copy": datetime("2007-10-20T12:55:38.000"), "friend-ids": {{ 1821427, 46609485, 4532131 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2009-02-15"), "end-date": date("2009-11-17") } ] }
+{ "id": 9890854, "id-copy": 9890854, "alias": "Linwood", "name": "LinwoodBrown", "user-since": datetime("2005-09-09T12:38:00.000"), "user-since-copy": datetime("2005-09-09T12:38:00.000"), "friend-ids": {{ 13728190, 31562633, 3437344, 13841675, 38528685 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2006-05-08"), "end-date": date("2009-08-26") } ] }
+{ "id": 9897094, "id-copy": 9897094, "alias": "Raynard", "name": "RaynardWade", "user-since": datetime("2010-05-12T19:44:55.000"), "user-since-copy": datetime("2010-05-12T19:44:55.000"), "friend-ids": {{ 21246472, 34504200, 43744110, 30518742, 1016046, 17644601, 47173648, 11643135, 22382871, 38535297, 17156487, 30328939, 14770807, 9365820, 36893585, 30122942, 37610936, 44304872 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2004-02-18") } ] }
+{ "id": 9910003, "id-copy": 9910003, "alias": "Arline", "name": "ArlineElinor", "user-since": datetime("2012-07-20T16:57:36.000"), "user-since-copy": datetime("2012-07-20T16:57:36.000"), "friend-ids": {{ 34121202, 19342891, 45323168, 17272278, 6471047, 3726738, 48003127, 32423724, 38588754, 44816854, 13688032, 12876442 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2009-07-19"), "end-date": date("2009-04-17") } ] }
+{ "id": 9934939, "id-copy": 9934939, "alias": "Camilla", "name": "CamillaRhinehart", "user-since": datetime("2008-12-06T10:44:45.000"), "user-since-copy": datetime("2008-12-06T10:44:45.000"), "friend-ids": {{ 17020237, 36188716, 32765819, 20068359, 23060675, 16692600 }}, "employment": [ { "organization-name": "Ransaofan", "start-date": date("2012-04-05") } ] }
+{ "id": 9951325, "id-copy": 9951325, "alias": "Sarah", "name": "SarahRockwell", "user-since": datetime("2009-08-25T01:56:51.000"), "user-since-copy": datetime("2009-08-25T01:56:51.000"), "friend-ids": {{ 14846488, 32939876, 43509116, 36687501, 6496360, 47346160, 20558288, 21828060 }}, "employment": [ { "organization-name": "Quoline", "start-date": date("2002-11-18") } ] }
+{ "id": 9955486, "id-copy": 9955486, "alias": "Jerrod", "name": "JerrodBeach", "user-since": datetime("2007-04-18T07:24:36.000"), "user-since-copy": datetime("2007-04-18T07:24:36.000"), "friend-ids": {{ 9760902, 36268051, 11373781, 42337286, 41818514, 20451257, 23673069, 14313303, 6548991, 34820597, 17346574, 46871090, 263833, 38179383, 14434022 }}, "employment": [ { "organization-name": "Roundhex", "start-date": date("2003-09-01"), "end-date": date("2007-06-11") } ] }
+{ "id": 9958378, "id-copy": 9958378, "alias": "Floyd", "name": "FloydErrett", "user-since": datetime("2006-07-06T02:51:46.000"), "user-since-copy": datetime("2006-07-06T02:51:46.000"), "friend-ids": {{ 38108839, 44502073, 19244279, 45055684, 32489890, 25184431, 34275591, 47288414, 46973922, 28264345, 10024409, 4791958, 40576138, 33446414, 359486, 25595793, 25140170, 23149057, 47032976, 4283407 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2001-08-08") } ] }
+{ "id": 9959077, "id-copy": 9959077, "alias": "Josephine", "name": "JosephineLauffer", "user-since": datetime("2006-12-27T17:33:39.000"), "user-since-copy": datetime("2006-12-27T17:33:39.000"), "friend-ids": {{ 41423014, 33024139, 26147665, 14776436, 4726952, 12688804 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2001-03-26") } ] }
+{ "id": 9967888, "id-copy": 9967888, "alias": "Andrea", "name": "AndreaBerry", "user-since": datetime("2007-05-03T20:18:51.000"), "user-since-copy": datetime("2007-05-03T20:18:51.000"), "friend-ids": {{ 1106859, 38049440, 23056791, 16253206, 7727164, 19267641, 31798723, 30455164, 24738450, 15142413, 15111012, 3782070, 11502933, 44299958, 30277689, 3512757, 41960838, 7667284, 9192069, 12267931, 34901540, 20633036, 37186032, 1734718 }}, "employment": [ { "organization-name": "ganjalax", "start-date": date("2007-04-01"), "end-date": date("2011-09-07") } ] }
+{ "id": 9974485, "id-copy": 9974485, "alias": "Leo", "name": "LeoRawls", "user-since": datetime("2005-02-12T12:01:58.000"), "user-since-copy": datetime("2005-02-12T12:01:58.000"), "friend-ids": {{ 41189338, 33744557, 2485502, 8308490, 43237410 }}, "employment": [ { "organization-name": "itlab", "start-date": date("2002-11-05"), "end-date": date("2009-04-12") } ] }
+{ "id": 9996817, "id-copy": 9996817, "alias": "Vere", "name": "VereWilkerson", "user-since": datetime("2012-02-05T22:05:44.000"), "user-since-copy": datetime("2012-02-05T22:05:44.000"), "friend-ids": {{ 30010110, 31604568, 5741065, 29161468, 22429704, 16954129, 26525860, 1490181, 11444321, 24455724, 10411850, 39851031, 16059860, 32050795, 13116007, 12071588 }}, "employment": [ { "organization-name": "Ganjatax", "start-date": date("2004-11-04") } ] }
+{ "id": 10001410, "id-copy": 10001410, "alias": "Denzil", "name": "DenzilLedgerwood", "user-since": datetime("2006-12-24T10:56:58.000"), "user-since-copy": datetime("2006-12-24T10:56:58.000"), "friend-ids": {{ 25633920, 39748697, 3557647, 44396047, 25225495, 38723684, 5854330 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2000-08-14"), "end-date": date("2011-07-20") } ] }
+{ "id": 10017829, "id-copy": 10017829, "alias": "Adam", "name": "AdamTrovato", "user-since": datetime("2009-04-15T20:21:48.000"), "user-since-copy": datetime("2009-04-15T20:21:48.000"), "friend-ids": {{ 7572792, 20961281, 47727918, 25262896, 33740076, 14418354, 42807653, 34174665, 12459426, 28777106, 44409513, 39753872, 9172361, 36746114, 196755 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2007-09-25") } ] }
+{ "id": 10025086, "id-copy": 10025086, "alias": "Peggy", "name": "PeggyOlphert", "user-since": datetime("2009-06-24T16:14:48.000"), "user-since-copy": datetime("2009-06-24T16:14:48.000"), "friend-ids": {{ 13659719, 46045788, 35841713, 32392118, 24785179, 45483286, 47287227, 42691471, 7471992, 47671331, 25747076, 2368606, 34452743, 14570607, 31436760, 36423303, 31381129, 29414651, 10005587, 14082638, 13311890, 11592210, 1585557 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2008-07-20") } ] }
+{ "id": 10026061, "id-copy": 10026061, "alias": "Nonie", "name": "NonieChappel", "user-since": datetime("2007-06-22T10:06:38.000"), "user-since-copy": datetime("2007-06-22T10:06:38.000"), "friend-ids": {{ 38760716, 16809503, 6592849, 3736630, 32388289, 40487693, 27146403, 22621793, 35615399, 10839746, 693037, 25222841, 46448329, 40740448, 21652202, 30069817, 21957966 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2010-08-19"), "end-date": date("2010-08-17") } ] }
+{ "id": 10047001, "id-copy": 10047001, "alias": "Darcy", "name": "DarcyKava", "user-since": datetime("2012-02-25T17:16:18.000"), "user-since-copy": datetime("2012-02-25T17:16:18.000"), "friend-ids": {{ 15613341, 46557569, 20439965, 22442508, 32423739, 40757483, 36365324, 40706148, 12537361, 47741886, 24508947, 34168899, 10674474, 34285157, 28222068, 11113263 }}, "employment": [ { "organization-name": "kin-ron", "start-date": date("2008-01-27") } ] }
+{ "id": 10047373, "id-copy": 10047373, "alias": "Rexana", "name": "RexanaDennis", "user-since": datetime("2010-01-05T15:43:34.000"), "user-since-copy": datetime("2010-01-05T15:43:34.000"), "friend-ids": {{ 1594, 40130182 }}, "employment": [ { "organization-name": "freshdox", "start-date": date("2004-07-04"), "end-date": date("2007-12-28") } ] }
+{ "id": 10073002, "id-copy": 10073002, "alias": "Josefa", "name": "JosefaNewman", "user-since": datetime("2010-10-06T09:28:29.000"), "user-since-copy": datetime("2010-10-06T09:28:29.000"), "friend-ids": {{ 7549910, 7287709, 24063891, 41208589, 22325854, 16465930, 45067165, 42784968, 26414870, 16479308, 22681119, 40811475, 9603161, 23525416, 15131604, 4782290, 36997646, 35862360, 42008502, 438438, 25913601, 39300786, 15041382, 37410001 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2011-02-05"), "end-date": date("2011-10-24") } ] }
+{ "id": 10100707, "id-copy": 10100707, "alias": "Brittni", "name": "BrittniEaster", "user-since": datetime("2008-10-03T02:27:48.000"), "user-since-copy": datetime("2008-10-03T02:27:48.000"), "friend-ids": {{ 28725707, 8497950, 18892135, 1016149, 32023719, 34079976, 39582966, 15469248, 14059091, 6681733, 18398487, 41385960 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2006-04-21") } ] }
+{ "id": 10151953, "id-copy": 10151953, "alias": "Howard", "name": "HowardHoopengarner", "user-since": datetime("2006-07-23T01:43:57.000"), "user-since-copy": datetime("2006-07-23T01:43:57.000"), "friend-ids": {{ 32564548, 19333543, 27610653, 27936980, 7471201, 1353451, 30864511, 41582907, 22918030, 6011307, 21622284, 44695813, 34728110, 33062051, 29420834, 37472592, 3655974, 34618485, 21615748, 14107596, 15317302, 21805666, 4563480 }}, "employment": [ { "organization-name": "Doncare", "start-date": date("2012-06-08") } ] }
+{ "id": 10162495, "id-copy": 10162495, "alias": "Malina", "name": "MalinaTrout", "user-since": datetime("2006-12-19T12:12:55.000"), "user-since-copy": datetime("2006-12-19T12:12:55.000"), "friend-ids": {{ 40578475, 43374248, 7059820, 18838227, 45149295, 47680877, 11640348, 19081155, 9959453, 46807478, 45192583, 39333999, 4869981, 42888726, 32789666, 19653202 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2000-11-08") } ] }
+{ "id": 10166767, "id-copy": 10166767, "alias": "Leon", "name": "LeonWardle", "user-since": datetime("2008-05-19T07:05:45.000"), "user-since-copy": datetime("2008-05-19T07:05:45.000"), "friend-ids": {{ 41883510, 44504996, 36617462, 32609381, 11246739, 18717645, 32225763, 25136144, 18258339, 4951535, 40063362, 38810936, 1994155, 16613514, 25411748, 34221779, 44135463 }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2006-10-11") } ] }
+{ "id": 10173691, "id-copy": 10173691, "alias": "Elissa", "name": "ElissaWilliams", "user-since": datetime("2011-09-26T16:07:17.000"), "user-since-copy": datetime("2011-09-26T16:07:17.000"), "friend-ids": {{ 2526422 }}, "employment": [ { "organization-name": "Whitemedia", "start-date": date("2001-07-22") } ] }
+{ "id": 10190329, "id-copy": 10190329, "alias": "Rachyl", "name": "RachylAdams", "user-since": datetime("2005-08-25T14:09:48.000"), "user-since-copy": datetime("2005-08-25T14:09:48.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2002-11-17") } ] }
+{ "id": 10197700, "id-copy": 10197700, "alias": "Frederica", "name": "FredericaCherry", "user-since": datetime("2006-04-10T01:23:53.000"), "user-since-copy": datetime("2006-04-10T01:23:53.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-01-01"), "end-date": date("2009-07-14") } ] }
+{ "id": 10212385, "id-copy": 10212385, "alias": "Alice", "name": "AliceJones", "user-since": datetime("2009-05-16T16:08:03.000"), "user-since-copy": datetime("2009-05-16T16:08:03.000"), "friend-ids": {{ 4158604, 3204211, 21491737, 39619715, 9750334 }}, "employment": [ { "organization-name": "Viatechi", "start-date": date("2012-04-19") } ] }
+{ "id": 10241767, "id-copy": 10241767, "alias": "Lewin", "name": "LewinBurkett", "user-since": datetime("2008-03-24T21:09:05.000"), "user-since-copy": datetime("2008-03-24T21:09:05.000"), "friend-ids": {{ 5503, 32598090, 36950887, 22362781, 16089120, 30220805, 6197105, 44773004, 17924848, 36033966, 41338779, 38304288, 18528858, 6384026, 46633327, 18024168, 13983021, 7158391, 31922078, 1082072 }}, "employment": [ { "organization-name": "Salthex", "start-date": date("2004-02-17") } ] }
+{ "id": 10252147, "id-copy": 10252147, "alias": "Concha", "name": "ConchaMckinnon", "user-since": datetime("2009-12-21T03:27:35.000"), "user-since-copy": datetime("2009-12-21T03:27:35.000"), "friend-ids": {{ 8837048, 7758233, 2108777, 31062874, 34698247, 33766563, 10653492, 25103733, 24629375, 38758275, 37539109, 47252638, 41559516, 41883197, 9608881, 26501553, 39435548, 43307321, 46890131, 29908109 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2011-05-09") } ] }
+{ "id": 10261300, "id-copy": 10261300, "alias": "Nick", "name": "NickRohtin", "user-since": datetime("2007-01-24T17:56:52.000"), "user-since-copy": datetime("2007-01-24T17:56:52.000"), "friend-ids": {{ 37649902 }}, "employment": [ { "organization-name": "Solophase", "start-date": date("2004-03-06"), "end-date": date("2007-05-20") } ] }
+{ "id": 10270597, "id-copy": 10270597, "alias": "Ava", "name": "AvaTanner", "user-since": datetime("2010-04-23T11:49:39.000"), "user-since-copy": datetime("2010-04-23T11:49:39.000"), "friend-ids": {{ 38894360, 9403074, 25855965, 36511208, 4947767, 10318201, 3532083, 28684767, 22730535, 17994309, 21209113, 14980333, 5611975, 31951870, 16697364, 5033131, 13637894, 18107216, 9769275, 25479923, 15320268, 28897820, 22865104 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2011-04-12"), "end-date": date("2011-09-07") } ] }
+{ "id": 10272571, "id-copy": 10272571, "alias": "Jarrett", "name": "JarrettGoldvogel", "user-since": datetime("2010-04-28T23:24:22.000"), "user-since-copy": datetime("2010-04-28T23:24:22.000"), "friend-ids": {{ 47024505, 36647273, 32152567, 28239957, 11739703, 47515825, 17408763, 41224279, 41487670, 43339913 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2004-04-06"), "end-date": date("2010-02-14") } ] }
+{ "id": 10277731, "id-copy": 10277731, "alias": "Gallagher", "name": "GallagherMagor", "user-since": datetime("2007-07-02T07:37:02.000"), "user-since-copy": datetime("2007-07-02T07:37:02.000"), "friend-ids": {{ 22730683, 9352614, 42748868, 24014877, 21749502, 30751403, 41768964, 13317192, 31877814, 35318552, 26843471, 21232937, 11268529, 21902785 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-09-03") } ] }
+{ "id": 10278550, "id-copy": 10278550, "alias": "Parker", "name": "ParkerWinton", "user-since": datetime("2008-03-02T18:54:35.000"), "user-since-copy": datetime("2008-03-02T18:54:35.000"), "friend-ids": {{ 281420, 13481584, 25554653, 2922131, 15313837, 33567564, 20182917, 20143660, 35884326, 22038516, 183180 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2002-12-16"), "end-date": date("2010-08-04") } ] }
+{ "id": 10284583, "id-copy": 10284583, "alias": "Salal", "name": "SalalButterfill", "user-since": datetime("2011-02-05T13:39:36.000"), "user-since-copy": datetime("2011-02-05T13:39:36.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2008-08-10"), "end-date": date("2011-05-02") } ] }
+{ "id": 10298530, "id-copy": 10298530, "alias": "Natalee", "name": "NataleeBell", "user-since": datetime("2010-09-07T14:14:59.000"), "user-since-copy": datetime("2010-09-07T14:14:59.000"), "friend-ids": {{ 36077399, 47946678, 4189158, 42122618, 14179077, 26433248, 25903252, 23116624, 33542934, 1071320, 31914369, 28408518, 40811454, 19212473, 25057330, 42758915 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-02-17") } ] }
+{ "id": 10299298, "id-copy": 10299298, "alias": "Belinda", "name": "BelindaRockwell", "user-since": datetime("2005-03-08T07:13:05.000"), "user-since-copy": datetime("2005-03-08T07:13:05.000"), "friend-ids": {{ 31301282, 34653696, 23868758 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2008-08-19") } ] }
+{ "id": 10307155, "id-copy": 10307155, "alias": "Rhetta", "name": "RhettaGarneys", "user-since": datetime("2008-03-17T00:33:40.000"), "user-since-copy": datetime("2008-03-17T00:33:40.000"), "friend-ids": {{ 5658375, 40536479, 47961112, 28517297, 26103231, 32434876, 44285321, 44471686 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2006-06-07"), "end-date": date("2010-10-03") } ] }
+{ "id": 10390954, "id-copy": 10390954, "alias": "Lucinda", "name": "LucindaWatson", "user-since": datetime("2006-11-16T21:20:41.000"), "user-since-copy": datetime("2006-11-16T21:20:41.000"), "friend-ids": {{ 36017573, 9298650, 16054222, 21985420, 23378246, 30163820, 20942039, 28917630, 20851877, 41794807, 45887537, 39768986, 42476881, 5070921, 29487760, 24953551, 32065985, 16342096, 41522555, 41923127, 34675252, 10040601, 32604114, 23852658 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2009-03-14") } ] }
+{ "id": 10397017, "id-copy": 10397017, "alias": "Holly", "name": "HollyHatch", "user-since": datetime("2006-04-12T03:26:11.000"), "user-since-copy": datetime("2006-04-12T03:26:11.000"), "friend-ids": {{ 1504006, 21411501, 20934982, 24019384, 8634101, 25659178, 16581112, 2481631, 15544800 }}, "employment": [ { "organization-name": "linedexon", "start-date": date("2000-12-04") } ] }
+{ "id": 10405423, "id-copy": 10405423, "alias": "Pauletta", "name": "PaulettaGuess", "user-since": datetime("2007-06-11T02:54:36.000"), "user-since-copy": datetime("2007-06-11T02:54:36.000"), "friend-ids": {{ 14845791, 24263161, 2648994, 30766767, 10127359, 20706390 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2002-10-27") } ] }
+{ "id": 10412287, "id-copy": 10412287, "alias": "Wren", "name": "WrenElizabeth", "user-since": datetime("2009-06-25T07:26:48.000"), "user-since-copy": datetime("2009-06-25T07:26:48.000"), "friend-ids": {{ 23487913, 35496582, 14824955, 5998721, 10925419, 38937432, 6285652 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2012-07-20"), "end-date": date("2012-07-12") } ] }
+{ "id": 10464121, "id-copy": 10464121, "alias": "Enriqueta", "name": "EnriquetaHincken", "user-since": datetime("2005-11-19T09:43:20.000"), "user-since-copy": datetime("2005-11-19T09:43:20.000"), "friend-ids": {{ 31238269, 29421316, 14426443, 30128291, 9926275, 33523504, 19113054, 402505, 12662005, 36090974, 8733776, 18706660, 14174144, 46009221, 17906304, 41780430, 21807110, 22521282, 21492740, 34033053, 16784027, 11948555 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2008-09-23") } ] }
+{ "id": 10473718, "id-copy": 10473718, "alias": "Elissa", "name": "ElissaStainforth", "user-since": datetime("2007-06-20T07:46:54.000"), "user-since-copy": datetime("2007-06-20T07:46:54.000"), "friend-ids": {{ 1645948, 612724, 46091510, 32750261, 40622752, 10190250, 42030152, 28645649, 27513961 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2003-06-21"), "end-date": date("2011-09-05") } ] }
+{ "id": 10474273, "id-copy": 10474273, "alias": "Juliana", "name": "JulianaLing", "user-since": datetime("2005-05-04T20:58:12.000"), "user-since-copy": datetime("2005-05-04T20:58:12.000"), "friend-ids": {{ 8881381, 34113161, 15553599, 40081858, 12450920, 42147178, 568875, 11891228, 13309462, 39127120, 34765111, 19162279, 29505162, 891909, 33485893, 25658561, 36146447, 37027867, 39396759 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2008-03-03") } ] }
+{ "id": 10478512, "id-copy": 10478512, "alias": "Remona", "name": "RemonaPittman", "user-since": datetime("2007-06-19T12:20:07.000"), "user-since-copy": datetime("2007-06-19T12:20:07.000"), "friend-ids": {{ 12750727 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2001-02-02") } ] }
+{ "id": 10514428, "id-copy": 10514428, "alias": "Eliseo", "name": "EliseoHoffhants", "user-since": datetime("2012-08-24T08:40:51.000"), "user-since-copy": datetime("2012-08-24T08:40:51.000"), "friend-ids": {{ 45751891, 26026786, 24531389, 26239368, 34021241 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2010-03-01"), "end-date": date("2010-08-02") } ] }
+{ "id": 10532791, "id-copy": 10532791, "alias": "Byrne", "name": "ByrneLafortune", "user-since": datetime("2010-03-13T13:21:05.000"), "user-since-copy": datetime("2010-03-13T13:21:05.000"), "friend-ids": {{ 35020297, 40002497, 16857157, 47134232, 37864297, 31029450, 36968713, 36672267, 15503365, 43888732, 29395734, 35372186, 19093208, 21774877, 9785166, 22833579 }}, "employment": [ { "organization-name": "silfind", "start-date": date("2003-12-21") } ] }
+{ "id": 10577128, "id-copy": 10577128, "alias": "Charnette", "name": "CharnettePyle", "user-since": datetime("2008-08-20T21:25:22.000"), "user-since-copy": datetime("2008-08-20T21:25:22.000"), "friend-ids": {{ 30078840, 16315930, 12006652, 31984600, 12053254, 41773411, 43318427, 21592935, 40739515, 30608076, 21922300, 5687640 }}, "employment": [ { "organization-name": "Alphadax", "start-date": date("2001-11-25"), "end-date": date("2002-08-12") } ] }
+{ "id": 10594069, "id-copy": 10594069, "alias": "Clinton", "name": "ClintonMiller", "user-since": datetime("2007-03-12T05:19:19.000"), "user-since-copy": datetime("2007-03-12T05:19:19.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "itlab", "start-date": date("2010-06-06") } ] }
+{ "id": 10607341, "id-copy": 10607341, "alias": "Evander", "name": "EvanderPycroft", "user-since": datetime("2005-08-09T23:36:46.000"), "user-since-copy": datetime("2005-08-09T23:36:46.000"), "friend-ids": {{ 46200658, 38004155 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2004-06-13") } ] }
+{ "id": 10613617, "id-copy": 10613617, "alias": "Jeanie", "name": "JeanieEiford", "user-since": datetime("2007-02-09T12:16:09.000"), "user-since-copy": datetime("2007-02-09T12:16:09.000"), "friend-ids": {{ 24843944, 3651507, 25077638, 18662161, 46723847, 31558857, 11235682, 15640606, 31889112, 45342233, 25865191, 1530020, 39187188, 4939030, 19220487, 19619126, 25284665, 1206869, 40740763 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2001-07-22") } ] }
+{ "id": 10637896, "id-copy": 10637896, "alias": "Hiram", "name": "HiramRohtin", "user-since": datetime("2006-11-05T14:44:03.000"), "user-since-copy": datetime("2006-11-05T14:44:03.000"), "friend-ids": {{ 1387663, 11367203, 24828245 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2012-02-18"), "end-date": date("2012-02-12") } ] }
+{ "id": 10640851, "id-copy": 10640851, "alias": "Tabitha", "name": "TabithaWhitten", "user-since": datetime("2010-01-28T14:25:58.000"), "user-since-copy": datetime("2010-01-28T14:25:58.000"), "friend-ids": {{ 42792549, 5330514, 24582133, 43384590, 38083439, 31221232, 18064537, 21736064, 7919520, 18998284, 20165148, 28492287, 21987533, 23638155 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2006-06-18"), "end-date": date("2007-07-20") } ] }
+{ "id": 10655089, "id-copy": 10655089, "alias": "Quinn", "name": "QuinnHays", "user-since": datetime("2009-11-25T04:42:39.000"), "user-since-copy": datetime("2009-11-25T04:42:39.000"), "friend-ids": {{ 17385636, 24378500, 37614592, 32315940, 18046144, 45823175, 29709981, 28423306, 23783823, 10623867, 27782698 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2012-06-09") } ] }
+{ "id": 10714447, "id-copy": 10714447, "alias": "Leone", "name": "LeoneCoughenour", "user-since": datetime("2012-06-13T05:05:11.000"), "user-since-copy": datetime("2012-06-13T05:05:11.000"), "friend-ids": {{ 13098839, 21185838, 26566436, 37464340, 8086775, 37143068, 40377316, 39371296 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2005-04-16") } ] }
+{ "id": 10729942, "id-copy": 10729942, "alias": "Valda", "name": "ValdaFea", "user-since": datetime("2005-07-16T09:31:53.000"), "user-since-copy": datetime("2005-07-16T09:31:53.000"), "friend-ids": {{ 20145015, 42027050, 38819467, 3406065, 4977132, 47154979, 23685067 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2008-10-12") } ] }
+{ "id": 10754107, "id-copy": 10754107, "alias": "Jeri", "name": "JeriSanner", "user-since": datetime("2009-11-15T23:47:08.000"), "user-since-copy": datetime("2009-11-15T23:47:08.000"), "friend-ids": {{ 19868241, 28778419, 16761189, 28588239, 1592484, 41256056, 36550491, 10555328, 3086612, 37431116, 45976270 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2004-11-06") } ] }
+{ "id": 10760020, "id-copy": 10760020, "alias": "Emeline", "name": "EmelineCowher", "user-since": datetime("2006-03-11T07:02:10.000"), "user-since-copy": datetime("2006-03-11T07:02:10.000"), "friend-ids": {{ 2652618, 22247716, 39487944, 16288504, 8109009, 34390947, 2041892, 27800644, 5979423, 12674908 }}, "employment": [ { "organization-name": "sonstreet", "start-date": date("2007-12-26"), "end-date": date("2007-09-04") } ] }
+{ "id": 10766221, "id-copy": 10766221, "alias": "Rosalyn", "name": "RosalynBaxter", "user-since": datetime("2009-04-16T15:46:54.000"), "user-since-copy": datetime("2009-04-16T15:46:54.000"), "friend-ids": {{ 43759575, 1264811, 9906031, 21579594, 45786210, 14876191, 10711745, 25134652, 25426644, 29987806, 1953812, 29568099, 38860088, 7073296, 13746927, 11395655, 36208297, 25317651, 21356968 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2000-07-04") } ] }
+{ "id": 10800157, "id-copy": 10800157, "alias": "Tiara", "name": "TiaraFuhrer", "user-since": datetime("2010-05-24T21:52:36.000"), "user-since-copy": datetime("2010-05-24T21:52:36.000"), "friend-ids": {{ 34031723 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2003-03-18"), "end-date": date("2005-09-20") } ] }
+{ "id": 10824484, "id-copy": 10824484, "alias": "Linda", "name": "LindaStanfield", "user-since": datetime("2009-03-03T12:54:55.000"), "user-since-copy": datetime("2009-03-03T12:54:55.000"), "friend-ids": {{ 39164563, 20321780, 19901289, 37969494, 15051354, 42576590, 14550253, 33649901, 6008727, 17749643, 7792769, 18652053, 8565400, 43899372, 7433016, 42506713 }}, "employment": [ { "organization-name": "Techitechi", "start-date": date("2012-03-21") } ] }
+{ "id": 10835521, "id-copy": 10835521, "alias": "Margeret", "name": "MargeretEve", "user-since": datetime("2010-02-13T16:16:55.000"), "user-since-copy": datetime("2010-02-13T16:16:55.000"), "friend-ids": {{ 40363275, 44184724, 42855751, 10492711, 561147, 45516609, 38567828, 9695088, 40235757 }}, "employment": [ { "organization-name": "Kanelectrics", "start-date": date("2012-06-08"), "end-date": date("2012-06-27") } ] }
+{ "id": 10851595, "id-copy": 10851595, "alias": "Juan", "name": "JuanSoames", "user-since": datetime("2006-02-16T05:34:28.000"), "user-since-copy": datetime("2006-02-16T05:34:28.000"), "friend-ids": {{ 34589906, 8801547, 38357163, 39649840, 18254469, 38911658, 17825991, 26015024, 29742264, 13155934, 28459597, 34931012, 20376527 }}, "employment": [ { "organization-name": "Newhotplus", "start-date": date("2008-11-17"), "end-date": date("2009-01-13") } ] }
+{ "id": 10853926, "id-copy": 10853926, "alias": "Kennard", "name": "KennardGarland", "user-since": datetime("2007-11-28T20:40:40.000"), "user-since-copy": datetime("2007-11-28T20:40:40.000"), "friend-ids": {{ 47687855, 28575858 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2005-07-17") } ] }
+{ "id": 10861183, "id-copy": 10861183, "alias": "Zilla", "name": "ZillaOneal", "user-since": datetime("2008-03-12T23:37:18.000"), "user-since-copy": datetime("2008-03-12T23:37:18.000"), "friend-ids": {{ 26262188, 17172669, 43068853, 47767064, 34552281, 33602720, 35448839, 6347557, 11913432, 45186875, 10451537, 46881437, 27965706 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-09-03"), "end-date": date("2009-07-22") } ] }
+{ "id": 10867444, "id-copy": 10867444, "alias": "Tetty", "name": "TettyZundel", "user-since": datetime("2012-07-26T17:54:45.000"), "user-since-copy": datetime("2012-07-26T17:54:45.000"), "friend-ids": {{ 17830961, 13154371, 12005619, 15279158, 15766172, 3071670, 4314512, 29378453, 33264674, 32657723, 37875054, 6208013, 23310809, 11994927, 9787690, 25069760, 11104605, 44517542, 45829337, 26593992 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2001-09-25") } ] }
+{ "id": 10878898, "id-copy": 10878898, "alias": "Webster", "name": "WebsterCarr", "user-since": datetime("2006-07-28T21:17:56.000"), "user-since-copy": datetime("2006-07-28T21:17:56.000"), "friend-ids": {{ 11755002, 37594815, 4340697, 27424145, 22193377, 31509516, 31372689, 47386546, 30347891, 4070454, 18531894, 28306285, 14110568, 17830332 }}, "employment": [ { "organization-name": "Medflex", "start-date": date("2002-03-12") } ] }
+{ "id": 10882393, "id-copy": 10882393, "alias": "Erica", "name": "EricaHynes", "user-since": datetime("2006-09-16T16:39:05.000"), "user-since-copy": datetime("2006-09-16T16:39:05.000"), "friend-ids": {{ 23491370, 13390922, 19685128, 47763240, 9493285, 10823383, 45076071, 14858340, 12545499, 40367152, 2150593, 45723007, 21362425, 25435409, 776198, 8016739, 21691528, 21036410, 3131225, 20078710, 28405287, 15599245, 39126345, 36208574 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2012-05-14"), "end-date": date("2012-05-22") } ] }
+{ "id": 10883062, "id-copy": 10883062, "alias": "Lamar", "name": "LamarFelbrigge", "user-since": datetime("2005-02-12T03:19:28.000"), "user-since-copy": datetime("2005-02-12T03:19:28.000"), "friend-ids": {{ 26304238, 21048260, 26614197, 41153844, 17163890, 27772117, 26679939, 22001103, 46907785, 21321841, 46215643, 31285577, 14997749, 46997910, 44367495, 13858871, 20405288, 36784906, 33752927, 30769058, 43188289, 34006518, 23022696 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2012-06-16") } ] }
+{ "id": 10884241, "id-copy": 10884241, "alias": "Anamaria", "name": "AnamariaMoon", "user-since": datetime("2005-03-28T11:38:17.000"), "user-since-copy": datetime("2005-03-28T11:38:17.000"), "friend-ids": {{ 21445295, 42154978, 41608378, 3406391, 26013137, 45437958, 22377352, 26150886, 25726611, 31834547, 17506680, 22932063, 16700407, 22939810, 152978, 45307280, 42212660, 30124140, 9494103, 35217706, 41538534, 26586744, 26538590 }}, "employment": [ { "organization-name": "Newphase", "start-date": date("2011-09-10"), "end-date": date("2011-02-06") } ] }
+{ "id": 10889389, "id-copy": 10889389, "alias": "Roselyn", "name": "RoselynLlora", "user-since": datetime("2012-03-25T15:21:06.000"), "user-since-copy": datetime("2012-03-25T15:21:06.000"), "friend-ids": {{ 38921827, 1378686, 22284385, 17464785, 16302500, 47598267, 25016712, 11151378, 16381115, 16371401 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2005-12-02") } ] }
+{ "id": 10892830, "id-copy": 10892830, "alias": "Audrie", "name": "AudrieHawkins", "user-since": datetime("2011-11-19T00:51:33.000"), "user-since-copy": datetime("2011-11-19T00:51:33.000"), "friend-ids": {{ 8838768, 18321840, 16958648, 27000957, 19090823, 11772058, 18573458, 24662627, 27415154, 4998699, 44522833, 44994903, 6514403, 43833807, 38512495, 6964420, 11334788, 14298721, 25316052, 11632302 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2005-02-04") } ] }
+{ "id": 10901047, "id-copy": 10901047, "alias": "Salvador", "name": "SalvadorBynum", "user-since": datetime("2012-01-13T02:30:17.000"), "user-since-copy": datetime("2012-01-13T02:30:17.000"), "friend-ids": {{ 29122263, 27975257, 7988516, 9270552, 17837898, 42339445, 46097101, 32303800, 17233223, 10656090, 36709955, 17535336, 27157992, 30360627, 15304415, 28922979, 27243261, 9307382, 43171015, 31593421, 21246902, 40452339, 25735551, 23716187 }}, "employment": [ { "organization-name": "Striptaxon", "start-date": date("2010-11-27") } ] }
+{ "id": 10902049, "id-copy": 10902049, "alias": "Fae", "name": "FaeRing", "user-since": datetime("2008-06-15T12:54:57.000"), "user-since-copy": datetime("2008-06-15T12:54:57.000"), "friend-ids": {{ 2667467, 46445373, 11696423, 42003744, 47667382, 34088774, 4279683, 29934858, 21213543, 44195034, 38786294, 14946433, 38805114, 9972575, 3309290, 5324029, 32663319, 20577589, 9110909, 27272396, 47622938 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2007-11-15") } ] }
+{ "id": 10902649, "id-copy": 10902649, "alias": "Makenzie", "name": "MakenzieWerner", "user-since": datetime("2005-12-20T00:23:45.000"), "user-since-copy": datetime("2005-12-20T00:23:45.000"), "friend-ids": {{ 9011568, 38173487, 45649445, 11873586 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2000-01-06"), "end-date": date("2009-03-24") } ] }
+{ "id": 10907953, "id-copy": 10907953, "alias": "Wymond", "name": "WymondSnyder", "user-since": datetime("2006-02-25T03:33:22.000"), "user-since-copy": datetime("2006-02-25T03:33:22.000"), "friend-ids": {{ 16280602, 26846293, 39235173, 4686537, 30457440, 23649561, 34348317, 28099021, 1622222, 24073647, 4742953, 14925763, 17026705, 46257859, 22592244 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2012-07-22") } ] }
+{ "id": 10912441, "id-copy": 10912441, "alias": "Janae", "name": "JanaeErschoff", "user-since": datetime("2009-04-17T09:26:36.000"), "user-since-copy": datetime("2009-04-17T09:26:36.000"), "friend-ids": {{ 11445243, 13239218, 2302326, 37976140, 45374131, 14136536, 2051767, 7824391, 42808044, 41836900, 35275542, 33493951, 8497237, 42991362, 24049395, 32159562, 23378256, 4723574, 47010157 }}, "employment": [ { "organization-name": "zoomplus", "start-date": date("2012-04-20"), "end-date": date("2012-04-04") } ] }
+{ "id": 10931563, "id-copy": 10931563, "alias": "Laraine", "name": "LaraineCountryman", "user-since": datetime("2012-03-17T17:06:59.000"), "user-since-copy": datetime("2012-03-17T17:06:59.000"), "friend-ids": {{ 17266368, 75990, 37678426, 43207424, 37434492, 26338447, 33450799, 5401110, 44962643, 5514847 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2008-09-08") } ] }
+{ "id": 10931647, "id-copy": 10931647, "alias": "Bertina", "name": "BertinaStraub", "user-since": datetime("2011-05-25T19:21:43.000"), "user-since-copy": datetime("2011-05-25T19:21:43.000"), "friend-ids": {{ 12208030, 43810737, 43870253, 20720324, 7601394, 22266404, 21210273, 10076577, 25757258, 1909792, 26189079, 37799329, 24923233, 31687015, 37580896, 44906728, 46928405, 10679805, 14520239, 1690125, 37459202, 36684838, 30982356 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2011-02-17"), "end-date": date("2011-06-20") } ] }
+{ "id": 10933138, "id-copy": 10933138, "alias": "Gwendoline", "name": "GwendolineCypret", "user-since": datetime("2006-04-10T03:55:29.000"), "user-since-copy": datetime("2006-04-10T03:55:29.000"), "friend-ids": {{ 9996028, 18756914, 15079751, 34129343, 44558538, 25387070, 44250368, 37560291, 5178625, 10379959, 39639296, 8784216, 13429736, 22802431, 11154064, 2453387, 24748342, 34032462, 32570963, 4861587, 19421488, 10848442 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2008-12-24"), "end-date": date("2010-05-20") } ] }
+{ "id": 10943026, "id-copy": 10943026, "alias": "Raeburn", "name": "RaeburnAllshouse", "user-since": datetime("2008-08-26T04:51:27.000"), "user-since-copy": datetime("2008-08-26T04:51:27.000"), "friend-ids": {{ 6784667, 1651647, 45052591, 21630976, 20049039, 37839759, 38694475, 23340828, 8641638, 4568782, 35684305, 20895609, 2213341, 8612199, 14260231, 8621325, 21926952, 41656664, 45180955 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2007-09-28") } ] }
+{ "id": 10943104, "id-copy": 10943104, "alias": "Prudence", "name": "PrudencePriebe", "user-since": datetime("2006-04-27T21:00:43.000"), "user-since-copy": datetime("2006-04-27T21:00:43.000"), "friend-ids": {{ 43633941, 38710166, 34456560, 11324015, 21000755, 23356715, 21056830, 27295754 }}, "employment": [ { "organization-name": "Ontotanin", "start-date": date("2012-08-30") } ] }
+{ "id": 10951918, "id-copy": 10951918, "alias": "Doran", "name": "DoranBell", "user-since": datetime("2005-08-22T14:07:50.000"), "user-since-copy": datetime("2005-08-22T14:07:50.000"), "friend-ids": {{ 6952033, 22223086, 5858716, 35128893, 22115927, 5821006, 16264772, 4151991, 40384467, 19801357, 42871024, 46855275, 35241988, 17208259, 47420533, 25182232, 14247140, 19664015, 33132502, 47813026, 12819081, 29321093, 42851957, 30756972 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2012-01-22") } ] }
+{ "id": 10962466, "id-copy": 10962466, "alias": "Zoey", "name": "ZoeyCady", "user-since": datetime("2012-07-15T20:02:23.000"), "user-since-copy": datetime("2012-07-15T20:02:23.000"), "friend-ids": {{ 12726157, 268799, 29381478, 15699674, 1150948, 8000369, 41608951, 11382366, 770690, 25889785, 37815043, 40437016, 38679636, 32956275, 34853801 }}, "employment": [ { "organization-name": "Villa-dox", "start-date": date("2012-05-08") } ] }
+{ "id": 10972447, "id-copy": 10972447, "alias": "Loretta", "name": "LorettaBriggs", "user-since": datetime("2005-07-01T10:25:33.000"), "user-since-copy": datetime("2005-07-01T10:25:33.000"), "friend-ids": {{ 6898813, 6606991, 14092255, 9865734, 23960698, 47354873, 19345256 }}, "employment": [ { "organization-name": "Sanjodax", "start-date": date("2005-06-02") } ] }
+{ "id": 10993267, "id-copy": 10993267, "alias": "Esmund", "name": "EsmundDunkle", "user-since": datetime("2005-11-16T21:18:20.000"), "user-since-copy": datetime("2005-11-16T21:18:20.000"), "friend-ids": {{ 1277480, 11393524, 32336542, 41857626, 7807437, 25280677, 17518254, 7723810, 18423045, 11937236, 21507800 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2006-12-03"), "end-date": date("2011-11-26") } ] }
+{ "id": 11004067, "id-copy": 11004067, "alias": "Vickie", "name": "VickieRosenstiehl", "user-since": datetime("2012-04-15T02:37:43.000"), "user-since-copy": datetime("2012-04-15T02:37:43.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Inchex", "start-date": date("2012-08-01"), "end-date": date("2012-08-06") } ] }
+{ "id": 11007700, "id-copy": 11007700, "alias": "Elly", "name": "EllyWard", "user-since": datetime("2009-04-20T08:46:09.000"), "user-since-copy": datetime("2009-04-20T08:46:09.000"), "friend-ids": {{ 9712756, 6523354 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2007-12-07"), "end-date": date("2007-07-27") } ] }
+{ "id": 11015908, "id-copy": 11015908, "alias": "Giuseppe", "name": "GiuseppeWard", "user-since": datetime("2008-09-14T16:37:40.000"), "user-since-copy": datetime("2008-09-14T16:37:40.000"), "friend-ids": {{ 9972151, 40271551, 46207899, 29987388, 19876511, 47546614, 17051350, 1579198, 2151480, 26507940, 18177808, 25866392, 40253780 }}, "employment": [ { "organization-name": "Scotcity", "start-date": date("2008-02-24") } ] }
+{ "id": 11032186, "id-copy": 11032186, "alias": "Tabby", "name": "TabbySealis", "user-since": datetime("2007-12-10T21:45:46.000"), "user-since-copy": datetime("2007-12-10T21:45:46.000"), "friend-ids": {{ 8190058, 5089537, 18167034, 19113649, 38817127, 7644664, 12427817, 39615196, 11451538, 27188211, 27425673, 33084974, 10726858, 40696324, 41487982, 42282364, 17084607, 41647211, 40268195, 29075837, 41802984, 9719771, 29747340, 28103359 }}, "employment": [ { "organization-name": "Redelectronics", "start-date": date("2008-07-13"), "end-date": date("2010-12-04") } ] }
+{ "id": 11032477, "id-copy": 11032477, "alias": "Wilmer", "name": "WilmerWortman", "user-since": datetime("2007-06-03T19:27:24.000"), "user-since-copy": datetime("2007-06-03T19:27:24.000"), "friend-ids": {{ 18685187, 2599612, 27305395, 20825021, 20327586, 21301262, 29222955, 20377452, 11211553, 37446807, 20533832, 10098143, 43828837, 37254072, 46029810, 16401947, 7537056, 41738273, 4665729, 27400110, 146251, 14185116 }}, "employment": [ { "organization-name": "Transhigh", "start-date": date("2006-03-17"), "end-date": date("2011-08-03") } ] }
+{ "id": 11039716, "id-copy": 11039716, "alias": "Piedad", "name": "PiedadHowe", "user-since": datetime("2011-02-23T17:18:37.000"), "user-since-copy": datetime("2011-02-23T17:18:37.000"), "friend-ids": {{ 13323345 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2009-03-26"), "end-date": date("2009-06-17") } ] }
+{ "id": 11051014, "id-copy": 11051014, "alias": "Tad", "name": "TadWilson", "user-since": datetime("2011-05-05T14:48:34.000"), "user-since-copy": datetime("2011-05-05T14:48:34.000"), "friend-ids": {{ 42862096, 17517240, 8058482, 9927174, 4207109, 4924943, 11531213 }}, "employment": [ { "organization-name": "Canline", "start-date": date("2005-01-25"), "end-date": date("2010-11-14") } ] }
+{ "id": 11081539, "id-copy": 11081539, "alias": "Haidee", "name": "HaideeStyle", "user-since": datetime("2012-06-13T11:37:34.000"), "user-since-copy": datetime("2012-06-13T11:37:34.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2001-03-05"), "end-date": date("2003-11-17") } ] }
+{ "id": 11090788, "id-copy": 11090788, "alias": "Randy", "name": "RandyClose", "user-since": datetime("2005-07-26T19:29:20.000"), "user-since-copy": datetime("2005-07-26T19:29:20.000"), "friend-ids": {{ 43392502, 7581874, 13279708, 16989391, 32340594, 7048512, 33084049, 16279611, 21735714, 23485799, 18185370, 43945382, 41653020, 13517043, 35395274, 24133848, 15355027, 4752815, 15007500, 25733540, 2114558, 37909789, 2805493, 16521087 }}, "employment": [ { "organization-name": "Voltlane", "start-date": date("2004-09-14") } ] }
+{ "id": 11092324, "id-copy": 11092324, "alias": "Paul", "name": "PaulOneal", "user-since": datetime("2006-11-20T10:50:19.000"), "user-since-copy": datetime("2006-11-20T10:50:19.000"), "friend-ids": {{ 44707820, 20249424, 18862268, 32895394, 29899430 }}, "employment": [ { "organization-name": "Vivaace", "start-date": date("2003-01-06") } ] }
+{ "id": 11097556, "id-copy": 11097556, "alias": "Tia", "name": "TiaHair", "user-since": datetime("2010-10-28T01:21:36.000"), "user-since-copy": datetime("2010-10-28T01:21:36.000"), "friend-ids": {{ 19746022, 42650092, 45679457, 43873545, 5490025, 42900988, 32855768, 20717716, 15007194, 23035301, 24322095, 27796211, 27751858, 4726224, 5570083, 18421959, 28424121, 22311092, 13781420, 18215783, 19934706, 18408890, 24792739, 4022527 }}, "employment": [ { "organization-name": "Trustbam", "start-date": date("2003-04-03") } ] }
+{ "id": 11103856, "id-copy": 11103856, "alias": "Dennise", "name": "DenniseGarland", "user-since": datetime("2008-10-19T11:09:14.000"), "user-since-copy": datetime("2008-10-19T11:09:14.000"), "friend-ids": {{ 2613052, 4777379, 29911213, 30822813, 44182985, 803163, 32630608, 7433428, 43625503, 19274272, 20950244, 21434389, 44059623, 40416129, 47937344, 12392360 }}, "employment": [ { "organization-name": "whitestreet", "start-date": date("2005-04-10"), "end-date": date("2005-07-26") } ] }
+{ "id": 11113168, "id-copy": 11113168, "alias": "Daphne", "name": "DaphneHindman", "user-since": datetime("2011-11-09T02:55:42.000"), "user-since-copy": datetime("2011-11-09T02:55:42.000"), "friend-ids": {{ 194785, 11696942, 23072861, 37052204, 17574763, 14099428, 44155581 }}, "employment": [ { "organization-name": "Basecone", "start-date": date("2002-07-16"), "end-date": date("2006-11-08") } ] }
+{ "id": 11140483, "id-copy": 11140483, "alias": "Nena", "name": "NenaBullard", "user-since": datetime("2008-02-23T10:24:08.000"), "user-since-copy": datetime("2008-02-23T10:24:08.000"), "friend-ids": {{ 26438400, 45201681, 12155417, 43414633, 14267296, 40906639, 8768744, 46840439, 43848021, 24521652, 41247005, 44999926, 13062334, 47731182 }}, "employment": [ { "organization-name": "Sancone", "start-date": date("2001-05-16") } ] }
+{ "id": 11152162, "id-copy": 11152162, "alias": "Tennille", "name": "TennilleGongaware", "user-since": datetime("2008-12-22T17:22:19.000"), "user-since-copy": datetime("2008-12-22T17:22:19.000"), "friend-ids": {{ 38167013, 48016045, 45757020, 26256748, 14740496, 36818162, 43284365, 29637839, 30820213, 535748, 31611626 }}, "employment": [ { "organization-name": "Newcom", "start-date": date("2005-07-27") } ] }
+{ "id": 11162977, "id-copy": 11162977, "alias": "Orson", "name": "OrsonFlick", "user-since": datetime("2010-02-17T21:05:53.000"), "user-since-copy": datetime("2010-02-17T21:05:53.000"), "friend-ids": {{ 12213318, 19062680, 20035734, 5154338, 24649936, 30379574, 38611249, 36143038, 13393939, 14976281, 34963200, 4510968, 45722224, 18820241 }}, "employment": [ { "organization-name": "Strongtone", "start-date": date("2001-03-14"), "end-date": date("2001-10-15") } ] }
+{ "id": 11174689, "id-copy": 11174689, "alias": "Thao", "name": "ThaoBrandenburg", "user-since": datetime("2012-04-21T05:25:58.000"), "user-since-copy": datetime("2012-04-21T05:25:58.000"), "friend-ids": {{ 37540210, 3918403, 33043564, 33664166 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2001-08-22"), "end-date": date("2004-11-19") } ] }
+{ "id": 11175613, "id-copy": 11175613, "alias": "Cuthbert", "name": "CuthbertHoover", "user-since": datetime("2008-04-25T01:12:49.000"), "user-since-copy": datetime("2008-04-25T01:12:49.000"), "friend-ids": {{ 27333562, 43896730, 6549030, 19576014, 4728367, 15430069, 22146931, 44593208, 14070342, 27801009, 6735368, 35798322, 47213791, 2388166 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2004-07-18") } ] }
+{ "id": 11188879, "id-copy": 11188879, "alias": "Corrie", "name": "CorrieOsterwise", "user-since": datetime("2011-01-20T21:11:19.000"), "user-since-copy": datetime("2011-01-20T21:11:19.000"), "friend-ids": {{ 47499393, 41394452, 27330253, 14958477, 14558879, 47694640, 28440147, 3437209, 40720108, 26390443 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2007-08-28") } ] }
+{ "id": 11209297, "id-copy": 11209297, "alias": "Merlin", "name": "MerlinLambert", "user-since": datetime("2012-07-01T09:30:07.000"), "user-since-copy": datetime("2012-07-01T09:30:07.000"), "friend-ids": {{ 28451212, 22119974, 1386726, 20860479, 37160852, 38281524, 17165711, 41076637, 19118162 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2012-06-26"), "end-date": date("2012-06-09") } ] }
+{ "id": 11216260, "id-copy": 11216260, "alias": "Randy", "name": "RandyEckhardstein", "user-since": datetime("2006-12-05T07:09:34.000"), "user-since-copy": datetime("2006-12-05T07:09:34.000"), "friend-ids": {{ 39744737, 14315897, 1342674, 1761832, 41393930, 21351330, 17845632, 39034426, 15297881, 11656496, 11376855 }}, "employment": [ { "organization-name": "Unijobam", "start-date": date("2009-12-19") } ] }
+{ "id": 11224090, "id-copy": 11224090, "alias": "Alayna", "name": "AlaynaHay", "user-since": datetime("2008-12-27T11:44:03.000"), "user-since-copy": datetime("2008-12-27T11:44:03.000"), "friend-ids": {{ 9220004, 31827642, 27616881, 26175415, 43152043, 36272681, 669731, 40783516, 31718359, 47123044, 24487696, 31178381, 39602057, 2619975, 27562896, 29215321, 35104306, 909466, 18897009, 35295634 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2003-02-01"), "end-date": date("2007-02-07") } ] }
+{ "id": 11226055, "id-copy": 11226055, "alias": "Tony", "name": "TonyBowman", "user-since": datetime("2011-06-27T19:37:38.000"), "user-since-copy": datetime("2011-06-27T19:37:38.000"), "friend-ids": {{ 38143523, 845148, 17273955, 5476646, 28032520, 29082922, 26004648, 7037738, 34413190, 22897549, 19873990, 22338498, 10902206, 43469888, 21968875, 5127825, 11962760, 43764181, 20623302, 23901531, 3402018, 15386752, 30847912, 205201 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-09-13"), "end-date": date("2011-01-10") } ] }
+{ "id": 11250445, "id-copy": 11250445, "alias": "Charlie", "name": "CharlieHaynes", "user-since": datetime("2009-06-08T22:50:05.000"), "user-since-copy": datetime("2009-06-08T22:50:05.000"), "friend-ids": {{ 18548568, 33185990, 25924893, 44738376, 17285644, 30895698, 40664753, 45663520, 13757940, 46543434, 27472319, 7112791, 45257808, 29363383, 24726693, 39990597, 36277676, 6623887, 42795972, 29019649, 22035134, 1362080, 9071131 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2008-02-21"), "end-date": date("2009-12-28") } ] }
+{ "id": 11268778, "id-copy": 11268778, "alias": "Chuck", "name": "ChuckRamos", "user-since": datetime("2005-09-24T12:19:57.000"), "user-since-copy": datetime("2005-09-24T12:19:57.000"), "friend-ids": {{ 2142650, 15399676, 40659179, 32507535, 32269323, 46947373, 46293990, 4237301, 41447393, 21345670, 47299716, 8515646, 27204593, 6676856, 21757183, 13647535, 28951520, 23198255, 1618106, 18189425, 46835891, 7056692, 26622607 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2004-06-24"), "end-date": date("2006-01-05") } ] }
+{ "id": 11269867, "id-copy": 11269867, "alias": "Bettye", "name": "BettyeTeagarden", "user-since": datetime("2006-02-15T08:28:04.000"), "user-since-copy": datetime("2006-02-15T08:28:04.000"), "friend-ids": {{ 3227122, 9086278, 26175058, 16380287, 15179776, 6343969, 15198730, 7420831, 38504400, 5337815, 35914644, 42885098, 2521174, 43359140, 17884442, 3131060, 35723204, 14956242, 78003, 7455524, 3371831, 46465463, 9947087 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2000-07-21"), "end-date": date("2007-10-28") } ] }
+{ "id": 11272591, "id-copy": 11272591, "alias": "Caris", "name": "CarisCatleay", "user-since": datetime("2007-01-27T07:35:12.000"), "user-since-copy": datetime("2007-01-27T07:35:12.000"), "friend-ids": {{ 26014944 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2012-07-15"), "end-date": date("2012-07-01") } ] }
+{ "id": 11306677, "id-copy": 11306677, "alias": "Chong", "name": "ChongPawle", "user-since": datetime("2007-09-13T00:31:41.000"), "user-since-copy": datetime("2007-09-13T00:31:41.000"), "friend-ids": {{ 11341417, 23669364, 41504484, 29889550, 268223, 26888454, 43915376, 23795433, 14021648, 25630355, 19831181, 15828987 }}, "employment": [ { "organization-name": "Fax-fax", "start-date": date("2011-01-06"), "end-date": date("2011-10-06") } ] }
+{ "id": 11309383, "id-copy": 11309383, "alias": "Lyn", "name": "LynKnapp", "user-since": datetime("2010-07-21T15:29:58.000"), "user-since-copy": datetime("2010-07-21T15:29:58.000"), "friend-ids": {{ 27610153 }}, "employment": [ { "organization-name": "Tanzumbam", "start-date": date("2012-08-28"), "end-date": date("2012-08-29") } ] }
+{ "id": 11335972, "id-copy": 11335972, "alias": "Emmett", "name": "EmmettBaxter", "user-since": datetime("2008-04-25T01:22:30.000"), "user-since-copy": datetime("2008-04-25T01:22:30.000"), "friend-ids": {{ 23133373, 28796661, 13045317, 34201656, 44749284, 42654826, 988887, 5039257, 18280226, 30366668, 22387991, 32676638, 24149069, 6307083, 17556069, 16687473, 4101198, 41964241, 39245728 }}, "employment": [ { "organization-name": "Greencare", "start-date": date("2004-11-22") } ] }
+{ "id": 11347261, "id-copy": 11347261, "alias": "Linda", "name": "LindaBaldwin", "user-since": datetime("2010-04-21T08:05:44.000"), "user-since-copy": datetime("2010-04-21T08:05:44.000"), "friend-ids": {{ 1423464, 7534626, 19522889, 25132532, 19933077, 36713596, 31725151, 46644015, 17758352, 37356325, 43714985, 29437022, 21616894, 32487769, 18527683, 32632034, 5598064, 47187635, 23490346 }}, "employment": [ { "organization-name": "overtech", "start-date": date("2005-06-22"), "end-date": date("2007-02-18") } ] }
+{ "id": 11348356, "id-copy": 11348356, "alias": "Chery", "name": "CherySandford", "user-since": datetime("2011-04-23T21:22:21.000"), "user-since-copy": datetime("2011-04-23T21:22:21.000"), "friend-ids": {{ 14076544, 42221517 }}, "employment": [ { "organization-name": "Freshfix", "start-date": date("2000-07-25") } ] }
+{ "id": 11350432, "id-copy": 11350432, "alias": "Fletcher", "name": "FletcherRowley", "user-since": datetime("2012-01-22T12:30:57.000"), "user-since-copy": datetime("2012-01-22T12:30:57.000"), "friend-ids": {{ 43655299, 46172971, 29175610, 22537183, 30612976, 21304031, 40531272, 6719806, 42232806, 18593968, 29334159 }}, "employment": [ { "organization-name": "highfax", "start-date": date("2002-02-17"), "end-date": date("2011-03-16") } ] }
+{ "id": 11403742, "id-copy": 11403742, "alias": "Neil", "name": "NeilHobbs", "user-since": datetime("2012-02-26T07:07:17.000"), "user-since-copy": datetime("2012-02-26T07:07:17.000"), "friend-ids": {{ 28387528, 39844931, 32868894, 45540524, 35239986, 44255870, 20859099 }}, "employment": [ { "organization-name": "Fix-touch", "start-date": date("2008-11-28"), "end-date": date("2009-06-01") } ] }
+{ "id": 11405905, "id-copy": 11405905, "alias": "Maria", "name": "MariaMoore", "user-since": datetime("2010-05-22T22:23:16.000"), "user-since-copy": datetime("2010-05-22T22:23:16.000"), "friend-ids": {{ 31883861, 37245457, 28570944, 34781997, 8502652, 44653970, 20757487, 13575261, 13950179, 14347829, 35701908, 35781889, 12226908, 35939258, 5106463, 43910072, 10696743, 21876393, 2309465, 1889615 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2008-03-27") } ] }
+{ "id": 11417455, "id-copy": 11417455, "alias": "Malka", "name": "MalkaWilkinson", "user-since": datetime("2012-04-11T17:22:49.000"), "user-since-copy": datetime("2012-04-11T17:22:49.000"), "friend-ids": {{ 29261780, 13274200, 41060932, 8851180, 34769837, 3296096, 19488423, 41776348, 44518076, 16669411, 19983817, 26799511, 16166476, 31396373, 4090033, 37968801, 36665813 }}, "employment": [ { "organization-name": "Icerunin", "start-date": date("2004-03-12") } ] }
+{ "id": 11423752, "id-copy": 11423752, "alias": "Eliott", "name": "EliottRoche", "user-since": datetime("2007-07-01T04:36:16.000"), "user-since-copy": datetime("2007-07-01T04:36:16.000"), "friend-ids": {{ 34273508, 10643569, 13667612, 19808579, 46658485, 43209365, 7962014, 24567991, 25086057 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2005-11-26") } ] }
+{ "id": 11424097, "id-copy": 11424097, "alias": "Vernie", "name": "VernieWynter", "user-since": datetime("2009-02-15T02:35:16.000"), "user-since-copy": datetime("2009-02-15T02:35:16.000"), "friend-ids": {{ 41874621, 26330221, 38930134, 39892396, 42859035, 8165423, 36128938, 5692990, 28144348, 40741492 }}, "employment": [ { "organization-name": "Tranzap", "start-date": date("2002-04-06") } ] }
+{ "id": 11437771, "id-copy": 11437771, "alias": "Brittani", "name": "BrittaniMoore", "user-since": datetime("2007-11-16T20:56:35.000"), "user-since-copy": datetime("2007-11-16T20:56:35.000"), "friend-ids": {{ 30502334, 18483492, 37360877, 25153720, 9181228, 28352241, 37928337, 13522608, 20974146, 30187156, 22832401, 20899789, 44606652, 3333090, 39581573, 34303132, 33802071, 27053375, 32467186, 40213342, 37254307, 7275338, 2622767 }}, "employment": [ { "organization-name": "Quadlane", "start-date": date("2010-02-07") } ] }
+{ "id": 11463820, "id-copy": 11463820, "alias": "Gaye", "name": "GayeWelty", "user-since": datetime("2005-01-04T14:32:34.000"), "user-since-copy": datetime("2005-01-04T14:32:34.000"), "friend-ids": {{ 44428980, 1291384, 10830264, 2433795, 17582948, 17416624, 21578025, 14538036, 41470487, 34384402, 42863727, 35119046, 35673193, 14814350, 29380258, 30253821, 41180218, 13945680, 15533641, 26510747 }}, "employment": [ { "organization-name": "Hexsanhex", "start-date": date("2011-03-01"), "end-date": date("2011-09-13") } ] }
+{ "id": 11489143, "id-copy": 11489143, "alias": "Clover", "name": "CloverWest", "user-since": datetime("2012-04-14T13:56:22.000"), "user-since-copy": datetime("2012-04-14T13:56:22.000"), "friend-ids": {{ 14606516, 25835971, 10555192, 4853088, 43631398, 45670230, 43866490, 25690294, 22040370, 7047997, 3374421, 34831455, 31517002, 2998558, 40893307, 40067725, 1601716, 43041725, 8953042, 33848939 }}, "employment": [ { "organization-name": "Xx-drill", "start-date": date("2005-02-03"), "end-date": date("2006-06-26") } ] }
+{ "id": 11518480, "id-copy": 11518480, "alias": "Amada", "name": "AmadaTanner", "user-since": datetime("2006-05-06T12:27:31.000"), "user-since-copy": datetime("2006-05-06T12:27:31.000"), "friend-ids": {{ }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2002-04-02") } ] }
+{ "id": 11525302, "id-copy": 11525302, "alias": "Marissa", "name": "MarissaEndsley", "user-since": datetime("2006-09-26T08:55:36.000"), "user-since-copy": datetime("2006-09-26T08:55:36.000"), "friend-ids": {{ 35476434, 12502442, 19198691, 35401830, 14414490, 11372357, 28886265, 3490052, 13587860, 8127851, 20732439, 44816539, 6616740, 12785784, 16907259, 10942007, 26207, 21026660, 39284170, 25761798, 20688453, 45805952, 15912564 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-10-07"), "end-date": date("2010-09-09") } ] }
+{ "id": 11559262, "id-copy": 11559262, "alias": "Herb", "name": "HerbPaul", "user-since": datetime("2011-04-09T22:23:26.000"), "user-since-copy": datetime("2011-04-09T22:23:26.000"), "friend-ids": {{ 46915837, 26659094 }}, "employment": [ { "organization-name": "Qvohouse", "start-date": date("2011-07-05"), "end-date": date("2011-07-07") } ] }
+{ "id": 11570386, "id-copy": 11570386, "alias": "Hollis", "name": "HollisIseman", "user-since": datetime("2009-07-11T12:26:25.000"), "user-since-copy": datetime("2009-07-11T12:26:25.000"), "friend-ids": {{ 28136044, 6945424, 35390131, 12649451, 38331381, 30399822, 47834313 }}, "employment": [ { "organization-name": "subtam", "start-date": date("2011-02-12") } ] }
+{ "id": 11570617, "id-copy": 11570617, "alias": "Deshawn", "name": "DeshawnBashline", "user-since": datetime("2006-04-14T01:05:38.000"), "user-since-copy": datetime("2006-04-14T01:05:38.000"), "friend-ids": {{ 9319940, 45556479, 44222390, 22928539, 27909778, 21162548, 8657905, 15375082, 38338906, 21416203, 7519884, 30405265, 32148274, 35560776, 29973785, 19277384, 44256954, 40425041, 30328494, 39977803, 40280359, 3079013, 18841024, 23001903 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2009-02-14") } ] }
+{ "id": 11571085, "id-copy": 11571085, "alias": "Reina", "name": "ReinaWheeler", "user-since": datetime("2010-04-28T08:05:29.000"), "user-since-copy": datetime("2010-04-28T08:05:29.000"), "friend-ids": {{ 25357083, 40592075, 10585644, 33173927, 42515085 }}, "employment": [ { "organization-name": "Zununoing", "start-date": date("2000-08-03") } ] }
+{ "id": 11571217, "id-copy": 11571217, "alias": "Modesto", "name": "ModestoPark", "user-since": datetime("2006-01-18T06:28:01.000"), "user-since-copy": datetime("2006-01-18T06:28:01.000"), "friend-ids": {{ 3765450, 13287809, 17696557, 32161653, 46823306, 2818286, 38794110, 24894266, 33129431, 26474332, 9356762, 38679272, 40502952, 34470547, 30005230, 32074010, 38611550 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2001-09-01"), "end-date": date("2003-04-11") } ] }
+{ "id": 11595592, "id-copy": 11595592, "alias": "Bert", "name": "BertAtkinson", "user-since": datetime("2011-09-03T07:24:42.000"), "user-since-copy": datetime("2011-09-03T07:24:42.000"), "friend-ids": {{ 36724561, 45824456, 33567747, 21400268, 11419574, 47463040, 6480088, 45216774, 26857982, 7140352, 1884512, 29610211, 2626672, 41371388, 43582371, 42445087, 14734124, 3580372, 40134022 }}, "employment": [ { "organization-name": "Keytech", "start-date": date("2006-06-27"), "end-date": date("2007-06-07") } ] }
+{ "id": 11617963, "id-copy": 11617963, "alias": "Sherry", "name": "SherryPirl", "user-since": datetime("2010-08-26T06:37:30.000"), "user-since-copy": datetime("2010-08-26T06:37:30.000"), "friend-ids": {{ 30179664, 7140787, 14622079, 5810238, 32189583, 17103583 }}, "employment": [ { "organization-name": "Statcode", "start-date": date("2000-02-07"), "end-date": date("2004-11-24") } ] }
+{ "id": 11637820, "id-copy": 11637820, "alias": "Aislin", "name": "AislinPyle", "user-since": datetime("2005-01-04T00:11:51.000"), "user-since-copy": datetime("2005-01-04T00:11:51.000"), "friend-ids": {{ 17232277, 46376966, 22503632, 14771156, 37550654, 3930020, 7116826, 38303815, 30210948, 10532544, 44382464, 32051602 }}, "employment": [ { "organization-name": "Mathtech", "start-date": date("2004-05-06") } ] }
+{ "id": 11642026, "id-copy": 11642026, "alias": "Brenden", "name": "BrendenLucy", "user-since": datetime("2010-09-18T13:14:17.000"), "user-since-copy": datetime("2010-09-18T13:14:17.000"), "friend-ids": {{ 4037044, 13420154, 10023579, 7611523, 10090302, 36514218, 24369151, 10481696, 341494 }}, "employment": [ { "organization-name": "Latsonity", "start-date": date("2007-07-05") } ] }
+{ "id": 11646016, "id-copy": 11646016, "alias": "Millard", "name": "MillardCribbs", "user-since": datetime("2012-07-01T13:28:56.000"), "user-since-copy": datetime("2012-07-01T13:28:56.000"), "friend-ids": {{ 29358027, 24800104, 1146956, 29116484, 12223225, 6324161, 46576675 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2004-04-28") } ] }
+{ "id": 11670331, "id-copy": 11670331, "alias": "Deetta", "name": "DeettaCrom", "user-since": datetime("2008-04-01T00:12:47.000"), "user-since-copy": datetime("2008-04-01T00:12:47.000"), "friend-ids": {{ 34871046, 45366633, 40484162, 45505621, 47279131, 5464046, 18435436, 24937987, 18253019, 5870229, 46379232, 13988659, 37921800, 2085103, 21652843, 4802881, 11658526, 40771399, 32938488, 8409007, 27179341, 4496744 }}, "employment": [ { "organization-name": "Dancode", "start-date": date("2003-10-21"), "end-date": date("2008-06-06") } ] }
+{ "id": 11670739, "id-copy": 11670739, "alias": "Rudyard", "name": "RudyardErrett", "user-since": datetime("2005-03-08T18:26:12.000"), "user-since-copy": datetime("2005-03-08T18:26:12.000"), "friend-ids": {{ 13253132, 38903405, 45479471, 11551894, 44803858, 34016119, 2477206, 27909363, 2584557, 29078732, 13687500, 1038800, 14467502, 3369722, 11731177, 15702876, 37034289, 21943459 }}, "employment": [ { "organization-name": "Indiex", "start-date": date("2011-01-20") } ] }
+{ "id": 11672578, "id-copy": 11672578, "alias": "Juli", "name": "JuliMcclymonds", "user-since": datetime("2010-07-17T13:53:57.000"), "user-since-copy": datetime("2010-07-17T13:53:57.000"), "friend-ids": {{ 16548983, 7350585, 44497037 }}, "employment": [ { "organization-name": "Groovetex", "start-date": date("2003-05-23"), "end-date": date("2009-08-01") } ] }
+{ "id": 11678242, "id-copy": 11678242, "alias": "Andy", "name": "AndyPritchard", "user-since": datetime("2008-05-26T06:52:12.000"), "user-since-copy": datetime("2008-05-26T06:52:12.000"), "friend-ids": {{ 24351029, 7396495, 11653891, 24314059, 17256129, 19177689, 23024021, 15135862, 9201238, 24204194 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2004-02-06"), "end-date": date("2011-10-22") } ] }
+{ "id": 11697754, "id-copy": 11697754, "alias": "Jeanette", "name": "JeanetteBullard", "user-since": datetime("2005-11-20T09:56:59.000"), "user-since-copy": datetime("2005-11-20T09:56:59.000"), "friend-ids": {{ 22439123, 42241829, 21396058, 6050318, 4951741, 4940964, 22719195, 21108984, 1496059, 41986346, 20838301, 34979646, 19524886, 6383593, 37747505, 26787944, 45486736, 7537516 }}, "employment": [ { "organization-name": "Inchdox", "start-date": date("2006-02-20") } ] }
+{ "id": 11755633, "id-copy": 11755633, "alias": "Amina", "name": "AminaBurkett", "user-since": datetime("2012-03-22T02:05:59.000"), "user-since-copy": datetime("2012-03-22T02:05:59.000"), "friend-ids": {{ 18177270, 40223354, 29458819, 37905784, 43047863, 2679271, 9768971, 32443429, 37829920, 35493852, 28086857, 11910843, 31003179, 40873211, 42786132, 44388462 }}, "employment": [ { "organization-name": "Tanzimcare", "start-date": date("2009-11-21"), "end-date": date("2011-03-16") } ] }
+{ "id": 11780581, "id-copy": 11780581, "alias": "Simona", "name": "SimonaDrumm", "user-since": datetime("2010-09-10T00:03:56.000"), "user-since-copy": datetime("2010-09-10T00:03:56.000"), "friend-ids": {{ 14930223, 14107902, 18276584, 12824637, 44738306, 252529, 17504815, 26802467, 33312123, 15516170, 9060069, 42300993, 15746839, 61844, 1966381, 31284798, 40145954, 31282156, 15764470, 9894586, 41833755 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-03-27") } ] }
+{ "id": 11788096, "id-copy": 11788096, "alias": "Camie", "name": "CamieCressman", "user-since": datetime("2007-10-25T23:38:14.000"), "user-since-copy": datetime("2007-10-25T23:38:14.000"), "friend-ids": {{ 29310801, 37328820, 47367940, 36796774, 21244245, 7126676, 8254586, 47578674, 39514952, 33623672, 12854915, 6679164, 44128364, 44434013, 20530444, 12243267 }}, "employment": [ { "organization-name": "Opeelectronics", "start-date": date("2000-06-20") } ] }
+{ "id": 11791471, "id-copy": 11791471, "alias": "Robt", "name": "RobtChristman", "user-since": datetime("2009-08-08T21:01:18.000"), "user-since-copy": datetime("2009-08-08T21:01:18.000"), "friend-ids": {{ 9265036, 17976405, 32435071, 7236713, 21936800, 42691957, 35478695, 40052609, 14063303, 43864025, 1254074, 39237113, 11307270, 37061951, 17360733, 21102633, 21364546, 35445000, 44857867 }}, "employment": [ { "organization-name": "Tripplelane", "start-date": date("2008-01-02"), "end-date": date("2010-05-19") } ] }
+{ "id": 11809528, "id-copy": 11809528, "alias": "Donya", "name": "DonyaNash", "user-since": datetime("2008-06-09T09:42:48.000"), "user-since-copy": datetime("2008-06-09T09:42:48.000"), "friend-ids": {{ 25365000, 20270987, 39083310, 16364767, 1960249, 39747742, 17169019, 780802, 37012712, 27956954, 35502958, 10600365, 38247667, 47815777, 25182855, 13670701, 27795853, 24952265 }}, "employment": [ { "organization-name": "Goldcity", "start-date": date("2011-10-15") } ] }
+{ "id": 11830822, "id-copy": 11830822, "alias": "Lincoln", "name": "LincolnFuchs", "user-since": datetime("2008-01-22T19:08:51.000"), "user-since-copy": datetime("2008-01-22T19:08:51.000"), "friend-ids": {{ 29014579, 29789039, 2225447, 37872940, 37026231, 3223799, 40601178 }}, "employment": [ { "organization-name": "Villa-tech", "start-date": date("2006-01-14"), "end-date": date("2010-04-24") } ] }
+{ "id": 11867464, "id-copy": 11867464, "alias": "Emmerson", "name": "EmmersonMoore", "user-since": datetime("2006-12-26T00:15:40.000"), "user-since-copy": datetime("2006-12-26T00:15:40.000"), "friend-ids": {{ 5310233, 16498267, 12436996, 24801626, 44135326, 45729147, 6922158, 25920138, 16324404, 30272475, 22873357, 720070, 9722837, 29718785, 5402637, 287196, 32557949 }}, "employment": [ { "organization-name": "tresline", "start-date": date("2007-06-16"), "end-date": date("2007-02-05") } ] }
+{ "id": 11912419, "id-copy": 11912419, "alias": "Wallis", "name": "WallisFuchs", "user-since": datetime("2012-01-07T08:13:18.000"), "user-since-copy": datetime("2012-01-07T08:13:18.000"), "friend-ids": {{ 11115387, 19639311, 33957302, 8746808, 20140328, 35866755, 29492622, 24246926, 14412186, 1610423, 1139443, 23667812, 6972455, 18354247, 7072427, 43742595, 20711654, 7179925, 66544, 12066267, 8914321, 35602734 }}, "employment": [ { "organization-name": "Fixdintex", "start-date": date("2008-10-23"), "end-date": date("2008-06-18") } ] }
+{ "id": 11918764, "id-copy": 11918764, "alias": "Jamison", "name": "JamisonKnight", "user-since": datetime("2012-02-28T12:46:09.000"), "user-since-copy": datetime("2012-02-28T12:46:09.000"), "friend-ids": {{ 5296309, 37783012, 18620712, 8255206, 10270999, 47361618, 39691488, 33528430, 22926601, 12751125, 34000354, 32638692, 19461108, 9760202, 30157968, 265361, 24683869, 19612648, 29021437, 40094162 }}, "employment": [ { "organization-name": "Dandamace", "start-date": date("2003-04-16"), "end-date": date("2011-08-28") } ] }
+{ "id": 11934781, "id-copy": 11934781, "alias": "Titus", "name": "TitusGertraht", "user-since": datetime("2011-05-02T12:41:28.000"), "user-since-copy": datetime("2011-05-02T12:41:28.000"), "friend-ids": {{ 32699552, 17016611, 46281182, 32515791, 12860342, 22463323, 33042577, 4477908, 37152051, 5462628, 45666108, 42424199, 44831639, 44546969, 30686685, 40580034 }}, "employment": [ { "organization-name": "Coneflex", "start-date": date("2000-04-16") } ] }
+{ "id": 11945014, "id-copy": 11945014, "alias": "Lavern", "name": "LavernRahl", "user-since": datetime("2005-08-13T08:07:58.000"), "user-since-copy": datetime("2005-08-13T08:07:58.000"), "friend-ids": {{ 15127940, 37543274, 13877909, 8961585, 13712343, 38178056, 21469501, 2994082, 24368304, 33508930, 41765591, 37858577, 42295002 }}, "employment": [ { "organization-name": "U-electrics", "start-date": date("2001-07-20") } ] }
+{ "id": 11951098, "id-copy": 11951098, "alias": "Tera", "name": "TeraByers", "user-since": datetime("2012-08-03T19:41:26.000"), "user-since-copy": datetime("2012-08-03T19:41:26.000"), "friend-ids": {{ 15537238, 13699967, 10587728, 23542817, 12703626, 25024772, 19223339, 5547239, 42576945, 27351017, 22726496, 25268071, 4361323, 24631578, 38669047, 44781738, 34646381 }}, "employment": [ { "organization-name": "Sublamdox", "start-date": date("2008-01-04"), "end-date": date("2011-01-14") } ] }
+{ "id": 11957011, "id-copy": 11957011, "alias": "Frannie", "name": "FrannieRoose", "user-since": datetime("2007-04-05T18:00:20.000"), "user-since-copy": datetime("2007-04-05T18:00:20.000"), "friend-ids": {{ 9114095, 4905395, 41862236, 21901856, 39479601, 4025127, 1517878, 16698416, 10853001, 18625728, 15395201, 17825510, 40384476, 18779630, 1832149, 41381869, 40010653, 21121933, 18598397, 12806945, 11465558 }}, "employment": [ { "organization-name": "Y-geohex", "start-date": date("2006-12-22") } ] }
+{ "id": 11965318, "id-copy": 11965318, "alias": "Donella", "name": "DonellaPriebe", "user-since": datetime("2010-10-25T19:45:41.000"), "user-since-copy": datetime("2010-10-25T19:45:41.000"), "friend-ids": {{ 40521325 }}, "employment": [ { "organization-name": "Doublezone", "start-date": date("2010-11-19"), "end-date": date("2011-08-18") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.4.adm
index fab2047..8d89e67 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.4.adm
@@ -1,3 +1,9 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 2 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ 3 ] }
+{ "values": [ 4 ] }
+{ "values": [ 7 ] }
{ "values": [ 555, 5 ] }
{ "values": [ 888, 8 ] }
{ "values": [ 999, 9 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.5.adm
index fab2047..8d89e67 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.5.adm
@@ -1,3 +1,9 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 2 ] }
+{ "values": [ null, 6 ] }
+{ "values": [ 3 ] }
+{ "values": [ 4 ] }
+{ "values": [ 7 ] }
{ "values": [ 555, 5 ] }
{ "values": [ 888, 8 ] }
{ "values": [ 999, 9 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.6.adm
index 761a8fc..3f68b53 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.6.adm
@@ -1 +1,3 @@
+{ "values": [ null, 1 ] }
+{ "values": [ 2 ] }
{ "values": [ 888, 8 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.8.adm
index 85c91e6..50f4b46 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/insert_nulls_with_secondary_idx/insert_nulls_with_secondary_idx.8.adm
@@ -1,3 +1,9 @@
+{ "values": [ null, 1 ] }
+{ "values": [ null, 7 ] }
+{ "values": [ null, 9 ] }
+{ "values": [ 3 ] }
+{ "values": [ 6 ] }
+{ "values": [ 8 ] }
{ "values": [ 222, 2 ] }
{ "values": [ 444, 4 ] }
{ "values": [ 555, 5 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1531/query-ASTERIXDB-1531.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1531/query-ASTERIXDB-1531.3.adm
index 9dbec6e..28ad77c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1531/query-ASTERIXDB-1531.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1531/query-ASTERIXDB-1531.3.adm
@@ -1,12 +1,12 @@
-{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
-{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
-{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
-{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
-{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
-{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
-{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
-{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
-{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
-{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
-{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
-{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/string_eq_01/string_eq_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/string_eq_01/string_eq_01.1.adm
index 17477c1..58e48d8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/string_eq_01/string_eq_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/string_eq_01/string_eq_01.1.adm
@@ -1 +1 @@
-{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/view_1/view_1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/view_1/view_1.2.adm
new file mode 100644
index 0000000..00cf01e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/view_1/view_1.2.adm
@@ -0,0 +1,3 @@
+{ "r": 0 }
+{ "r": 1 }
+{ "r": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/view_1/view_1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/view_1/view_1.3.adm
new file mode 100644
index 0000000..c746a5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/view_1/view_1.3.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "x/y", "DatasetName": "vy", "Dependencies": [ [ [ "x/z", "vz" ] ], [ ], [ ] ] }
+{ "DataverseName": "x/z", "DatasetName": "vz", "Dependencies": [ [ ], [ ], [ ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm
index 79cf3f6..64fb87e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.1.adm
@@ -1,4 +1,4 @@
-{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
-{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
-{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
-{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-empty-dataset-with-index/insert-into-empty-dataset-with-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-empty-dataset-with-index/insert-into-empty-dataset-with-index.1.adm
index 221f0c2..24abd7f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-empty-dataset-with-index/insert-into-empty-dataset-with-index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-empty-dataset-with-index/insert-into-empty-dataset-with-index.1.adm
@@ -1 +1 @@
-{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_01/insert-into-loaded-dataset-with-index_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_01/insert-into-loaded-dataset-with-index_01.1.adm
index c5ef3e1..a8a3f2f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_01/insert-into-loaded-dataset-with-index_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_01/insert-into-loaded-dataset-with-index_01.1.adm
@@ -1,7 +1,7 @@
-{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
-{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
-{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
-{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
-{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
-{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
-{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.1.adm
index 8e47a65..2f397b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.1.adm
@@ -1,4 +1,4 @@
-{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
-{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
-{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
-{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-index/load-with-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-index/load-with-index.1.adm
index f278c1c..14841c8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-index/load-with-index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-index/load-with-index.1.adm
@@ -1,4 +1,4 @@
-{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("32.84,67.14") }, "tweetid": 2, "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
-{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("47.51,83.99") }, "tweetid": 6, "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
-{ "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "sender-location": point("36.21,72.6") }, "tweetid": 7, "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
-{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("29.15,76.53") }, "tweetid": 10, "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("32.84,67.14") }, "tweetid": 2, "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("47.51,83.99") }, "tweetid": 6, "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "sender-location": point("36.21,72.6") }, "tweetid": 7, "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("29.15,76.53") }, "tweetid": 10, "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-ngram-index/load-with-ngram-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-ngram-index/load-with-ngram-index.1.adm
index 49b3840..5937906 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-ngram-index/load-with-ngram-index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-ngram-index/load-with-ngram-index.1.adm
@@ -1,6 +1,6 @@
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("47.44,80.65") }, "tweetid": 1, "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("29.72,75.8") }, "tweetid": 3, "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("39.28,70.48") }, "tweetid": 4, "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("40.09,92.69") }, "tweetid": 5, "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("46.05,93.34") }, "tweetid": 8, "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "tweetid": 9, "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("47.44,80.65") }, "tweetid": 1, "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("29.72,75.8") }, "tweetid": 3, "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("39.28,70.48") }, "tweetid": 4, "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("40.09,92.69") }, "tweetid": 5, "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("46.05,93.34") }, "tweetid": 8, "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "tweetid": 9, "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-rtree-index/load-with-rtree-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-rtree-index/load-with-rtree-index.1.adm
index 1acaf44..eac8946 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-rtree-index/load-with-rtree-index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-rtree-index/load-with-rtree-index.1.adm
@@ -1,7 +1,7 @@
-{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("32.84,67.14") }, "tweetid": 2, "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("29.72,75.8") }, "tweetid": 3, "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("39.28,70.48") }, "tweetid": 4, "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
-{ "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "sender-location": point("36.21,72.6") }, "tweetid": 7, "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "tweetid": 9, "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
-{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("29.15,76.53") }, "tweetid": 10, "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
-{ "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649, "sender-location": point("37.59,68.42") }, "tweetid": 11, "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("32.84,67.14") }, "tweetid": 2, "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("29.72,75.8") }, "tweetid": 3, "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("39.28,70.48") }, "tweetid": 4, "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "sender-location": point("36.21,72.6") }, "tweetid": 7, "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "tweetid": 9, "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159, "sender-location": point("29.15,76.53") }, "tweetid": 10, "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649, "sender-location": point("37.59,68.42") }, "tweetid": 11, "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-word-index/load-with-word-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-word-index/load-with-word-index.1.adm
index 49b3840..5937906 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-word-index/load-with-word-index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/nested-index-dml/load-with-word-index/load-with-word-index.1.adm
@@ -1,6 +1,6 @@
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("47.44,80.65") }, "tweetid": 1, "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("29.72,75.8") }, "tweetid": 3, "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("39.28,70.48") }, "tweetid": 4, "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("40.09,92.69") }, "tweetid": 5, "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("46.05,93.34") }, "tweetid": 8, "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
-{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "tweetid": 9, "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("47.44,80.65") }, "tweetid": 1, "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("29.72,75.8") }, "tweetid": 3, "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("39.28,70.48") }, "tweetid": 4, "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("40.09,92.69") }, "tweetid": 5, "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("46.05,93.34") }, "tweetid": 8, "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416, "sender-location": point("36.86,74.62") }, "tweetid": 9, "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.1.adm
new file mode 100644
index 0000000..18cf1ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.1.adm
@@ -0,0 +1,20 @@
+{ "c": 9 }
+{ "b": null, "c": 11 }
+{ "b": 5, "c": 10 }
+{ "b": 6, "c": 12 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": 8, "c": 16 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.10.adm
new file mode 100644
index 0000000..8c27a90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.10.adm
@@ -0,0 +1,20 @@
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.11.adm
new file mode 100644
index 0000000..db745b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.11.adm
@@ -0,0 +1,20 @@
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.12.adm
new file mode 100644
index 0000000..69dbeb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.12.adm
@@ -0,0 +1,20 @@
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.13.adm
new file mode 100644
index 0000000..cab4536
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.13.adm
@@ -0,0 +1,20 @@
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.14.adm
new file mode 100644
index 0000000..2206ce8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.14.adm
@@ -0,0 +1,20 @@
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.15.adm
new file mode 100644
index 0000000..2206ce8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.15.adm
@@ -0,0 +1,20 @@
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.2.adm
new file mode 100644
index 0000000..18cf1ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.2.adm
@@ -0,0 +1,20 @@
+{ "c": 9 }
+{ "b": null, "c": 11 }
+{ "b": 5, "c": 10 }
+{ "b": 6, "c": 12 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": 8, "c": 16 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.3.adm
new file mode 100644
index 0000000..ab6cbad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.3.adm
@@ -0,0 +1,20 @@
+{ "b": 5, "c": 10 }
+{ "b": 6, "c": 12 }
+{ "c": 9 }
+{ "b": null, "c": 11 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": null, "c": 15 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.4.adm
new file mode 100644
index 0000000..b9c2916
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.4.adm
@@ -0,0 +1,20 @@
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "c": 9 }
+{ "b": null, "c": 11 }
+{ "b": 5, "c": 10 }
+{ "b": 6, "c": 12 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": 4, "c": 8 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.5.adm
new file mode 100644
index 0000000..867353f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.5.adm
@@ -0,0 +1,20 @@
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": null, "c": 15 }
+{ "b": 5, "c": 10 }
+{ "b": 6, "c": 12 }
+{ "c": 9 }
+{ "b": null, "c": 11 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": null, "c": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.6.adm
new file mode 100644
index 0000000..ebe21d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.6.adm
@@ -0,0 +1,20 @@
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.7.adm
new file mode 100644
index 0000000..8c68e70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.7.adm
@@ -0,0 +1,20 @@
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.8.adm
new file mode 100644
index 0000000..ebe21d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.8.adm
@@ -0,0 +1,20 @@
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.9.adm
new file mode 100644
index 0000000..c5a3c81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-2/order-by-2.9.adm
@@ -0,0 +1,20 @@
+{ "a": 1, "b": null, "c": 3 }
+{ "a": 1, "c": 1 }
+{ "a": 1, "b": 2, "c": 4 }
+{ "a": 1, "b": 1, "c": 2 }
+{ "a": 3.0, "b": null, "c": 19 }
+{ "a": 3.0, "c": 17 }
+{ "a": 3.0, "b": 10, "c": 20 }
+{ "a": 3.0, "b": 9, "c": 18 }
+{ "a": "1", "b": null, "c": 15 }
+{ "a": "1", "c": 13 }
+{ "a": "1", "b": 8, "c": 16 }
+{ "a": "1", "b": 7, "c": 14 }
+{ "b": null, "c": 11 }
+{ "c": 9 }
+{ "b": 6, "c": 12 }
+{ "b": 5, "c": 10 }
+{ "a": null, "b": null, "c": 7 }
+{ "a": null, "c": 5 }
+{ "a": null, "b": 4, "c": 8 }
+{ "a": null, "b": 3, "c": 6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.10.adm
new file mode 100644
index 0000000..bb1aaeb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.10.adm
@@ -0,0 +1,20 @@
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.11.adm
new file mode 100644
index 0000000..8e493f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.11.adm
@@ -0,0 +1,20 @@
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.12.adm
new file mode 100644
index 0000000..5889e26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.12.adm
@@ -0,0 +1,20 @@
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.13.adm
new file mode 100644
index 0000000..737cf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.13.adm
@@ -0,0 +1,20 @@
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 11, "b": null }
+{ "c": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.14.adm
new file mode 100644
index 0000000..7b9f08e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.14.adm
@@ -0,0 +1,20 @@
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.15.adm
new file mode 100644
index 0000000..65b9dde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.15.adm
@@ -0,0 +1,20 @@
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.16.adm
new file mode 100644
index 0000000..f9cc447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.16.adm
@@ -0,0 +1,20 @@
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.17.adm
new file mode 100644
index 0000000..f9cc447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.17.adm
@@ -0,0 +1,20 @@
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.3.adm
new file mode 100644
index 0000000..462070c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.3.adm
@@ -0,0 +1,20 @@
+{ "c": 9 }
+{ "c": 11, "b": null }
+{ "c": 10, "b": 5 }
+{ "c": 12, "b": 6 }
+{ "c": 5, "a": null }
+{ "c": 7, "a": null, "b": null }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 1, "a": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 17, "a": 3.0 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 13, "a": "1" }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 16, "a": "1", "b": 8 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.4.adm
new file mode 100644
index 0000000..462070c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.4.adm
@@ -0,0 +1,20 @@
+{ "c": 9 }
+{ "c": 11, "b": null }
+{ "c": 10, "b": 5 }
+{ "c": 12, "b": 6 }
+{ "c": 5, "a": null }
+{ "c": 7, "a": null, "b": null }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 1, "a": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 17, "a": 3.0 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 13, "a": "1" }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 16, "a": "1", "b": 8 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.5.adm
new file mode 100644
index 0000000..ca31db4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.5.adm
@@ -0,0 +1,20 @@
+{ "c": 10, "b": 5 }
+{ "c": 12, "b": 6 }
+{ "c": 9 }
+{ "c": 11, "b": null }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 5, "a": null }
+{ "c": 7, "a": null, "b": null }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 1, "a": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 17, "a": 3.0 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 13, "a": "1" }
+{ "c": 15, "a": "1", "b": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.6.adm
new file mode 100644
index 0000000..336b3bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.6.adm
@@ -0,0 +1,20 @@
+{ "c": 1, "a": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 17, "a": 3.0 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 13, "a": "1" }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 9 }
+{ "c": 11, "b": null }
+{ "c": 10, "b": 5 }
+{ "c": 12, "b": 6 }
+{ "c": 5, "a": null }
+{ "c": 7, "a": null, "b": null }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 8, "a": null, "b": 4 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.7.adm
new file mode 100644
index 0000000..833b380
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.7.adm
@@ -0,0 +1,20 @@
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 1, "a": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 17, "a": 3.0 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 13, "a": "1" }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 10, "b": 5 }
+{ "c": 12, "b": 6 }
+{ "c": 9 }
+{ "c": 11, "b": null }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 5, "a": null }
+{ "c": 7, "a": null, "b": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.8.adm
new file mode 100644
index 0000000..bb1aaeb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.8.adm
@@ -0,0 +1,20 @@
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.9.adm
new file mode 100644
index 0000000..9ed9734
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset-2/order-by-from-dataset-2.9.adm
@@ -0,0 +1,20 @@
+{ "c": 11, "b": null }
+{ "c": 9 }
+{ "c": 12, "b": 6 }
+{ "c": 10, "b": 5 }
+{ "c": 7, "a": null, "b": null }
+{ "c": 5, "a": null }
+{ "c": 8, "a": null, "b": 4 }
+{ "c": 6, "a": null, "b": 3 }
+{ "c": 3, "a": 1, "b": null }
+{ "c": 1, "a": 1 }
+{ "c": 4, "a": 1, "b": 2 }
+{ "c": 2, "a": 1, "b": 1 }
+{ "c": 19, "a": 3.0, "b": null }
+{ "c": 17, "a": 3.0 }
+{ "c": 20, "a": 3.0, "b": 10 }
+{ "c": 18, "a": 3.0, "b": 9 }
+{ "c": 15, "a": "1", "b": null }
+{ "c": 13, "a": "1" }
+{ "c": 16, "a": "1", "b": 8 }
+{ "c": 14, "a": "1", "b": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.3.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.1.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.4.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.1.adm
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.4.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.5.adm
new file mode 100644
index 0000000..78c3ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.5.adm
@@ -0,0 +1,5 @@
+{ "b": 2, "a": 1 }
+{ "b": 9, "a": 3.0 }
+{ "b": 8, "a": "1" }
+{ "b": 4 }
+{ "b": 7, "a": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.6.adm
new file mode 100644
index 0000000..5e6cb57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.6.adm
@@ -0,0 +1,5 @@
+{ "b": 8, "a": "1" }
+{ "b": 9, "a": 3.0 }
+{ "b": 2, "a": 1 }
+{ "b": 7, "a": null }
+{ "b": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.7.adm
new file mode 100644
index 0000000..7c3f599
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.7.adm
@@ -0,0 +1,5 @@
+{ "b": 7, "a": null }
+{ "b": 4 }
+{ "b": 8, "a": "1" }
+{ "b": 9, "a": 3.0 }
+{ "b": 2, "a": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.8.adm
new file mode 100644
index 0000000..5e6cb57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by-from-dataset/order-by-from-dataset.8.adm
@@ -0,0 +1,5 @@
+{ "b": 8, "a": "1" }
+{ "b": 9, "a": 3.0 }
+{ "b": 2, "a": 1 }
+{ "b": 7, "a": null }
+{ "b": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.2.adm
new file mode 100644
index 0000000..df08dd2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.2.adm
@@ -0,0 +1,5 @@
+{ "b": 4 }
+{ "a": null, "b": 7 }
+{ "a": 1, "b": 2 }
+{ "a": 3.0, "b": 9 }
+{ "a": "1", "b": 8 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.3.adm
new file mode 100644
index 0000000..0d262a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.3.adm
@@ -0,0 +1,5 @@
+{ "a": 1, "b": 2 }
+{ "a": 3.0, "b": 9 }
+{ "a": "1", "b": 8 }
+{ "b": 4 }
+{ "a": null, "b": 7 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.4.adm
new file mode 100644
index 0000000..86c085e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.4.adm
@@ -0,0 +1,5 @@
+{ "a": "1", "b": 8 }
+{ "a": 3.0, "b": 9 }
+{ "a": 1, "b": 2 }
+{ "a": null, "b": 7 }
+{ "b": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.5.adm
new file mode 100644
index 0000000..a27ff5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.5.adm
@@ -0,0 +1,5 @@
+{ "a": null, "b": 7 }
+{ "b": 4 }
+{ "a": "1", "b": 8 }
+{ "a": 3.0, "b": 9 }
+{ "a": 1, "b": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.6.adm
new file mode 100644
index 0000000..86c085e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/order-by/order-by.6.adm
@@ -0,0 +1,5 @@
+{ "a": "1", "b": 8 }
+{ "a": 3.0, "b": 9 }
+{ "a": 1, "b": 2 }
+{ "a": null, "b": 7 }
+{ "b": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/print-ASTERIXDB-1885/print-ASTERIXDB-1885.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/print-ASTERIXDB-1885/print-ASTERIXDB-1885.1.adm
index 5d29db0..115ccbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/print-ASTERIXDB-1885/print-ASTERIXDB-1885.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/print-ASTERIXDB-1885/print-ASTERIXDB-1885.1.adm
@@ -1,4 +1,4 @@
-{ "date": date("-2012-12-12"), "time": time("23:49:12.390Z"), "datetime": datetime("2012-12-12T00:00:00.001Z"), "duration": duration("P20Y19DT4H14M23.34S"), "interval": interval(datetime("2012-12-12T00:00:00.001Z"), datetime("2013-08-10T22:10:15.398Z")) }
-{ "time": time("04:12:12.219Z"), "datetime": datetime("1920-12-21T11:29:18.478Z"), "interval": interval(time("04:29:30.000Z"), time("07:59:59.999Z")) }
-{ "datetime": datetime("-0290-03-22T17:59:48.999Z"), "duration": duration("-P27Y148D"), "interval": interval(date("-2012-03-17"), date("2013-04-01")) }
+{ "date": date("-2012-12-12"), "time": time("23:49:12.390"), "datetime": datetime("2012-12-12T00:00:00.001"), "duration": duration("P20Y19DT4H14M23.34S"), "interval": interval(datetime("2012-12-12T00:00:00.001"), datetime("2013-08-10T22:10:15.398")) }
+{ "time": time("12:12:12.219"), "datetime": datetime("1920-12-20T23:29:18.478"), "interval": interval(time("12:29:30.000"), time("23:59:59.999")) }
+{ "datetime": datetime("-0290-03-23T00:59:48.999"), "duration": duration("-P27Y148D"), "interval": interval(date("-2012-03-17"), date("2013-04-01")) }
{ "ymduration": year-month-duration("P31Y2M"), "dtduration": day-time-duration("-P148D") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/query-ASTERIXDB-2530/query-ASTERIXDB-2530.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/query-ASTERIXDB-2530/query-ASTERIXDB-2530.1.adm
index 6dd4bdb..c7c6d38 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/query-ASTERIXDB-2530/query-ASTERIXDB-2530.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/query-ASTERIXDB-2530/query-ASTERIXDB-2530.1.adm
@@ -1,2 +1,2 @@
-{ "res": datetime("2012-11-11T11:11:11.000Z") }
-{ "res": date("2013-11-11") }
\ No newline at end of file
+{ "res": datetime("2012-11-11T11:11:11.000") }
+{ "res": date("2013-11-11") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/round-half-up/round-half-up.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/round-half-up/round-half-up.1.adm
new file mode 100644
index 0000000..acc2789
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/round-half-up/round-half-up.1.adm
@@ -0,0 +1,10 @@
+{ "r": 0, "v1": 10.5, "v2": -10.5 }
+{ "r": 1, "v1": 20.6, "v2": -20.6 }
+{ "r": 2, "v1": 30.5, "v2": -30.5 }
+{ "r": 3, "v1": 40.6, "v2": -40.6 }
+{ "r": 4, "v1": INF, "v2": -INF }
+{ "r": 5, "v1": -INF, "v2": INF }
+{ "r": 6, "v1": 8, "v2": -8 }
+{ "r": 7, "v1": 16, "v2": -16 }
+{ "r": 8, "v1": 32, "v2": -32 }
+{ "r": 9, "v1": 64, "v2": -64 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
index 676edeb..800859b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
\ No newline at end of file
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.2.adm
new file mode 100644
index 0000000..79d1707
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.2.adm
@@ -0,0 +1 @@
+[ true, true, true, true, true, true, true, { "a": 1, "b": "x" }, { "e": null, "c": true, "d": false, "a": 1, "b": "x" }, { "b": null, "a": 2, "c": true }, { "a": 100, "b": { "x": 4, "y": 5 } }, { "a": 100, "b": { "x": 400, "y": 500 } } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.3.adm
new file mode 100644
index 0000000..237ea95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat/object_concat.3.adm
@@ -0,0 +1 @@
+{ "test2": "val2", "test": "val" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.03.adm
new file mode 100644
index 0000000..15a0c1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.03.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "oc": { "x": { "a": 1, "b": "1" }, "y": [ { "a": 1, "b": "1" } ] } }
+{ "id": 2, "oc": { "x": { "a": 2, "b": "2" }, "y": [ { "a": 2, "b": "2" } ] } }
+{ "id": 3, "oc": { "x": { "a": 3, "b": "3" }, "y": [ { "a": 3, "b": "3" } ] } }
+{ "id": 4, "oc": { "x": { "a": 4, "b": "4" }, "y": [ { "a": 4, "b": "4" } ] } }
+{ "id": 5, "oc": { "x": { "a": 5, "b": "5" }, "y": [ { "a": 5, "b": "5" } ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.04.adm
new file mode 100644
index 0000000..4eac053
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.04.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "oc": { "a": 1, "b": "1" } }
+{ "id": 2, "oc": { "a": 2, "b": "2" } }
+{ "id": 3, "oc": { "a": 3, "b": "3" } }
+{ "id": 4, "oc": { "a": 4, "b": "4" } }
+{ "id": 5, "oc": { "a": 5, "b": "5" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.05.adm
new file mode 100644
index 0000000..2f9609f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.05.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "oc": { "x": { "a": 1, "b": "1" }, "y": [ { "a": 1, "b": "1" } ] } }
+{ "id": 2, "oc": { "x2": { "a": 2, "b": "2" }, "y2": [ { "a": 2, "b": "2" } ], "x1": { "a": 1, "b": "1" }, "y1": [ { "a": 1, "b": "1" } ], "x3": { "a": 3, "b": "3" }, "y3": [ { "a": 3, "b": "3" } ] } }
+{ "id": 3, "oc": null }
+{ "id": 4, "oc": { "x": { "a": 4, "b": "4" }, "y": [ { "a": 4, "b": "4" } ] } }
+{ "id": 5, "oc": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.06.adm
new file mode 100644
index 0000000..8decdbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.06.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "oc": { "a": 1, "b": "1" } }
+{ "id": 2, "oc": null }
+{ "id": 3, "oc": { "x3": { "a": 3, "b": "3" }, "y3": [ { "a": 3, "b": "3" } ], "x2": { "a": 2, "b": "2" }, "y2": [ { "a": 2, "b": "2" } ], "x1": { "a": 1, "b": "1" }, "y1": [ { "a": 1, "b": "1" } ] } }
+{ "id": 4, "oc": { "a": 4, "b": "4" } }
+{ "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.07.adm
new file mode 100644
index 0000000..b1f2107
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_concat_with_array/object_concat_with_array.07.adm
@@ -0,0 +1,5 @@
+{ "id": 1, "oc": null }
+{ "id": 2, "oc": null }
+{ "id": 3, "oc": null }
+{ "id": 4, "oc": { "a": 4, "b": "4", "x": { "a": 4, "b": "4" }, "y": [ { "a": 4, "b": "4" } ] } }
+{ "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
index fc9a55a..4b97741 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
\ No newline at end of file
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)", "field": { "a": 1 } } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm
index 7ff6e89..fefd4c6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_remove/object_remove.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true ], "t2": { "b": 2 }, "t3": { "a": 1 }, "t4": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NathanGiesen@211", "friends_count": 18, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 } ], "t6": [ { "tweetid": "1", "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, { "tweetid": "10", "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, { "tweetid": "11", "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, { "tweetid": "12", "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, { "tweetid": "2", "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, { "tweetid": "3", "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, { "tweetid": "4", "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, { "tweetid": "5", "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, { "tweetid": "6", "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, { "tweetid": "7", "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, { "tweetid": "8", "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, { "tweetid": "9", "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" } ], "t7": [ { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 } ], "t8": [ { "screen-name": "ChangEwing_573", "name": "Chang Ewing", "followers_count": 32136 } ] }
\ No newline at end of file
+{ "t1": [ true, true, true, true ], "t2": { "b": 2 }, "t3": { "a": 1 }, "t4": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NathanGiesen@211", "friends_count": 18, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 } ], "t6": [ { "tweetid": "1", "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, { "tweetid": "10", "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, { "tweetid": "11", "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, { "tweetid": "12", "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, { "tweetid": "2", "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, { "tweetid": "3", "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, { "tweetid": "4", "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, { "tweetid": "5", "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, { "tweetid": "6", "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, { "tweetid": "7", "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, { "tweetid": "8", "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, { "tweetid": "9", "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" } ], "t7": [ { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NilaMilliron_tw", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, { "screen-name": "OliJackson_512", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "ColineGeyer@63", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, { "screen-name": "ChangEwing_573", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, { "screen-name": "NathanGiesen@211", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 } ], "t8": [ { "screen-name": "ChangEwing_573", "name": "Chang Ewing", "followers_count": 32136 } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm
index 16c2c6d..0f1f118 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_rename/object_rename.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true ], "t2": { "a": 1, "c": 2 }, "t3": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "language": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 } ], "t6": [ { "tweetid": "1", "user_details": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } ], "t7": [ { "screen-name": "NathanGiesen@211", "language": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 } ] }
\ No newline at end of file
+{ "t1": [ true, true, true, true ], "t2": { "a": 1, "c": 2 }, "t3": { "a": 1, "b": 2 }, "t5": [ { "screen-name": "ChangEwing_573", "language": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 } ], "t6": [ { "tweetid": "1", "user_details": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } ], "t7": [ { "screen-name": "NathanGiesen@211", "language": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_replace/object_replace.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_replace/object_replace.3.adm
index cd24aa1..d800215 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_replace/object_replace.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_replace/object_replace.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true ], "t2": { "a": 2, "b": 2 }, "t3": { "a": "1", "b": "3" }, "t4": { "a": null }, "t5": { "a": 2, "b": 2 }, "t6": { "a": 2, "b": 2 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": { "en": "native" }, "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1000", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } ] }
\ No newline at end of file
+{ "t1": [ true, true, true, true, true, true ], "t2": { "a": 2, "b": 2 }, "t3": { "a": "1", "b": "3" }, "t4": { "a": null }, "t5": { "a": 2, "b": 2 }, "t6": { "a": 2, "b": 2 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": { "en": "native" }, "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1000", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_values/object_values.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_values/object_values.3.adm
index d2a50b7..a195b50 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_values/object_values.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_values/object_values.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true ], "t2": [ { "a": 3 }, [ 1, 2 ], 4 ], "t3": [ [ "ChangEwing_573", "en", 182, 394, "Chang Ewing", 32136 ] ], "t4": [ [ "1", { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, point("47.44,80.65"), datetime("2008-04-26T10:10:00.000Z"), {{ "t-mobile", "customization" }}, " love t-mobile its customization is good:)" ] ], "t5": [ [ "NathanGiesen@211", "en", 39339, 473, "Nathan Giesen", 49416 ] ] }
\ No newline at end of file
+{ "t1": [ true, true, true ], "t2": [ { "a": 3 }, [ 1, 2 ], 4 ], "t3": [ [ "ChangEwing_573", "en", 182, 394, "Chang Ewing", 32136 ] ], "t4": [ [ "1", { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, point("47.44,80.65"), datetime("2008-04-26T10:10:00.000"), {{ "t-mobile", "customization" }}, " love t-mobile its customization is good:)" ] ], "t5": [ [ "NathanGiesen@211", "en", 39339, 473, "Nathan Giesen", 49416 ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/pairs/pairs.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/pairs/pairs.3.adm
index 41715ad..874f3b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/pairs/pairs.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/pairs/pairs.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true ], "t2": [ [ ], [ ], [ ], [ ], [ [ "a", 3 ] ] ], "t3": [ [ "object", { "a": { "b": { "c": 3 } } } ], [ "a", { "b": { "c": 3 } } ], [ "b", { "c": 3 } ], [ "c", 3 ], [ "array", [ 1, 2 ] ], [ "array", 1 ], [ "array", 2 ], [ "primitive", 4 ] ], "t4": [ [ [ "screen-name", "ChangEwing_573" ], [ "lang", "en" ], [ "friends_count", 182 ], [ "statuses_count", 394 ], [ "name", "Chang Ewing" ], [ "followers_count", 32136 ] ] ], "t5": [ [ [ "tweetid", "1" ], [ "user", { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 } ], [ "screen-name", "NathanGiesen@211" ], [ "lang", "en" ], [ "friends_count", 39339 ], [ "statuses_count", 473 ], [ "name", "Nathan Giesen" ], [ "followers_count", 49416 ], [ "sender-location", point("47.44,80.65") ], [ "send-time", datetime("2008-04-26T10:10:00.000Z") ], [ "referred-topics", {{ "t-mobile", "customization" }} ], [ "referred-topics", "t-mobile" ], [ "referred-topics", "customization" ], [ "message-text", " love t-mobile its customization is good:)" ] ] ] }
+{ "t1": [ true, true, true ], "t2": [ [ ], [ ], [ ], [ ], [ [ "a", 3 ] ] ], "t3": [ [ "object", { "a": { "b": { "c": 3 } } } ], [ "a", { "b": { "c": 3 } } ], [ "b", { "c": 3 } ], [ "c", 3 ], [ "array", [ 1, 2 ] ], [ "array", 1 ], [ "array", 2 ], [ "primitive", 4 ] ], "t4": [ [ [ "screen-name", "ChangEwing_573" ], [ "lang", "en" ], [ "friends_count", 182 ], [ "statuses_count", 394 ], [ "name", "Chang Ewing" ], [ "followers_count", 32136 ] ] ], "t5": [ [ [ "tweetid", "1" ], [ "user", { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 } ], [ "screen-name", "NathanGiesen@211" ], [ "lang", "en" ], [ "friends_count", 39339 ], [ "statuses_count", 473 ], [ "name", "Nathan Giesen" ], [ "followers_count", 49416 ], [ "sender-location", point("47.44,80.65") ], [ "send-time", datetime("2008-04-26T10:10:00.000") ], [ "referred-topics", {{ "t-mobile", "customization" }} ], [ "referred-topics", "t-mobile" ], [ "referred-topics", "customization" ], [ "message-text", " love t-mobile its customization is good:)" ] ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue236/query-issue236.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue236/query-issue236.1.adm
index 65bdb8f..084683a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue236/query-issue236.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue236/query-issue236.1.adm
@@ -1 +1 @@
-{ "tweetid": "1111387810", "tweetid-copy": "1111387810", "user": { "screen-name": "TonyNapier#786", "lang": "en", "friends_count": 4241366, "statuses_count": 97, "name": "Tony Napier", "followers_count": 5984113 }, "sender-location": point("29.24,78.35"), "send-time": datetime("2011-11-24T14:24:51.000Z"), "send-time-copy": datetime("2011-11-24T14:24:51.000Z"), "referred-topics": {{ "sprint", "wireless" }}, "message-text": " love sprint its wireless is mind-blowing:)" }
+{ "tweetid": "1111387810", "tweetid-copy": "1111387810", "user": { "screen-name": "TonyNapier#786", "lang": "en", "friends_count": 4241366, "statuses_count": 97, "name": "Tony Napier", "followers_count": 5984113 }, "sender-location": point("29.24,78.35"), "send-time": datetime("2011-11-24T14:24:51.000"), "send-time-copy": datetime("2011-11-24T14:24:51.000"), "referred-topics": {{ "sprint", "wireless" }}, "message-text": " love sprint its wireless is mind-blowing:)" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
index 307d3d8..9494045 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
@@ -1 +1 @@
-{ "id": 13, "name": "Nancy", "age": 32.5, "salary": 12.0, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
+{ "id": 13, "name": "Nancy", "age": 32.5, "salary": 12.0, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000"), "datetime": datetime("-1951-12-27T12:20:30.000"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
index 307d3d8..9494045 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
@@ -1 +1 @@
-{ "id": 13, "name": "Nancy", "age": 32.5, "salary": 12.0, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
+{ "id": 13, "name": "Nancy", "age": 32.5, "salary": 12.0, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000"), "datetime": datetime("-1951-12-27T12:20:30.000"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
index 6466feb..d43ff11 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
@@ -1,10 +1,10 @@
-{ "tweetid": "1", "tweetid-copy": "1", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstein", "followers_count": 3311368 }, "sender-location": point("42.13,80.43"), "send-time": datetime("2005-12-05T21:06:41.000Z"), "send-time-copy": datetime("2005-12-05T21:06:41.000Z"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " love samsung the plan is amazing" }
-{ "tweetid": "10", "tweetid-copy": "10", "user": { "screen-name": "Rolldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstful", "followers_count": 3311368 }, "sender-location": point("46.94,93.98"), "send-time": datetime("2011-04-07T14:08:46.000Z"), "send-time-copy": datetime("2011-04-07T14:08:46.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is good" }
-{ "tweetid": "2", "tweetid-copy": "2", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "David Eckhardstein", "followers_count": 3311368 }, "sender-location": point("28.86,70.44"), "send-time": datetime("2007-08-15T06:44:17.000Z"), "send-time-copy": datetime("2007-08-15T06:44:17.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint its voice-clarity is mind-blowing" }
-{ "tweetid": "3", "tweetid-copy": "3", "user": { "screen-name": "RollandEckhard#500", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Hetfield", "followers_count": 3311368 }, "sender-location": point("39.84,86.48"), "send-time": datetime("2008-12-24T00:07:04.000Z"), "send-time-copy": datetime("2008-12-24T00:07:04.000Z"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is terrible:(" }
-{ "tweetid": "4", "tweetid-copy": "4", "user": { "screen-name": "RollandEckhardstein#221", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstinz", "followers_count": 3311368 }, "sender-location": point("27.67,87.32"), "send-time": datetime("2007-02-05T16:39:13.000Z"), "send-time-copy": datetime("2007-02-05T16:39:13.000Z"), "referred-topics": {{ "t-mobile", "customer-service" }}, "message-text": " love t-mobile its customer-service is mind-blowing" }
-{ "tweetid": "5", "tweetid-copy": "5", "user": { "screen-name": "RollandEcstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardst", "followers_count": 3311368 }, "sender-location": point("27.3,92.77"), "send-time": datetime("2010-09-12T06:15:28.000Z"), "send-time-copy": datetime("2010-09-12T06:15:28.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile the customization is amazing:)" }
-{ "tweetid": "6", "tweetid-copy": "6", "user": { "screen-name": "Rollkhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Kirk Hammette ", "followers_count": 3311368 }, "sender-location": point("45.62,84.78"), "send-time": datetime("2012-01-23T06:23:13.000Z"), "send-time-copy": datetime("2012-01-23T06:23:13.000Z"), "referred-topics": {{ "iphone", "network" }}, "message-text": " like iphone its network is awesome:)" }
-{ "tweetid": "7", "tweetid-copy": "7", "user": { "screen-name": "andEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland khardstein", "followers_count": 3311368 }, "sender-location": point("44.12,81.46"), "send-time": datetime("2012-02-17T17:30:26.000Z"), "send-time-copy": datetime("2012-02-17T17:30:26.000Z"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " hate t-mobile the network is bad" }
-{ "tweetid": "8", "tweetid-copy": "8", "user": { "screen-name": "Rolltein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Ron Eckhardstein", "followers_count": 3311368 }, "sender-location": point("36.86,90.71"), "send-time": datetime("2009-03-12T13:18:04.000Z"), "send-time-copy": datetime("2009-03-12T13:18:04.000Z"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " dislike at&t its touch-screen is OMG" }
-{ "tweetid": "9", "tweetid-copy": "9", "user": { "screen-name": "Roldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckdstein", "followers_count": 3311368 }, "sender-location": point("29.07,97.05"), "send-time": datetime("2012-08-15T20:19:46.000Z"), "send-time-copy": datetime("2012-08-15T20:19:46.000Z"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " hate verizon its speed is bad" }
+{ "tweetid": "1", "tweetid-copy": "1", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstein", "followers_count": 3311368 }, "sender-location": point("42.13,80.43"), "send-time": datetime("2005-12-05T21:06:41.000"), "send-time-copy": datetime("2005-12-05T21:06:41.000"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " love samsung the plan is amazing" }
+{ "tweetid": "10", "tweetid-copy": "10", "user": { "screen-name": "Rolldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstful", "followers_count": 3311368 }, "sender-location": point("46.94,93.98"), "send-time": datetime("2011-04-07T14:08:46.000"), "send-time-copy": datetime("2011-04-07T14:08:46.000"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is good" }
+{ "tweetid": "2", "tweetid-copy": "2", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "David Eckhardstein", "followers_count": 3311368 }, "sender-location": point("28.86,70.44"), "send-time": datetime("2007-08-15T06:44:17.000"), "send-time-copy": datetime("2007-08-15T06:44:17.000"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint its voice-clarity is mind-blowing" }
+{ "tweetid": "3", "tweetid-copy": "3", "user": { "screen-name": "RollandEckhard#500", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Hetfield", "followers_count": 3311368 }, "sender-location": point("39.84,86.48"), "send-time": datetime("2008-12-24T00:07:04.000"), "send-time-copy": datetime("2008-12-24T00:07:04.000"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is terrible:(" }
+{ "tweetid": "4", "tweetid-copy": "4", "user": { "screen-name": "RollandEckhardstein#221", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstinz", "followers_count": 3311368 }, "sender-location": point("27.67,87.32"), "send-time": datetime("2007-02-05T16:39:13.000"), "send-time-copy": datetime("2007-02-05T16:39:13.000"), "referred-topics": {{ "t-mobile", "customer-service" }}, "message-text": " love t-mobile its customer-service is mind-blowing" }
+{ "tweetid": "5", "tweetid-copy": "5", "user": { "screen-name": "RollandEcstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardst", "followers_count": 3311368 }, "sender-location": point("27.3,92.77"), "send-time": datetime("2010-09-12T06:15:28.000"), "send-time-copy": datetime("2010-09-12T06:15:28.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile the customization is amazing:)" }
+{ "tweetid": "6", "tweetid-copy": "6", "user": { "screen-name": "Rollkhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Kirk Hammette ", "followers_count": 3311368 }, "sender-location": point("45.62,84.78"), "send-time": datetime("2012-01-23T06:23:13.000"), "send-time-copy": datetime("2012-01-23T06:23:13.000"), "referred-topics": {{ "iphone", "network" }}, "message-text": " like iphone its network is awesome:)" }
+{ "tweetid": "7", "tweetid-copy": "7", "user": { "screen-name": "andEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland khardstein", "followers_count": 3311368 }, "sender-location": point("44.12,81.46"), "send-time": datetime("2012-02-17T17:30:26.000"), "send-time-copy": datetime("2012-02-17T17:30:26.000"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " hate t-mobile the network is bad" }
+{ "tweetid": "8", "tweetid-copy": "8", "user": { "screen-name": "Rolltein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Ron Eckhardstein", "followers_count": 3311368 }, "sender-location": point("36.86,90.71"), "send-time": datetime("2009-03-12T13:18:04.000"), "send-time-copy": datetime("2009-03-12T13:18:04.000"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " dislike at&t its touch-screen is OMG" }
+{ "tweetid": "9", "tweetid-copy": "9", "user": { "screen-name": "Roldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckdstein", "followers_count": 3311368 }, "sender-location": point("29.07,97.05"), "send-time": datetime("2012-08-15T20:19:46.000"), "send-time-copy": datetime("2012-08-15T20:19:46.000"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " hate verizon its speed is bad" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
index e5ac8ab..3a7c19a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
@@ -1 +1 @@
-{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420, "hobbies": [ "basket weaving", "mud wrestling" ] }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000Z"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420, "hobbies": [ "basket weaving", "mud wrestling" ] }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.10.adm
deleted file mode 100644
index 3c1ad9d..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.10.adm
+++ /dev/null
@@ -1,32 +0,0 @@
-distribute result [$$17]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$17] <- [{"id": $$21, "name": $$22}]
- -- ASSIGN |PARTITIONED|
- project ([$$21, $$22])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$20(ASC) ] |PARTITIONED|
- order (ASC, $$20)
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$21, $$22, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
- -- ASSIGN |PARTITIONED|
- project ([$$19, $$20])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project (user, id)
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.2.json
deleted file mode 100644
index 7e235c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.2.json
+++ /dev/null
@@ -1,7 +0,0 @@
-{ "id": 8, "age": 10 }
-{ "id": 9, "age": 20 }
-{ "id": 10, "age": 30 }
-{ "id": 11, "age": 40 }
-{ "id": 12, "age": 50 }
-{ "id": 13, "age": 60 }
-{ "id": 14, "age": 70 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.5.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.5.json
deleted file mode 100644
index 88b6965..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.5.json
+++ /dev/null
@@ -1,7 +0,0 @@
-{ "p1": { "id": 8, "age": 10 }, "id": 8 }
-{ "p1": { "id": 9, "age": 20 }, "id": 9 }
-{ "p1": { "id": 10, "age": 30 }, "id": 10 }
-{ "p1": { "id": 11, "age": 40 }, "id": 11 }
-{ "p1": { "id": 12, "age": 50 }, "id": 12 }
-{ "p1": { "id": 13, "age": 60 }, "id": 13 }
-{ "p1": { "id": 14, "age": 70 }, "id": 14 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.6.adm
deleted file mode 100644
index d22d967..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.6.adm
+++ /dev/null
@@ -1,46 +0,0 @@
-distribute result [$$28]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$28])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$28] <- [{"p1": $$p1, "id": $$30}]
- -- ASSIGN |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$30(ASC) ] |PARTITIONED|
- order (ASC, $$30)
- -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$p1, $$30])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$31, $$30))
- -- HYBRID_HASH_JOIN [$$31][$$30] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
- assign [$$31] <- [$$p1.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
- project ([$$30])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$30] <- [$$p2.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset2 project (id)
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.8.adm
deleted file mode 100644
index afd33c4..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.8.adm
+++ /dev/null
@@ -1,50 +0,0 @@
-distribute result [$$29]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$29])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$29] <- [{"age": $$34, "name": $$35}]
- -- ASSIGN |PARTITIONED|
- project ([$$34, $$35])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$32(ASC) ] |PARTITIONED|
- order (ASC, $$32)
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$34, $$35, $$32])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- join (eq($$31, $$32))
- -- HYBRID_HASH_JOIN [$$31][$$32] |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
- project ([$$34, $$31])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p1] <- test.ParquetDataset project (age, id)
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- exchange
- -- HASH_PARTITION_EXCHANGE [$$32] |PARTITIONED|
- project ([$$35, $$32])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p2] <- test.ParquetDataset3 project (name, id)
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.2.json
deleted file mode 100644
index 8876910..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.2.json
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "f1": true, "f2": true }
-{ "f1": true, "f2": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.2.adm
deleted file mode 100644
index a097e74..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.2.adm
+++ /dev/null
@@ -1,24 +0,0 @@
-distribute result [$$16]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$16])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$18(ASC) ] |PARTITIONED|
- order (ASC, $$18)
- -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$16, $$18])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project (coordinates, user, id)
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.4.adm
deleted file mode 100644
index a5f69e0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.4.adm
+++ /dev/null
@@ -1,28 +0,0 @@
-distribute result [$$15]
--- DISTRIBUTE_RESULT |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$15])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$15] <- [object-concat($$18, $$19)]
- -- ASSIGN |PARTITIONED|
- project ([$$18, $$19])
- -- STREAM_PROJECT |PARTITIONED|
- exchange
- -- SORT_MERGE_EXCHANGE [$$17(ASC) ] |PARTITIONED|
- order (ASC, $$17)
- -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- project ([$$18, $$19, $$17])
- -- STREAM_PROJECT |PARTITIONED|
- assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")]
- -- ASSIGN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- data-scan []<-[$$p] <- test.ParquetDataset project (user, coordinates, id)
- -- DATASOURCE_SCAN |PARTITIONED|
- exchange
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- empty-tuple-source
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.2.json
deleted file mode 100644
index 53f2518..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.2.json
+++ /dev/null
@@ -1,2 +0,0 @@
-{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "entities": { "urls": [ { "display_url": "string", "expanded_url": "string", "indices": [ 1 ], "url": "string" } ], "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "0000000", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
-{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "11111111111111111111", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.2.json
deleted file mode 100644
index d8263ee..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.2.json
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.3.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.3.json
deleted file mode 100644
index d8263ee..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.3.json
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.4.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.4.json
deleted file mode 100644
index d8263ee..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.4.json
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.2.json
deleted file mode 100644
index 6fb86be..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.2.json
+++ /dev/null
@@ -1,2 +0,0 @@
-2
-301
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.1.adm
new file mode 100644
index 0000000..4746c44
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.1.adm
@@ -0,0 +1,2 @@
+{ "items": [ 1, 2 ] }
+{ "items": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.2.adm
new file mode 100644
index 0000000..01c0fff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.2.adm
@@ -0,0 +1,2 @@
+{ "items": [ 1, 2 ], "other_items": [ 1, 2, 3 ] }
+{ "items": [ 1, 2, 3 ], "other_items": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.adm
new file mode 100644
index 0000000..4f412c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/query-ASTERIXDB-2947/query-ASTERIXDB-2947.3.adm
@@ -0,0 +1,2 @@
+{ "yy": [ { "y": "A" }, { "y": "B" }, { "y": "C" } ], "x": 1 }
+{ "yy": [ { "y": "G" }, { "y": "H" }, { "y": "I" } ], "x": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/add_replica/add_replica.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/add_replica/add_replica.2.adm
index e48cbcf..5f87a0c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/add_replica/add_replica.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/add_replica/add_replica.2.adm
@@ -2,6 +2,7 @@
"partition" : 0,
"replicas" : [ {
"location" : "127.0.0.1:2002",
+ "nodeId" : "asterix_nc2",
"status" : "IN_SYNC"
} ]
} ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm
index a6afbb4..677a9f0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm
@@ -2,6 +2,7 @@
"partition" : 2,
"replicas" : [ {
"location" : "127.0.0.1:2001",
+ "nodeId" : "asterix_nc1",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm
index 53648a2..6c227fc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm
@@ -2,6 +2,7 @@
"partition" : 3,
"replicas" : [ {
"location" : "127.0.0.1:2001",
+ "nodeId" : "asterix_nc1",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm
index 7e92c87..a34a57a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm
@@ -2,6 +2,7 @@
"partition" : 0,
"replicas" : [ {
"location" : "127.0.0.1:2002",
+ "nodeId" : "asterix_nc2",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm
index a6afbb4..677a9f0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm
@@ -2,6 +2,7 @@
"partition" : 2,
"replicas" : [ {
"location" : "127.0.0.1:2001",
+ "nodeId" : "asterix_nc1",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm
index 53648a2..6c227fc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm
@@ -2,6 +2,7 @@
"partition" : 3,
"replicas" : [ {
"location" : "127.0.0.1:2001",
+ "nodeId" : "asterix_nc1",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.2.adm
index 7e92c87..a34a57a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.2.adm
@@ -2,6 +2,7 @@
"partition" : 0,
"replicas" : [ {
"location" : "127.0.0.1:2002",
+ "nodeId" : "asterix_nc2",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/release_partition/release_partition.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/release_partition/release_partition.2.adm
index e48cbcf..c34bcbc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/release_partition/release_partition.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/release_partition/release_partition.2.adm
@@ -1,7 +1,8 @@
[ {
- "partition" : 0,
+ "partition" : 1,
"replicas" : [ {
"location" : "127.0.0.1:2002",
+ "nodeId" : "asterix_nc2",
"status" : "IN_SYNC"
} ]
} ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-cleanjson/alltypes_01.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-cleanjson/alltypes_01.1.json
index dd319a4..707f9a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-cleanjson/alltypes_01.1.json
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-cleanjson/alltypes_01.1.json
@@ -1,2 +1 @@
-[ { "id": 10, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": 125, "int16": 32765, "int32": 294967295, "int64": 1700000000000000000, "unorderedList": [ "reading", "writing" ], "orderedList": [ "Brad", "Scott" ], "record": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "date": "-2011-01-27", "time": "12:20:30.000Z", "datetime": "-1951-12-27T12:20:30.000Z", "duration": "P10Y11M12DT10H50M30S", "point": [41.0, 44.0], "point3d": [44.0, 13.0, 41.0], "line": [ [10.1, 11.1], [10.2, 11.2] ], "rectangle": [ [5.1, 11.8], [87.6, 15.6548] ], "polygon": [ [1.2, 1.3], [2.1, 2.5], [3.5, 3.6], [4.6, 4.8] ], "circle": [ [10.1, 11.1], 10.2 ], "binary": "ABCDEF0123456789", "uuid": "5c848e5c-6b6a-498f-8452-8847a2957421" }
- ]
+{ "id": 10, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": 125, "int16": 32765, "int32": 294967295, "int64": 1700000000000000000, "unorderedList": [ "reading", "writing" ], "orderedList": [ "Brad", "Scott" ], "record": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "date": "-2011-01-27", "time": "12:20:30.000", "datetime": "-1951-12-27T12:20:30.000", "duration": "P10Y11M12DT10H50M30S", "point": [ 41.0, 44.0 ], "point3d": [ 44.0, 13.0, 41.0 ], "line": [ [ 10.1, 11.1 ], [ 10.2, 11.2 ] ], "rectangle": [ [ 5.1, 11.8 ], [ 87.6, 15.6548 ] ], "polygon": [ [ 1.2, 1.3 ], [ 2.1, 2.5 ], [ 3.5, 3.6 ], [ 4.6, 4.8 ] ], "circle": [ [ 10.1, 11.1 ], 10.2 ], "binary": "ABCDEF0123456789", "uuid": "5c848e5c-6b6a-498f-8452-8847a2957421" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-cleanjson/alltypes_01.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-cleanjson/alltypes_01.2.json
new file mode 100644
index 0000000..aeb5ce5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-cleanjson/alltypes_01.2.json
@@ -0,0 +1 @@
+{ "t1_array_of_unknown": [ null, null ], "t2_multiset_of_unknown": [ null, null ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessadmjson/alltypes_01.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessadmjson/alltypes_01.1.json
new file mode 100644
index 0000000..62acb55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessadmjson/alltypes_01.1.json
@@ -0,0 +1 @@
+{ "id": 10, "string": ":Nancy", "float": "0B:1107427328", "double": "0C:-4566801342117107042", "boolean": true, "int8": "01:125", "int16": "02:32765", "int32": "03:294967295", "int64": 1700000000000000000, "unorderedList": [ ":reading", ":writing" ], "orderedList": [ ":Brad", ":Scott" ], "record": { "number": 8389, "street": ":Hill St.", "city": ":Mountain View" }, "date": "11:-1454004", "time": "12:44430000", "datetime": "10:-123703587570000", "duration": "13:131:1075830000", "point": "14:4630967054332067840:4631389266797133824", "point3d": "15:4631389266797133824:4623507967449235456:4630967054332067840", "line": "1E:4621875412584313651:4622438362537734963:4621931707579655782:4622494657533077094", "rectangle": "21:4617428107952285286:4622832427505129882:4635864718926833254:4625002486985578355", "polygon": "1F:4:4608083138725491507:4608533498688228557:4611911198408756429:4612811918334230528:4615063718147915776:4615288898129284301:4616865157998863974:4617090337980232499", "circle": "20:4621875412584313651:4622438362537734963:4621931707579655782", "binary": "09:q83vASNFZ4k=", "uuid": "26:5c848e5c-6b6a-498f-8452-8847a2957421" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessadmjson/alltypes_01.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessadmjson/alltypes_01.2.json
new file mode 100644
index 0000000..3510c7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessadmjson/alltypes_01.2.json
@@ -0,0 +1 @@
+{ "t1_array_of_unknown": [ null, "0E" ], "t2_multiset_of_unknown": [ null, "0E" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessjson/alltypes_01.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessjson/alltypes_01.1.json
index 2e99438..a4148c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessjson/alltypes_01.1.json
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessjson/alltypes_01.1.json
@@ -1,2 +1 @@
-[ { "id": { "int64": 10 }, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": { "int8": 125 }, "int16": { "int16": 32765 }, "int32": { "int32": 294967295 }, "int64": { "int64": 1700000000000000000 }, "unorderedList": { "unorderedlist": [ "reading", "writing" ] }, "orderedList": { "orderedlist": [ "Brad", "Scott" ] }, "record": { "number": { "int64": 8389 }, "street": "Hill St.", "city": "Mountain View" }, "date": { "date": "-2011-01-27" }, "time": { "time": "12:20:30.000Z" }, "datetime": { "datetime": "-1951-12-27T12:20:30.000Z" }, "duration": { "duration": "P10Y11M12DT10H50M30S" }, "point": { "point": [41.0, 44.0] }, "point3d": { "point3d": [44.0, 13.0, 41.0] }, "line": { "line": [ { "point": [10.1, 11.1] }, { "point": [10.2, 11.2] } ] }, "rectangle": { "rectangle": [{ "point": [5.1, 11.8] }, { "point": [87.6, 15.6548] } ] }, "polygon": { "polygon": [{ "point": [1.2, 1.3] },{ "point": [2.1, 2.5] },{ "point": [3.5, 3.6] },{ "point": [4.6, 4.8] }] }, "circle": { "circle": [ { "point": [10.1, 11.1] }, 10.2 ] }, "binary": "ABCDEF0123456789", "uuid": "5c848e5c-6b6a-498f-8452-8847a2957421" }
- ]
+{ "id": { "int64": 10 }, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": { "int8": 125 }, "int16": { "int16": 32765 }, "int32": { "int32": 294967295 }, "int64": { "int64": 1700000000000000000 }, "unorderedList": { "unorderedlist": [ "reading", "writing" ] }, "orderedList": { "orderedlist": [ "Brad", "Scott" ] }, "record": { "number": { "int64": 8389 }, "street": "Hill St.", "city": "Mountain View" }, "date": { "date": "-2011-01-27" }, "time": { "time": "12:20:30.000" }, "datetime": { "datetime": "-1951-12-27T12:20:30.000" }, "duration": { "duration": "P10Y11M12DT10H50M30S" }, "point": { "point": [ 41.0, 44.0 ] }, "point3d": { "point3d": [ 44.0, 13.0, 41.0 ] }, "line": { "line": [ { "point": [ 10.1, 11.1 ] }, { "point": [ 10.2, 11.2 ] } ] }, "rectangle": { "rectangle": [ { "point": [ 5.1, 11.8 ] }, { "point": [ 87.6, 15.6548 ] } ] }, "polygon": { "polygon": [ { "point": [ 1.2, 1.3 ] }, { "point": [ 2.1, 2.5 ] }, { "point": [ 3.5, 3.6 ] }, { "point": [ 4.6, 4.8 ] } ] }, "circle": { "circle": [ { "point": [ 10.1, 11.1 ] }, 10.2 ] }, "binary": "ABCDEF0123456789", "uuid": "5c848e5c-6b6a-498f-8452-8847a2957421" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessjson/alltypes_01.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessjson/alltypes_01.2.json
new file mode 100644
index 0000000..f0a1c92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01-losslessjson/alltypes_01.2.json
@@ -0,0 +1 @@
+{ "t1_array_of_unknown": { "orderedlist": [ null, null ] }, "t2_multiset_of_unknown": { "unorderedlist": [ null, null ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01/alltypes_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01/alltypes_01.1.adm
index 2e678ca..e7a6ab1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01/alltypes_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01/alltypes_01.1.adm
@@ -1 +1 @@
-{ "id": 10, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": 125, "int16": 32765, "int32": 294967295, "int64": 1700000000000000000, "unorderedList": {{ "reading", "writing" }}, "orderedList": [ "Brad", "Scott" ], "record": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "date": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "point": point("41.0,44.0"), "point3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "rectangle": rectangle("5.1,11.8 87.6,15.6548"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "binary": hex("ABCDEF0123456789"), "uuid": uuid("5c848e5c-6b6a-498f-8452-8847a2957421") }
+{ "id": 10, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": 125, "int16": 32765, "int32": 294967295, "int64": 1700000000000000000, "unorderedList": {{ "reading", "writing" }}, "orderedList": [ "Brad", "Scott" ], "record": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "date": date("-2011-01-27"), "time": time("12:20:30.000"), "datetime": datetime("-1951-12-27T12:20:30.000"), "duration": duration("P10Y11M12DT10H50M30S"), "point": point("41.0,44.0"), "point3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "rectangle": rectangle("5.1,11.8 87.6,15.6548"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "binary": hex("ABCDEF0123456789"), "uuid": uuid("5c848e5c-6b6a-498f-8452-8847a2957421") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01/alltypes_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01/alltypes_01.2.adm
new file mode 100644
index 0000000..abea200
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_01/alltypes_01.2.adm
@@ -0,0 +1 @@
+{ "t1_array_of_unknown": [ null, null ], "t2_multiset_of_unknown": {{ null, null }} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_02/alltypes_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_02/alltypes_02.1.adm
index 2e678ca..e7a6ab1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_02/alltypes_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/alltypes_02/alltypes_02.1.adm
@@ -1 +1 @@
-{ "id": 10, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": 125, "int16": 32765, "int32": 294967295, "int64": 1700000000000000000, "unorderedList": {{ "reading", "writing" }}, "orderedList": [ "Brad", "Scott" ], "record": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "date": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "point": point("41.0,44.0"), "point3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "rectangle": rectangle("5.1,11.8 87.6,15.6548"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "binary": hex("ABCDEF0123456789"), "uuid": uuid("5c848e5c-6b6a-498f-8452-8847a2957421") }
+{ "id": 10, "string": "Nancy", "float": 32.5, "double": -2013.5938237483274, "boolean": true, "int8": 125, "int16": 32765, "int32": 294967295, "int64": 1700000000000000000, "unorderedList": {{ "reading", "writing" }}, "orderedList": [ "Brad", "Scott" ], "record": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "date": date("-2011-01-27"), "time": time("12:20:30.000"), "datetime": datetime("-1951-12-27T12:20:30.000"), "duration": duration("P10Y11M12DT10H50M30S"), "point": point("41.0,44.0"), "point3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "rectangle": rectangle("5.1,11.8 87.6,15.6548"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "binary": hex("ABCDEF0123456789"), "uuid": uuid("5c848e5c-6b6a-498f-8452-8847a2957421") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/temp_types_01/temp_types_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/temp_types_01/temp_types_01.1.adm
index fff02aa..a2c0d62 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/temp_types_01/temp_types_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/scan/temp_types_01/temp_types_01.1.adm
@@ -1 +1 @@
-{ "id": 10, "date": date("2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("1951-12-27T12:20:30.000Z"), "duration": duration("P30Y10M15DT10H30M50S"), "date2": date("2011-01-27"), "time2": time("17:20:30.999Z"), "datetime2": datetime("1951-12-27T21:05:30.250Z"), "duration2": duration("P10M15DT10H50S"), "date3": date("-2011-01-27"), "time3": time("12:20:30.999Z"), "datetime3": datetime("-1951-12-27T20:35:30.250Z"), "duration3": duration("-PT50S"), "date4": date("-2011-01-27"), "time4": time("12:20:30.000Z"), "datetime4": datetime("-1951-12-27T12:20:30.250Z"), "duration4": duration("-P10M"), "date5": date("2011-01-27"), "time5": time("12:20:30.000Z"), "datetime5": datetime("1951-12-27T12:20:30.000Z"), "duration5": duration("P30Y10M15DT10H30M50S") }
+{ "id": 10, "date": date("2011-01-27"), "time": time("12:20:30.000"), "datetime": datetime("1951-12-27T12:20:30.000"), "duration": duration("P30Y10M15DT10H30M50S"), "date2": date("2011-01-27"), "time2": time("12:20:30.999"), "datetime2": datetime("1951-12-27T12:20:30.250"), "duration2": duration("P10M15DT10H50S"), "date3": date("-2011-01-27"), "time3": time("12:20:30.999"), "datetime3": datetime("-1951-12-27T12:20:30.250"), "duration3": duration("-PT50S"), "date4": date("-2011-01-27"), "time4": time("12:20:30.000"), "datetime4": datetime("-1951-12-27T12:20:30.250"), "duration4": duration("-P10M"), "date5": date("2011-01-27"), "time5": time("12:20:30.000"), "datetime5": datetime("1951-12-27T12:20:30.000"), "duration5": duration("P30Y10M15DT10H30M50S") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.10.adm
index 5b2e7d4..15fa0e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.10.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.10.adm
@@ -1,9 +1,9 @@
-datetime("2006-02-12T17:09:34.000Z")
-datetime("2007-03-11T16:10:24.000Z")
-datetime("2009-02-04T13:34:18.000Z")
-datetime("2010-08-11T08:31:39.000Z")
-datetime("2012-01-12T12:31:39.000Z")
-datetime("2012-01-12T18:31:39.000Z")
-datetime("2013-01-12T12:31:39.000Z")
-datetime("2014-02-07T12:29:39.000Z")
-datetime("2014-02-07T13:32:23.000Z")
\ No newline at end of file
+datetime("2006-02-12T17:09:34.000")
+datetime("2007-03-11T16:10:24.000")
+datetime("2009-02-04T13:34:18.000")
+datetime("2010-08-11T08:31:39.000")
+datetime("2012-01-12T12:31:39.000")
+datetime("2012-01-12T18:31:39.000")
+datetime("2013-01-12T12:31:39.000")
+datetime("2014-02-07T12:29:39.000")
+datetime("2014-02-07T13:32:23.000")
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.8.adm
index bd7179f..f32d0de 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sorting/range_hint/range_hint.8.adm
@@ -1,9 +1,9 @@
-time("02:16:11.300Z")
-time("03:13:10.400Z")
-time("08:24:10.800Z")
-time("09:28:10.200Z")
-time("11:23:20.000Z")
-time("12:37:10.000Z")
-time("16:54:43.123Z")
-time("19:38:23.765Z")
-time("23:12:23.938Z")
\ No newline at end of file
+time("02:16:11.300")
+time("03:13:10.400")
+time("08:24:10.800")
+time("09:28:10.200")
+time("11:23:20.000")
+time("12:37:10.000")
+time("16:54:43.123")
+time("19:38:23.765")
+time("23:12:23.938")
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.adm
new file mode 100644
index 0000000..255c5f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.adm
@@ -0,0 +1 @@
+{ "$1": 34 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.adm
new file mode 100644
index 0000000..255c5f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.adm
@@ -0,0 +1 @@
+{ "$1": 34 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_projection_check/spatial_join_projection_check.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_projection_check/spatial_join_projection_check.1.adm
new file mode 100644
index 0000000..3272f31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_projection_check/spatial_join_projection_check.1.adm
@@ -0,0 +1,8 @@
+{ "name": "r1" }
+{ "name": "r1" }
+{ "name": "r1" }
+{ "name": "r1" }
+{ "name": "r2" }
+{ "name": "r2" }
+{ "name": "r2" }
+{ "name": "r2" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.adm
new file mode 100644
index 0000000..255c5f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.adm
@@ -0,0 +1 @@
+{ "$1": 34 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.adm
new file mode 100644
index 0000000..190a8f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.adm
@@ -0,0 +1 @@
+{ "$1": 104 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.1.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.1.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.2.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.2.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.3.adm
new file mode 100644
index 0000000..3237a60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.3.adm
@@ -0,0 +1,3 @@
+0
+1
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.4.adm
new file mode 100644
index 0000000..3237a60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/in_non_list_01/in_non_list_01.4.adm
@@ -0,0 +1,3 @@
+0
+1
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm
new file mode 100644
index 0000000..5e02a06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm
@@ -0,0 +1,13 @@
+{ "id": 1, "ai": 1 }
+{ "id": 2, "ai": 1 }
+{ "id": 2, "ai": 2 }
+{ "id": 3, "ai": null }
+{ "id": 4, "ai": null }
+{ "id": 5, "ai": null }
+{ "id": 6, "ai": 1 }
+{ "id": 6, "ai": 2 }
+{ "id": 6, "ai": 3 }
+{ "id": 7 }
+{ "id": 7, "ai": null }
+{ "id": 7, "ai": 1 }
+{ "id": 7, "ai": 4 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm
new file mode 100644
index 0000000..d0a8bd8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm
@@ -0,0 +1,13 @@
+{ "id": 1, "pi": 1, "ai": 1 }
+{ "id": 2, "pi": 1, "ai": 1 }
+{ "id": 2, "pi": 2, "ai": 2 }
+{ "id": 3, "pi": null, "ai": null }
+{ "id": 4, "pi": null, "ai": null }
+{ "id": 5, "pi": null, "ai": null }
+{ "id": 6, "pi": 1, "ai": 1 }
+{ "id": 6, "pi": 2, "ai": 2 }
+{ "id": 6, "pi": 3, "ai": 3 }
+{ "id": 7, "pi": 2 }
+{ "id": 7, "pi": 3, "ai": null }
+{ "id": 7, "pi": 1, "ai": 1 }
+{ "id": 7, "pi": 4, "ai": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/select_star_01/select_star_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/select_star_01/select_star_01.1.adm
new file mode 100644
index 0000000..7c1ec37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/select_star_01/select_star_01.1.adm
@@ -0,0 +1,2 @@
+{ "x": 1, "y": -1 }
+{ "x": 2, "y": -2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.1.adm
new file mode 100644
index 0000000..b56511d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.1.adm
@@ -0,0 +1,3 @@
+{ "r1": 1, "r3": 1 }
+{ "r1": 2, "r3": 2 }
+{ "r1": 3, "r3": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.10.adm
new file mode 100644
index 0000000..b56511d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.10.adm
@@ -0,0 +1,3 @@
+{ "r1": 1, "r3": 1 }
+{ "r1": 2, "r3": 2 }
+{ "r1": 3, "r3": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.2.adm
new file mode 100644
index 0000000..ccb7162
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.2.adm
@@ -0,0 +1,3 @@
+{ "r1": 1, "r3": true }
+{ "r1": 2, "r3": false }
+{ "r1": 3, "r3": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.3.adm
new file mode 100644
index 0000000..572906d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.3.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 2 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.4.adm
new file mode 100644
index 0000000..b691ed70d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.4.adm
@@ -0,0 +1,2 @@
+{ "r1": [ 1 ], "x": false }
+{ "r1": [ 2, 3 ], "x": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.5.adm
new file mode 100644
index 0000000..2b2c0bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.5.adm
@@ -0,0 +1,2 @@
+{ "k": 1 }
+{ "k": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.6.adm
new file mode 100644
index 0000000..9abbfb0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.6.adm
@@ -0,0 +1,3 @@
+{ "r1": 1, "r2": 1 }
+{ "r1": 2, "r2": 1 }
+{ "r1": 3, "r2": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.7.adm
new file mode 100644
index 0000000..2f0a933
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.7.adm
@@ -0,0 +1,3 @@
+{ "r1": 1, "r3": [ { "$2": 1 } ], "r4": [ { "$1": 2 } ] }
+{ "r1": 2, "r3": [ { "$2": 2 } ], "r4": [ { "$1": 2 } ] }
+{ "r1": 3, "r3": [ { "$2": 3 } ], "r4": [ { "$1": 2 } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.8.adm
new file mode 100644
index 0000000..69413bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.8.adm
@@ -0,0 +1,3 @@
+{ "r1": 1, "r3": [ 1 ] }
+{ "r1": 2, "r3": [ 2 ] }
+{ "r1": 3, "r3": [ 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.9.adm
new file mode 100644
index 0000000..b56511d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_01_scalar/subquery_coercion_01_scalar.9.adm
@@ -0,0 +1,3 @@
+{ "r1": 1, "r3": 1 }
+{ "r1": 2, "r3": 2 }
+{ "r1": 3, "r3": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.1.adm
new file mode 100644
index 0000000..29cc57e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.1.adm
@@ -0,0 +1,2 @@
+{ "r1": 1 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.10.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.10.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.11.adm
new file mode 100644
index 0000000..6280446
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.11.adm
@@ -0,0 +1 @@
+{ "cnt": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.12.adm
new file mode 100644
index 0000000..29cc57e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.12.adm
@@ -0,0 +1,2 @@
+{ "r1": 1 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.13.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.13.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.14.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.14.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.15.adm
new file mode 100644
index 0000000..572906d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.15.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 2 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.2.adm
new file mode 100644
index 0000000..29cc57e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.2.adm
@@ -0,0 +1,2 @@
+{ "r1": 1 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.3.adm
new file mode 100644
index 0000000..29cc57e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.3.adm
@@ -0,0 +1,2 @@
+{ "r1": 1 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.4.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.4.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.5.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.5.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.6.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.6.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.7.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.7.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.8.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.8.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.9.adm
new file mode 100644
index 0000000..7dd2c57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_03_cmp/subquery_coercion_03_cmp.9.adm
@@ -0,0 +1 @@
+{ "r1": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.1.adm
new file mode 100644
index 0000000..6372bdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.1.adm
@@ -0,0 +1,2 @@
+{ "r1": 2 }
+{ "r1": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.10.adm
new file mode 100644
index 0000000..bcec791
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.10.adm
@@ -0,0 +1,2 @@
+{ "r1": 3 }
+{ "r1": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.11.adm
new file mode 100644
index 0000000..bfd12d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.11.adm
@@ -0,0 +1,2 @@
+{ "r1": 4 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.12.adm
new file mode 100644
index 0000000..bfd12d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.12.adm
@@ -0,0 +1,2 @@
+{ "r1": 4 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.13.adm
new file mode 100644
index 0000000..bfd12d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.13.adm
@@ -0,0 +1,2 @@
+{ "r1": 4 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.14.adm
new file mode 100644
index 0000000..bfd12d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.14.adm
@@ -0,0 +1,2 @@
+{ "r1": 4 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.2.adm
new file mode 100644
index 0000000..6372bdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.2.adm
@@ -0,0 +1,2 @@
+{ "r1": 2 }
+{ "r1": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.3.adm
new file mode 100644
index 0000000..6372bdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.3.adm
@@ -0,0 +1,2 @@
+{ "r1": 2 }
+{ "r1": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.4.adm
new file mode 100644
index 0000000..6372bdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.4.adm
@@ -0,0 +1,2 @@
+{ "r1": 2 }
+{ "r1": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.5.adm
new file mode 100644
index 0000000..776e930
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.5.adm
@@ -0,0 +1,4 @@
+{ "r1": 1 }
+{ "r1": 2 }
+{ "r1": 5 }
+{ "r1": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.6.adm
new file mode 100644
index 0000000..f90118e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.6.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 3 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.7.adm
new file mode 100644
index 0000000..f90118e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.7.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 3 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.8.adm
new file mode 100644
index 0000000..f90118e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.8.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 3 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.9.adm
new file mode 100644
index 0000000..f90118e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_05_in/subquery_coercion_05_in.9.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 3 }
+{ "r1": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.1.adm
new file mode 100644
index 0000000..572906d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.1.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 2 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.2.adm
new file mode 100644
index 0000000..902bb20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.2.adm
@@ -0,0 +1,3 @@
+{ "nr1": -2, "nr3": -2 }
+{ "nr1": -3, "nr3": -3 }
+{ "nr1": -4, "nr3": -4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.3.adm
new file mode 100644
index 0000000..f93af83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_07_from/subquery_coercion_07_from.3.adm
@@ -0,0 +1,9 @@
+{ "x": 1, "y": 1 }
+{ "x": 1, "y": 2 }
+{ "x": 1, "y": 3 }
+{ "x": 2, "y": 2 }
+{ "x": 2, "y": 3 }
+{ "x": 2, "y": 4 }
+{ "x": 3, "y": 3 }
+{ "x": 3, "y": 4 }
+{ "x": 3, "y": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.1.adm
new file mode 100644
index 0000000..250a30a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.1.adm
@@ -0,0 +1,2 @@
+{ "r1": 1 }
+{ "r1": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.2.adm
new file mode 100644
index 0000000..572906d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.2.adm
@@ -0,0 +1,3 @@
+{ "r1": 1 }
+{ "r1": 2 }
+{ "r1": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.3.adm
new file mode 100644
index 0000000..6cd767c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.3.adm
@@ -0,0 +1,2 @@
+{ "r1": 2, "x": [ { "a": 2, "b": 3 } ] }
+{ "r1": 3, "x": [ { "a": 2, "b": 3 }, { "a": 3, "b": 4 } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.4.adm
new file mode 100644
index 0000000..82978d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/subquery_coercion_08_misc/subquery_coercion_08_misc.4.adm
@@ -0,0 +1 @@
+{ "r1": 1, "x": [ ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.3.adm
new file mode 100644
index 0000000..53680ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.3.adm
@@ -0,0 +1,13 @@
+{ "id": 202, "a": 4, "b": 202 }
+{ "id": 201, "a": 3, "b": 201 }
+{ "id": 200, "a": 3, "b": 200 }
+{ "id": 102, "a": 3, "b": 102 }
+{ "id": 101, "a": 2, "b": 101 }
+{ "id": 100, "a": 2, "b": 100 }
+{ "id": 7, "a": 4, "b": 7 }
+{ "id": 6, "a": 3, "b": 6 }
+{ "id": 5, "a": 3, "b": 5 }
+{ "id": 4, "a": 2, "b": 4 }
+{ "id": 3, "a": 2, "b": 3 }
+{ "id": 2, "a": 1, "b": 2 }
+{ "id": 1, "a": 1, "b": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.4.adm
new file mode 100644
index 0000000..b1ed67c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.4.adm
@@ -0,0 +1,13 @@
+{ "a": 4, "b": 202 }
+{ "a": 4, "b": 7 }
+{ "a": 3, "b": 201 }
+{ "a": 3, "b": 200 }
+{ "a": 3, "b": 102 }
+{ "a": 3, "b": 6 }
+{ "a": 3, "b": 5 }
+{ "a": 2, "b": 101 }
+{ "a": 2, "b": 100 }
+{ "a": 2, "b": 4 }
+{ "a": 2, "b": 3 }
+{ "a": 1, "b": 2 }
+{ "a": 1, "b": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.5.adm
new file mode 100644
index 0000000..60dc861
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.5.adm
@@ -0,0 +1,13 @@
+{ "id": 202, "e": 4, "f": 202 }
+{ "id": 201, "e": 3, "f": 201 }
+{ "id": 200, "e": 3, "f": 200 }
+{ "id": 102, "c": 3, "d": 102 }
+{ "id": 101, "c": 2, "d": 101 }
+{ "id": 100, "c": 2, "d": 100 }
+{ "id": 7, "a": 4, "b": 7 }
+{ "id": 6, "a": 3, "b": 6 }
+{ "id": 5, "a": 3, "b": 5 }
+{ "id": 4, "a": 2, "b": 4 }
+{ "id": 3, "a": 2, "b": 3 }
+{ "id": 2, "a": 1, "b": 2 }
+{ "id": 1, "a": 1, "b": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.6.adm
new file mode 100644
index 0000000..6ce501e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.6.adm
@@ -0,0 +1,3 @@
+{ "x": 2, "y": 3 }
+{ "x": 2, "y": 100 }
+{ "x": 3, "y": 200 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.7.adm
new file mode 100644
index 0000000..68d0470
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.7.adm
@@ -0,0 +1,3 @@
+{ "x": 1 }
+{ "x": 101 }
+{ "x": 201 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.8.adm
new file mode 100644
index 0000000..ada7b8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/union_all_01/union_all_01.8.adm
@@ -0,0 +1,2 @@
+{ "x": 0, "y": 2 }
+{ "x": 1, "y": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-literal1/string-literal1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-literal1/string-literal1.1.adm
new file mode 100644
index 0000000..1804cf8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/string-literal1/string-literal1.1.adm
@@ -0,0 +1,8 @@
+{ "i": 0, "s": "xs0" }
+{ "i": 1, "s": "xs1'ys1" }
+{ "i": 2, "s": "xs2'ys2'zs2" }
+{ "i": 3, "s": "xs3" }
+{ "i": 4, "s": "xd0" }
+{ "i": 5, "s": "xd1\"yd1" }
+{ "i": 6, "s": "xd2\"yd2\"zd2" }
+{ "i": 7, "s": "xd3" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.adm
new file mode 100644
index 0000000..a36b551
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.adm
@@ -0,0 +1 @@
+{ "s": "•\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789\n•\tabcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substr1/substr1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substr1/substr1.1.adm
index ba46923..f524208 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substr1/substr1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substr1/substr1.1.adm
@@ -1 +1 @@
-[ "g", null, "ab", "bc", "cd" ]
\ No newline at end of file
+[ "g", "ab", "ab", "bc", "cd" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substring1/substring1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substring1/substring1.1.adm
index ba46923..f524208 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substring1/substring1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/offset1/substring1/substring1.1.adm
@@ -1 +1 @@
-[ "g", null, "ab", "bc", "cd" ]
\ No newline at end of file
+[ "g", "ab", "ab", "bc", "cd" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substr1/substr1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substr1/substr1.1.adm
index 772cf57..fb038ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substr1/substr1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substr1/substr1.1.adm
@@ -1 +1 @@
-[ "g", null, "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
+[ "g", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substring1/substring1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substring1/substring1.1.adm
index 772cf57..fb038ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substring1/substring1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-1/offset1/substring1/substring1.1.adm
@@ -1 +1 @@
-[ "g", null, "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
+[ "g", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.3.adm
new file mode 100644
index 0000000..99fe65f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-3006/query-ASTERIXDB-3006.3.adm
@@ -0,0 +1,2 @@
+{ "id": 2, "a": [ { "b": 2 }, { "b": 22 }, { "b": 222 } ] }
+{ "id": 3, "a": [ { "b": 3 }, { "b": 33 }, { "b": 333 } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors/accessors.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors/accessors.1.adm
index 9689c04..a4ecb44 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors/accessors.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors/accessors.1.adm
@@ -1 +1 @@
-{ "year1": 2010, "year2": 1987, "year3": -1987, "year4": 928, "year5": 1937, "year6": -3, "year7": 9, "year8": 9, "year-null": null, "month1": 10, "month2": 11, "month3": 11, "month4": 3, "month5": 12, "month6": 1, "month7": 0, "month-null": null, "day1": 30, "day2": 19, "day3": 19, "day4": 29, "day5": 29, "day6": 634, "day7": 34, "day-null": null, "hour1": 23, "hour2": 20, "hour3": 5, "hour4": 14, "hour5": 14, "hour-null": null, "min1": 49, "min2": 3, "min3": 23, "min4": 28, "min5": 28, "min-null": null, "second1": 23, "second2": 6, "second3": 34, "second4": 48, "second5": 48, "second-null": null, "ms1": 938, "ms2": 280, "ms3": 930, "ms4": 94, "ms5": 94, "ms-null": null }
+{ "year1": 2010, "year2": 1987, "year3": -1987, "year4": 928, "year5": 1937, "year6": -3, "year7": 9, "year8": 9, "year-null": null, "month1": 10, "month2": 11, "month3": 11, "month4": 3, "month5": 12, "month6": 1, "month7": 0, "month-null": null, "day1": 30, "day2": 19, "day3": 19, "day4": 29, "day5": 29, "day6": 634, "day7": 34, "day-null": null, "hour1": 23, "hour2": 20, "hour3": 12, "hour4": 14, "hour5": 14, "hour-null": null, "min1": 49, "min2": 3, "min3": 23, "min4": 28, "min5": 28, "min-null": null, "second1": 23, "second2": 6, "second3": 34, "second4": 48, "second5": 48, "second-null": null, "ms1": 938, "ms2": 280, "ms3": 930, "ms4": 94, "ms5": 94, "ms-null": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors_interval/accessors_interval.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors_interval/accessors_interval.1.adm
index bdb8264..b15057e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors_interval/accessors_interval.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/accessors_interval/accessors_interval.1.adm
@@ -1 +1 @@
-{ "start1": date("2010-10-30"), "end1": date("2013-04-01"), "start2": time("08:09:10.234Z"), "end2": time("12:30:40.567Z"), "start3": datetime("2009-08-31T16:00:00.000Z"), "end3": datetime("2013-04-04T16:00:00.000Z") }
+{ "start1": date("2010-10-30"), "end1": date("2013-04-01"), "start2": time("08:09:10.234"), "end2": time("20:30:40.567"), "start3": datetime("2009-09-01T00:00:00.000"), "end3": datetime("2013-04-04T00:00:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_max/agg_max.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_max/agg_max.1.adm
index cd8ae70..916ff4b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_max/agg_max.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_max/agg_max.1.adm
@@ -1 +1 @@
-{ "m0": time("23:49:23.938Z"), "m1": date("2194-07-06"), "m2": datetime("2013-01-12T12:31:39.000Z"), "m3": day-time-duration("P3DT2S"), "m4": year-month-duration("P2Y5M") }
+{ "m0": time("23:49:23.938"), "m1": date("2194-07-06"), "m2": datetime("2013-01-12T12:31:39.000"), "m3": day-time-duration("P3DT2S"), "m4": year-month-duration("P2Y5M") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_min/agg_min.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_min/agg_min.1.adm
index 268d760..501b4fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_min/agg_min.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/agg_min/agg_min.1.adm
@@ -1 +1 @@
-{ "m0": time("09:28:10.900Z"), "m1": date("-1904-01-06"), "m2": datetime("2012-01-12T12:31:39.000Z"), "m3": day-time-duration("PT5.329S"), "m4": year-month-duration("P5M") }
+{ "m0": time("09:28:10.900"), "m1": date("-1904-01-06"), "m2": datetime("2012-01-12T12:31:39.000"), "m3": day-time-duration("PT5.329S"), "m4": year-month-duration("P5M") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/current_date_time_01/current_date_time_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/current_date_time_01/current_date_time_01.1.adm
new file mode 100644
index 0000000..56ec78b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/current_date_time_01/current_date_time_01.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "cnt": 1000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/current_date_time_01/current_date_time_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/current_date_time_01/current_date_time_01.2.adm
new file mode 100644
index 0000000..9f4721f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/current_date_time_01/current_date_time_01.2.adm
@@ -0,0 +1,2 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true }
+{ "t1": true, "t2": true, "t3": true, "t4": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/date_functions/date_functions.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/date_functions/date_functions.1.adm
index 5529bfd..e568b7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/date_functions/date_functions.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/date_functions/date_functions.1.adm
@@ -1 +1 @@
-{ "date1": date("2012-09-17"), "date2": date("1327-12-02"), "date3": date("2012-10-10"), "date4": date("2010-05-17"), "date5": date("1703-08-09"), "unix1": 15600, "duration1": duration("P137216D"), "duration2": duration("-P854D"), "c1": true, "c2": true, "null1": null, "nullunix1": null, "null2": null, "null3": null, "null4": null, "null5": null, "null6": null }
+{ "date1": date("2012-09-17"), "date2": date("1327-12-02"), "date3": date("2012-10-11"), "date4": date("2010-05-17"), "date5": date("1703-08-09"), "unix1": 15600, "duration1": duration("P137216D"), "duration2": duration("-P854D"), "c1": true, "c2": true, "null1": null, "nullunix1": null, "null2": null, "null3": null, "null4": null, "null5": null, "null6": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/datetime_functions/datetime_functions.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/datetime_functions/datetime_functions.1.adm
index 5ed1acf..12cda4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/datetime_functions/datetime_functions.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/datetime_functions/datetime_functions.1.adm
@@ -1 +1 @@
-{ "datetime1": datetime("1970-01-12T01:33:27.429Z"), "datetime1secs": datetime("2012-12-21T00:00:00.000Z"), "datetime2": datetime("1327-12-02T23:35:49.938Z"), "datetime3": datetime("1327-12-02T23:35:49.938Z"), "datetime4": datetime("1969-12-12T01:33:26.429Z"), "datetime5": datetime("1970-02-12T01:33:28.429Z"), "unixms1": 956007429, "unixsecs1": 1356048000, "duration1": duration("-P234526DT1H57M37.491S"), "c1": true, "null1": null, "null1secs": null, "nullunixms1": null, "nullunixsecs1": null, "null2": null, "null3": null, "null4": null, "null5": null, "null6": null, "null7": null }
+{ "datetime1": datetime("1970-01-12T01:33:27.429"), "datetime1secs": datetime("2012-12-21T00:00:00.000"), "datetime2": datetime("1327-12-02T15:35:49.938"), "datetime3": datetime("1327-12-02T15:35:49.938"), "datetime4": datetime("1969-12-12T01:33:26.429"), "datetime5": datetime("1970-02-12T01:33:28.429"), "unixms1": 956007429, "unixsecs1": 1356048000, "duration1": duration("-P234526DT9H57M37.491S"), "c1": true, "null1": null, "null1secs": null, "nullunixms1": null, "nullunixsecs1": null, "null2": null, "null3": null, "null4": null, "null5": null, "null6": null, "null7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/datetime_functions/datetime_functions.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/datetime_functions/datetime_functions.2.adm
new file mode 100644
index 0000000..8aed57e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/datetime_functions/datetime_functions.2.adm
@@ -0,0 +1,3 @@
+{ "tz": "UTC", "s1": "2021-09-15T02:01:01.000", "s2": "2021-09-15T02:01:01.000", "m1": "2021-09-15T02:01:01.000", "m2": "2021-09-15T02:01:01.000", "us1": true, "us2": true, "um1": true, "um2": true }
+{ "tz": "America/Los_Angeles", "s1": "2021-09-15T02:01:01.000", "s2": "2021-09-14T19:01:01.000", "m1": "2021-09-15T02:01:01.000", "m2": "2021-09-14T19:01:01.000", "us1": true, "us2": true, "um1": true, "um2": true }
+{ "tz": "Asia/Tokyo", "s1": "2021-09-15T02:01:01.000", "s2": "2021-09-15T11:01:01.000", "m1": "2021-09-15T02:01:01.000", "m2": "2021-09-15T11:01:01.000", "us1": true, "us2": true, "um1": true, "um2": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_week_01/day_of_week_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_week_01/day_of_week_01.2.adm
new file mode 100644
index 0000000..7b75fe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_week_01/day_of_week_01.2.adm
@@ -0,0 +1,7 @@
+{ "r": 1, "mn": 1, "mx": 1 }
+{ "r": 2, "mn": 7, "mx": 7 }
+{ "r": 3, "mn": 6, "mx": 6 }
+{ "r": 4, "mn": 5, "mx": 5 }
+{ "r": 5, "mn": 4, "mx": 4 }
+{ "r": 6, "mn": 3, "mx": 3 }
+{ "r": 7, "mn": 2, "mx": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_year_01/day_of_year_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_year_01/day_of_year_01.1.adm
new file mode 100644
index 0000000..c6f8a6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_year_01/day_of_year_01.1.adm
@@ -0,0 +1,7 @@
+{ "r": 1, "mn": 1, "mx": 1 }
+{ "r": 2, "mn": 2, "mx": 2 }
+{ "r": 3, "mn": 3, "mx": 3 }
+{ "r": 4, "mn": 4, "mx": 4 }
+{ "r": 5, "mn": 5, "mx": 5 }
+{ "r": 6, "mn": 6, "mx": 6 }
+{ "r": 7, "mn": 7, "mx": 7 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_year_01/day_of_year_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_year_01/day_of_year_01.2.adm
new file mode 100644
index 0000000..f509856
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/day_of_year_01/day_of_year_01.2.adm
@@ -0,0 +1 @@
+{ "s1": 66795, "s2": 66795 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/get_overlapping_interval/get_overlapping_interval.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/get_overlapping_interval/get_overlapping_interval.1.adm
index 35e5706..3f0583b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/get_overlapping_interval/get_overlapping_interval.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/get_overlapping_interval/get_overlapping_interval.1.adm
@@ -1 +1 @@
-{ "overlap1": interval(time("12:23:39.000Z"), time("18:27:19.000Z")), "overlap2": null, "overlap3": null, "overlap4": interval(date("2013-01-01"), date("2014-01-01")), "overlap5": interval(datetime("1989-03-04T12:23:39.000Z"), datetime("2000-10-30T18:27:19.000Z")), "overlap6": null }
+{ "overlap1": interval(time("12:23:39.000"), time("18:27:19.000")), "overlap2": null, "overlap3": null, "overlap4": interval(date("2013-01-01"), date("2014-01-01")), "overlap5": interval(datetime("1989-03-04T12:23:39.000"), datetime("2000-10-30T18:27:19.000")), "overlap6": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm
index 5d29db0..115ccbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm
@@ -1,4 +1,4 @@
-{ "date": date("-2012-12-12"), "time": time("23:49:12.390Z"), "datetime": datetime("2012-12-12T00:00:00.001Z"), "duration": duration("P20Y19DT4H14M23.34S"), "interval": interval(datetime("2012-12-12T00:00:00.001Z"), datetime("2013-08-10T22:10:15.398Z")) }
-{ "time": time("04:12:12.219Z"), "datetime": datetime("1920-12-21T11:29:18.478Z"), "interval": interval(time("04:29:30.000Z"), time("07:59:59.999Z")) }
-{ "datetime": datetime("-0290-03-22T17:59:48.999Z"), "duration": duration("-P27Y148D"), "interval": interval(date("-2012-03-17"), date("2013-04-01")) }
+{ "date": date("-2012-12-12"), "time": time("23:49:12.390"), "datetime": datetime("2012-12-12T00:00:00.001"), "duration": duration("P20Y19DT4H14M23.34S"), "interval": interval(datetime("2012-12-12T00:00:00.001"), datetime("2013-08-10T22:10:15.398")) }
+{ "time": time("12:12:12.219"), "datetime": datetime("1920-12-20T23:29:18.478"), "interval": interval(time("12:29:30.000"), time("23:59:59.999")) }
+{ "datetime": datetime("-0290-03-23T00:59:48.999"), "duration": duration("-P27Y148D"), "interval": interval(date("-2012-03-17"), date("2013-04-01")) }
{ "ymduration": year-month-duration("P31Y2M"), "dtduration": day-time-duration("-P148D") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.adm
index 2b03194..f5a0807 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds_2/insert_from_ext_ds_2.1.adm
@@ -1,3 +1,3 @@
-{ "date": date("-9971-09-24"), "time": time("11:38:17.154Z"), "datetime": datetime("1259-11-13T09:49:11.852Z"), "duration": duration("P473653Y9M4566143DT10H20M53.61S"), "year-month-duration": year-month-duration("P148233Y10M"), "day-time-duration": day-time-duration("-P7236357DT2H56M56.164S"), "date-interval": interval(date("-0255-09-06"), date("4925-05-03")), "time-interval": interval(time("12:10:45.169Z"), time("15:37:48.736Z")), "datetime-interval": interval(datetime("0534-12-08T08:20:31.487Z"), datetime("6778-02-16T22:40:21.653Z")) }
-{ "date": date("4619-11-23"), "time": time("14:29:36.786Z"), "datetime": datetime("2749-01-27T17:27:30.020Z"), "duration": duration("-P474133Y7M854630DT4H40M6.45S"), "year-month-duration": year-month-duration("P193989Y3M"), "day-time-duration": day-time-duration("P4477686DT4H49M31.87S"), "date-interval": interval(date("-9537-08-04"), date("9656-06-03")), "time-interval": interval(time("12:04:45.689Z"), time("12:41:59.002Z")), "datetime-interval": interval(datetime("-2640-10-11T17:32:15.675Z"), datetime("4104-02-01T05:59:11.902Z")) }
-{ "date": date("7986-11-25"), "time": time("12:49:39.736Z"), "datetime": datetime("-8337-01-30T15:23:07.598Z"), "duration": duration("-P184484Y7M2241423DT10H42M49.500S"), "year-month-duration": year-month-duration("-P546031Y3M"), "day-time-duration": day-time-duration("P2623386DT10H32M31.983S"), "date-interval": interval(date("-4514-05-24"), date("3337-08-26")), "time-interval": interval(time("04:16:42.321Z"), time("12:22:56.816Z")), "datetime-interval": interval(datetime("2129-12-12T13:18:35.758Z"), datetime("8647-07-01T13:10:19.691Z")) }
+{ "date": date("-9971-09-24"), "time": time("11:38:17.154"), "datetime": datetime("1259-11-13T09:49:11.852"), "duration": duration("P473653Y9M4566143DT10H20M53.61S"), "year-month-duration": year-month-duration("P148233Y10M"), "day-time-duration": day-time-duration("-P7236357DT2H56M56.164S"), "date-interval": interval(date("-0255-09-06"), date("4925-05-03")), "time-interval": interval(time("12:10:45.169"), time("15:37:48.736")), "datetime-interval": interval(datetime("0534-12-08T08:20:31.487"), datetime("6778-02-16T22:40:21.653")) }
+{ "date": date("4619-11-23"), "time": time("14:29:36.786"), "datetime": datetime("2749-01-27T17:27:30.020"), "duration": duration("-P474133Y7M854630DT4H40M6.45S"), "year-month-duration": year-month-duration("P193989Y3M"), "day-time-duration": day-time-duration("P4477686DT4H49M31.87S"), "date-interval": interval(date("-9537-08-04"), date("9656-06-03")), "time-interval": interval(time("12:04:45.689"), time("12:41:59.002")), "datetime-interval": interval(datetime("-2640-10-11T17:32:15.675"), datetime("4104-02-01T05:59:11.902")) }
+{ "date": date("7986-11-25"), "time": time("12:49:39.736"), "datetime": datetime("-8337-01-30T15:23:07.598"), "duration": duration("-P184484Y7M2241423DT10H42M49.500S"), "year-month-duration": year-month-duration("-P546031Y3M"), "day-time-duration": day-time-duration("P2623386DT10H32M31.983S"), "date-interval": interval(date("-4514-05-24"), date("3337-08-26")), "time-interval": interval(time("04:16:42.321"), time("12:22:56.816")), "datetime-interval": interval(datetime("2129-12-12T13:18:35.758"), datetime("8647-07-01T13:10:19.691")) }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin/interval_bin.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin/interval_bin.1.adm
index f791665..5f98e09 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin/interval_bin.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin/interval_bin.1.adm
@@ -1 +1 @@
-{ "bin1": interval(date("2010-01-01"), date("2011-01-01")), "bin2": interval(date("2010-01-01"), date("2011-01-01")), "bin3": interval(datetime("-1987-07-01T00:00:00.000Z"), datetime("-1986-01-01T00:00:00.000Z")), "bin4": interval(datetime("-1987-11-19T12:00:00.000Z"), datetime("-1987-11-20T00:00:00.000Z")), "bin5": interval(time("04:00:00.000Z"), time("06:00:00.000Z")), "bin6": null, "bin7": null, "bin8": null }
+{ "bin1": interval(date("2010-01-01"), date("2011-01-01")), "bin2": interval(date("2010-01-01"), date("2011-01-01")), "bin3": interval(datetime("-1987-07-01T00:00:00.000"), datetime("-1986-01-01T00:00:00.000")), "bin4": interval(datetime("-1987-11-19T12:00:00.000"), datetime("-1987-11-20T00:00:00.000")), "bin5": interval(time("12:00:00.000"), time("14:00:00.000")), "bin6": null, "bin7": null, "bin8": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_0/interval_bin_gby_0.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_0/interval_bin_gby_0.1.adm
index aa0f939..2174865 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_0/interval_bin_gby_0.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_0/interval_bin_gby_0.1.adm
@@ -1,4 +1,4 @@
-{ "tbin": interval(datetime("-1990-01-01T00:00:00.000Z"), datetime("-1970-01-01T00:00:00.000Z")), "count": 4 }
-{ "tbin": interval(datetime("-0990-01-01T00:00:00.000Z"), datetime("-0970-01-01T00:00:00.000Z")), "count": 1 }
-{ "tbin": interval(datetime("1970-01-01T00:00:00.000Z"), datetime("1990-01-01T00:00:00.000Z")), "count": 5 }
-{ "tbin": interval(datetime("2010-01-01T00:00:00.000Z"), datetime("2030-01-01T00:00:00.000Z")), "count": 2 }
+{ "tbin": interval(datetime("-1990-01-01T00:00:00.000"), datetime("-1970-01-01T00:00:00.000")), "count": 4 }
+{ "tbin": interval(datetime("-0990-01-01T00:00:00.000"), datetime("-0970-01-01T00:00:00.000")), "count": 1 }
+{ "tbin": interval(datetime("1970-01-01T00:00:00.000"), datetime("1990-01-01T00:00:00.000")), "count": 5 }
+{ "tbin": interval(datetime("2010-01-01T00:00:00.000"), datetime("2030-01-01T00:00:00.000")), "count": 2 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_1/interval_bin_gby_1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_1/interval_bin_gby_1.1.adm
index cbcc124..2aa1554 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_1/interval_bin_gby_1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_bin_gby_1/interval_bin_gby_1.1.adm
@@ -1,8 +1,8 @@
-{ "tbin": interval(time("00:20:00.000Z"), time("00:30:00.000Z")), "count": 1 }
-{ "tbin": interval(time("09:30:00.000Z"), time("09:40:00.000Z")), "count": 1 }
-{ "tbin": interval(time("17:20:00.000Z"), time("17:30:00.000Z")), "count": 1 }
-{ "tbin": interval(time("18:00:00.000Z"), time("18:10:00.000Z")), "count": 1 }
-{ "tbin": interval(time("23:20:00.000Z"), time("23:30:00.000Z")), "count": 1 }
-{ "tbin": interval(time("23:30:00.000Z"), time("23:40:00.000Z")), "count": 1 }
-{ "tbin": interval(time("23:40:00.000Z"), time("23:50:00.000Z")), "count": 5 }
-{ "tbin": interval(time("23:50:00.000Z"), time("00:00:00.000Z")), "count": 1 }
+{ "tbin": interval(time("00:20:00.000"), time("00:30:00.000")), "count": 1 }
+{ "tbin": interval(time("09:30:00.000"), time("09:40:00.000")), "count": 1 }
+{ "tbin": interval(time("17:20:00.000"), time("17:30:00.000")), "count": 1 }
+{ "tbin": interval(time("18:00:00.000"), time("18:10:00.000")), "count": 1 }
+{ "tbin": interval(time("23:20:00.000"), time("23:30:00.000")), "count": 1 }
+{ "tbin": interval(time("23:30:00.000"), time("23:40:00.000")), "count": 1 }
+{ "tbin": interval(time("23:40:00.000"), time("23:50:00.000")), "count": 5 }
+{ "tbin": interval(time("23:50:00.000"), time("00:00:00.000")), "count": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.1.adm
new file mode 100644
index 0000000..a105d6e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_join_projection_check/interval_join_projection_check.1.adm
@@ -0,0 +1,8 @@
+{ "name": "i1" }
+{ "name": "i1" }
+{ "name": "i1" }
+{ "name": "i1" }
+{ "name": "i2" }
+{ "name": "i2" }
+{ "name": "i2" }
+{ "name": "i2" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins/overlap_bins.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins/overlap_bins.1.adm
index 14f6aaea..fa097b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins/overlap_bins.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins/overlap_bins.1.adm
@@ -1 +1 @@
-{ "timebins": [ interval(time("17:00:00.000Z"), time("17:30:00.000Z")), interval(time("17:30:00.000Z"), time("18:00:00.000Z")), interval(time("18:00:00.000Z"), time("18:30:00.000Z")), interval(time("18:30:00.000Z"), time("19:00:00.000Z")) ], "datebins": [ interval(date("1980-01-01"), date("1990-01-01")), interval(date("1990-01-01"), date("2000-01-01")), interval(date("2000-01-01"), date("2010-01-01")), interval(date("2010-01-01"), date("2020-01-01")) ], "datetimebins": [ interval(datetime("1800-01-01T00:00:00.000Z"), datetime("1900-01-01T00:00:00.000Z")), interval(datetime("1900-01-01T00:00:00.000Z"), datetime("2000-01-01T00:00:00.000Z")), interval(datetime("2000-01-01T00:00:00.000Z"), datetime("2100-01-01T00:00:00.000Z")) ] }
+{ "timebins": [ interval(time("17:00:00.000"), time("17:30:00.000")), interval(time("17:30:00.000"), time("18:00:00.000")), interval(time("18:00:00.000"), time("18:30:00.000")), interval(time("18:30:00.000"), time("19:00:00.000")) ], "datebins": [ interval(date("1980-01-01"), date("1990-01-01")), interval(date("1990-01-01"), date("2000-01-01")), interval(date("2000-01-01"), date("2010-01-01")), interval(date("2010-01-01"), date("2020-01-01")) ], "datetimebins": [ interval(datetime("1800-01-01T00:00:00.000"), datetime("1900-01-01T00:00:00.000")), interval(datetime("1900-01-01T00:00:00.000"), datetime("2000-01-01T00:00:00.000")), interval(datetime("2000-01-01T00:00:00.000"), datetime("2100-01-01T00:00:00.000")) ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_0/overlap_bins_gby_0.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_0/overlap_bins_gby_0.1.adm
index 7b301a6..c6342b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_0/overlap_bins_gby_0.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_0/overlap_bins_gby_0.1.adm
@@ -1,23 +1,23 @@
-{ "tbin": interval(time("00:00:00.000Z"), time("01:30:00.000Z")), "interval": interval(time("00:27:13.432Z"), time("01:27:13.432Z")), "overlap": interval(time("00:27:13.432Z"), time("01:27:13.432Z")) }
-{ "tbin": interval(time("07:30:00.000Z"), time("09:00:00.000Z")), "interval": interval(time("07:49:23.938Z"), time("10:49:23.938Z")), "overlap": interval(time("07:49:23.938Z"), time("09:00:00.000Z")) }
-{ "tbin": interval(time("09:00:00.000Z"), time("10:30:00.000Z")), "interval": interval(time("07:49:23.938Z"), time("10:49:23.938Z")), "overlap": interval(time("09:00:00.000Z"), time("10:30:00.000Z")) }
-{ "tbin": interval(time("09:00:00.000Z"), time("10:30:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("09:35:28.039Z"), time("10:30:00.000Z")) }
-{ "tbin": interval(time("10:30:00.000Z"), time("12:00:00.000Z")), "interval": interval(time("07:49:23.938Z"), time("10:49:23.938Z")), "overlap": interval(time("10:30:00.000Z"), time("10:49:23.938Z")) }
-{ "tbin": interval(time("10:30:00.000Z"), time("12:00:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("10:30:00.000Z"), time("12:00:00.000Z")) }
-{ "tbin": interval(time("12:00:00.000Z"), time("13:30:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("12:00:00.000Z"), time("13:30:00.000Z")) }
-{ "tbin": interval(time("12:00:00.000Z"), time("13:30:00.000Z")), "interval": interval(time("12:49:23.938Z"), time("15:49:23.938Z")), "overlap": interval(time("12:49:23.938Z"), time("13:30:00.000Z")) }
-{ "tbin": interval(time("13:30:00.000Z"), time("15:00:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("13:30:00.000Z"), time("15:00:00.000Z")) }
-{ "tbin": interval(time("13:30:00.000Z"), time("15:00:00.000Z")), "interval": interval(time("12:49:23.938Z"), time("15:49:23.938Z")), "overlap": interval(time("13:30:00.000Z"), time("15:00:00.000Z")) }
-{ "tbin": interval(time("15:00:00.000Z"), time("16:30:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("15:00:00.000Z"), time("16:30:00.000Z")) }
-{ "tbin": interval(time("15:00:00.000Z"), time("16:30:00.000Z")), "interval": interval(time("12:49:23.938Z"), time("15:49:23.938Z")), "overlap": interval(time("15:00:00.000Z"), time("15:49:23.938Z")) }
-{ "tbin": interval(time("16:30:00.000Z"), time("18:00:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("16:30:00.000Z"), time("18:00:00.000Z")) }
-{ "tbin": interval(time("16:30:00.000Z"), time("18:00:00.000Z")), "interval": interval(time("17:28:13.900Z"), time("17:28:32.900Z")), "overlap": interval(time("17:28:13.900Z"), time("17:28:32.900Z")) }
-{ "tbin": interval(time("18:00:00.000Z"), time("19:30:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("18:00:00.000Z"), time("19:30:00.000Z")) }
-{ "tbin": interval(time("18:00:00.000Z"), time("19:30:00.000Z")), "interval": interval(time("18:00:00.000Z"), time("20:00:00.000Z")), "overlap": interval(time("18:00:00.000Z"), time("19:30:00.000Z")) }
-{ "tbin": interval(time("19:30:00.000Z"), time("21:00:00.000Z")), "interval": interval(time("09:35:28.039Z"), time("20:25:28.039Z")), "overlap": interval(time("19:30:00.000Z"), time("20:25:28.039Z")) }
-{ "tbin": interval(time("19:30:00.000Z"), time("21:00:00.000Z")), "interval": interval(time("18:00:00.000Z"), time("20:00:00.000Z")), "overlap": interval(time("19:30:00.000Z"), time("20:00:00.000Z")) }
-{ "tbin": interval(time("22:30:00.000Z"), time("00:00:00.000Z")), "interval": interval(time("23:22:38.000Z"), time("23:42:38.000Z")), "overlap": interval(time("23:22:38.000Z"), time("23:42:38.000Z")) }
-{ "tbin": interval(time("22:30:00.000Z"), time("00:00:00.000Z")), "interval": interval(time("23:30:00.000Z"), time("23:32:00.000Z")), "overlap": interval(time("23:30:00.000Z"), time("23:32:00.000Z")) }
-{ "tbin": interval(time("22:30:00.000Z"), time("00:00:00.000Z")), "interval": interval(time("23:49:23.938Z"), time("23:54:23.938Z")), "overlap": interval(time("23:49:23.938Z"), time("23:54:23.938Z")) }
-{ "tbin": interval(time("22:30:00.000Z"), time("00:00:00.000Z")), "interval": interval(time("23:49:33.938Z"), time("23:50:03.938Z")), "overlap": interval(time("23:49:33.938Z"), time("23:50:03.938Z")) }
-{ "tbin": interval(time("22:30:00.000Z"), time("00:00:00.000Z")), "interval": interval(time("23:58:17.038Z"), time("23:58:56.420Z")), "overlap": interval(time("23:58:17.038Z"), time("23:58:56.420Z")) }
+{ "tbin": interval(time("00:00:00.000"), time("01:30:00.000")), "interval": interval(time("00:27:13.432"), time("01:27:13.432")), "overlap": interval(time("00:27:13.432"), time("01:27:13.432")) }
+{ "tbin": interval(time("07:30:00.000"), time("09:00:00.000")), "interval": interval(time("07:49:23.938"), time("10:49:23.938")), "overlap": interval(time("07:49:23.938"), time("09:00:00.000")) }
+{ "tbin": interval(time("09:00:00.000"), time("10:30:00.000")), "interval": interval(time("07:49:23.938"), time("10:49:23.938")), "overlap": interval(time("09:00:00.000"), time("10:30:00.000")) }
+{ "tbin": interval(time("09:00:00.000"), time("10:30:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("09:35:28.039"), time("10:30:00.000")) }
+{ "tbin": interval(time("10:30:00.000"), time("12:00:00.000")), "interval": interval(time("07:49:23.938"), time("10:49:23.938")), "overlap": interval(time("10:30:00.000"), time("10:49:23.938")) }
+{ "tbin": interval(time("10:30:00.000"), time("12:00:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("10:30:00.000"), time("12:00:00.000")) }
+{ "tbin": interval(time("12:00:00.000"), time("13:30:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("12:00:00.000"), time("13:30:00.000")) }
+{ "tbin": interval(time("12:00:00.000"), time("13:30:00.000")), "interval": interval(time("12:49:23.938"), time("15:49:23.938")), "overlap": interval(time("12:49:23.938"), time("13:30:00.000")) }
+{ "tbin": interval(time("13:30:00.000"), time("15:00:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("13:30:00.000"), time("15:00:00.000")) }
+{ "tbin": interval(time("13:30:00.000"), time("15:00:00.000")), "interval": interval(time("12:49:23.938"), time("15:49:23.938")), "overlap": interval(time("13:30:00.000"), time("15:00:00.000")) }
+{ "tbin": interval(time("15:00:00.000"), time("16:30:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("15:00:00.000"), time("16:30:00.000")) }
+{ "tbin": interval(time("15:00:00.000"), time("16:30:00.000")), "interval": interval(time("12:49:23.938"), time("15:49:23.938")), "overlap": interval(time("15:00:00.000"), time("15:49:23.938")) }
+{ "tbin": interval(time("16:30:00.000"), time("18:00:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("16:30:00.000"), time("18:00:00.000")) }
+{ "tbin": interval(time("16:30:00.000"), time("18:00:00.000")), "interval": interval(time("17:28:13.900"), time("17:28:32.900")), "overlap": interval(time("17:28:13.900"), time("17:28:32.900")) }
+{ "tbin": interval(time("18:00:00.000"), time("19:30:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("18:00:00.000"), time("19:30:00.000")) }
+{ "tbin": interval(time("18:00:00.000"), time("19:30:00.000")), "interval": interval(time("18:00:00.000"), time("20:00:00.000")), "overlap": interval(time("18:00:00.000"), time("19:30:00.000")) }
+{ "tbin": interval(time("19:30:00.000"), time("21:00:00.000")), "interval": interval(time("09:35:28.039"), time("20:25:28.039")), "overlap": interval(time("19:30:00.000"), time("20:25:28.039")) }
+{ "tbin": interval(time("19:30:00.000"), time("21:00:00.000")), "interval": interval(time("18:00:00.000"), time("20:00:00.000")), "overlap": interval(time("19:30:00.000"), time("20:00:00.000")) }
+{ "tbin": interval(time("22:30:00.000"), time("00:00:00.000")), "interval": interval(time("23:22:38.000"), time("23:42:38.000")), "overlap": interval(time("23:22:38.000"), time("23:42:38.000")) }
+{ "tbin": interval(time("22:30:00.000"), time("00:00:00.000")), "interval": interval(time("23:30:00.000"), time("23:32:00.000")), "overlap": interval(time("23:30:00.000"), time("23:32:00.000")) }
+{ "tbin": interval(time("22:30:00.000"), time("00:00:00.000")), "interval": interval(time("23:49:23.938"), time("23:54:23.938")), "overlap": interval(time("23:49:23.938"), time("23:54:23.938")) }
+{ "tbin": interval(time("22:30:00.000"), time("00:00:00.000")), "interval": interval(time("23:49:33.938"), time("23:50:03.938")), "overlap": interval(time("23:49:33.938"), time("23:50:03.938")) }
+{ "tbin": interval(time("22:30:00.000"), time("00:00:00.000")), "interval": interval(time("23:58:17.038"), time("23:58:56.420")), "overlap": interval(time("23:58:17.038"), time("23:58:56.420")) }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_1/overlap_bins_gby_1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_1/overlap_bins_gby_1.1.adm
index 98ad324..5bfc81e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_1/overlap_bins_gby_1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_1/overlap_bins_gby_1.1.adm
@@ -1,11 +1,11 @@
-{ "timebin": interval(time("00:00:00.000Z"), time("01:30:00.000Z")), "count": 1, "total_ms": 3600000 }
-{ "timebin": interval(time("07:30:00.000Z"), time("09:00:00.000Z")), "count": 1, "total_ms": 4236062 }
-{ "timebin": interval(time("09:00:00.000Z"), time("10:30:00.000Z")), "count": 2, "total_ms": 8671961 }
-{ "timebin": interval(time("10:30:00.000Z"), time("12:00:00.000Z")), "count": 2, "total_ms": 6563938 }
-{ "timebin": interval(time("12:00:00.000Z"), time("13:30:00.000Z")), "count": 2, "total_ms": 7836062 }
-{ "timebin": interval(time("13:30:00.000Z"), time("15:00:00.000Z")), "count": 2, "total_ms": 10800000 }
-{ "timebin": interval(time("15:00:00.000Z"), time("16:30:00.000Z")), "count": 2, "total_ms": 8363938 }
-{ "timebin": interval(time("16:30:00.000Z"), time("18:00:00.000Z")), "count": 2, "total_ms": 5419000 }
-{ "timebin": interval(time("18:00:00.000Z"), time("19:30:00.000Z")), "count": 2, "total_ms": 10800000 }
-{ "timebin": interval(time("19:30:00.000Z"), time("21:00:00.000Z")), "count": 2, "total_ms": 5128039 }
-{ "timebin": interval(time("22:30:00.000Z"), time("00:00:00.000Z")), "count": 5, "total_ms": 1689382 }
+{ "timebin": interval(time("00:00:00.000"), time("01:30:00.000")), "count": 1, "total_ms": 3600000 }
+{ "timebin": interval(time("07:30:00.000"), time("09:00:00.000")), "count": 1, "total_ms": 4236062 }
+{ "timebin": interval(time("09:00:00.000"), time("10:30:00.000")), "count": 2, "total_ms": 8671961 }
+{ "timebin": interval(time("10:30:00.000"), time("12:00:00.000")), "count": 2, "total_ms": 6563938 }
+{ "timebin": interval(time("12:00:00.000"), time("13:30:00.000")), "count": 2, "total_ms": 7836062 }
+{ "timebin": interval(time("13:30:00.000"), time("15:00:00.000")), "count": 2, "total_ms": 10800000 }
+{ "timebin": interval(time("15:00:00.000"), time("16:30:00.000")), "count": 2, "total_ms": 8363938 }
+{ "timebin": interval(time("16:30:00.000"), time("18:00:00.000")), "count": 2, "total_ms": 5419000 }
+{ "timebin": interval(time("18:00:00.000"), time("19:30:00.000")), "count": 2, "total_ms": 10800000 }
+{ "timebin": interval(time("19:30:00.000"), time("21:00:00.000")), "count": 2, "total_ms": 5128039 }
+{ "timebin": interval(time("22:30:00.000"), time("00:00:00.000")), "count": 5, "total_ms": 1689382 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_3/overlap_bins_gby_3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_3/overlap_bins_gby_3.1.adm
index 6f26bb6..070a078 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_3/overlap_bins_gby_3.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/overlap_bins_gby_3/overlap_bins_gby_3.1.adm
@@ -1,42 +1,42 @@
-{ "timebin": interval(time("10:27:00.000Z"), time("10:28:00.000Z")), "subgroups": [ { "subgid": "Email", "item_count": 2 }, { "subgid": "Facebook", "item_count": 2 } ] }
-{ "timebin": interval(time("10:28:00.000Z"), time("10:29:00.000Z")), "subgroups": [ { "subgid": "Facebook", "item_count": 1 } ] }
-{ "timebin": interval(time("10:29:00.000Z"), time("10:30:00.000Z")), "subgroups": [ { "subgid": "Email", "item_count": 1 }, { "subgid": "Facebook", "item_count": 2 } ] }
-{ "timebin": interval(time("10:30:00.000Z"), time("10:31:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:31:00.000Z"), time("10:32:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:32:00.000Z"), time("10:33:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:33:00.000Z"), time("10:34:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:34:00.000Z"), time("10:35:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:35:00.000Z"), time("10:36:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:36:00.000Z"), time("10:37:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:37:00.000Z"), time("10:38:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:38:00.000Z"), time("10:39:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:39:00.000Z"), time("10:40:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:40:00.000Z"), time("10:41:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:41:00.000Z"), time("10:42:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:42:00.000Z"), time("10:43:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:43:00.000Z"), time("10:44:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:44:00.000Z"), time("10:45:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:45:00.000Z"), time("10:46:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:46:00.000Z"), time("10:47:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:47:00.000Z"), time("10:48:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:48:00.000Z"), time("10:49:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:49:00.000Z"), time("10:50:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:50:00.000Z"), time("10:51:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:51:00.000Z"), time("10:52:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:52:00.000Z"), time("10:53:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:53:00.000Z"), time("10:54:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:54:00.000Z"), time("10:55:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:55:00.000Z"), time("10:56:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:56:00.000Z"), time("10:57:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:57:00.000Z"), time("10:58:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:58:00.000Z"), time("10:59:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("10:59:00.000Z"), time("11:00:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:00:00.000Z"), time("11:01:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:01:00.000Z"), time("11:02:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:02:00.000Z"), time("11:03:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:03:00.000Z"), time("11:04:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:04:00.000Z"), time("11:05:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:05:00.000Z"), time("11:06:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:06:00.000Z"), time("11:07:00.000Z")), "subgroups": [ ] }
-{ "timebin": interval(time("11:07:00.000Z"), time("11:08:00.000Z")), "subgroups": [ { "subgid": "Email", "item_count": 2 } ] }
-{ "timebin": interval(time("11:08:00.000Z"), time("11:09:00.000Z")), "subgroups": [ ] }
+{ "timebin": interval(time("10:27:00.000"), time("10:28:00.000")), "subgroups": [ { "subgid": "Email", "item_count": 2 }, { "subgid": "Facebook", "item_count": 2 } ] }
+{ "timebin": interval(time("10:28:00.000"), time("10:29:00.000")), "subgroups": [ { "subgid": "Facebook", "item_count": 1 } ] }
+{ "timebin": interval(time("10:29:00.000"), time("10:30:00.000")), "subgroups": [ { "subgid": "Email", "item_count": 1 }, { "subgid": "Facebook", "item_count": 2 } ] }
+{ "timebin": interval(time("10:30:00.000"), time("10:31:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:31:00.000"), time("10:32:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:32:00.000"), time("10:33:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:33:00.000"), time("10:34:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:34:00.000"), time("10:35:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:35:00.000"), time("10:36:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:36:00.000"), time("10:37:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:37:00.000"), time("10:38:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:38:00.000"), time("10:39:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:39:00.000"), time("10:40:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:40:00.000"), time("10:41:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:41:00.000"), time("10:42:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:42:00.000"), time("10:43:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:43:00.000"), time("10:44:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:44:00.000"), time("10:45:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:45:00.000"), time("10:46:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:46:00.000"), time("10:47:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:47:00.000"), time("10:48:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:48:00.000"), time("10:49:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:49:00.000"), time("10:50:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:50:00.000"), time("10:51:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:51:00.000"), time("10:52:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:52:00.000"), time("10:53:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:53:00.000"), time("10:54:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:54:00.000"), time("10:55:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:55:00.000"), time("10:56:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:56:00.000"), time("10:57:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:57:00.000"), time("10:58:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:58:00.000"), time("10:59:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("10:59:00.000"), time("11:00:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:00:00.000"), time("11:01:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:01:00.000"), time("11:02:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:02:00.000"), time("11:03:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:03:00.000"), time("11:04:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:04:00.000"), time("11:05:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:05:00.000"), time("11:06:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:06:00.000"), time("11:07:00.000")), "subgroups": [ ] }
+{ "timebin": interval(time("11:07:00.000"), time("11:08:00.000")), "subgroups": [ { "subgid": "Email", "item_count": 2 } ] }
+{ "timebin": interval(time("11:08:00.000"), time("11:09:00.000")), "subgroups": [ ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm
index a7aebec..4562600 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_01/parse_01.1.adm
@@ -1 +1 @@
-{ "date1": date("2013-08-23"), "date2": date("-0012-08-12"), "date3": date("-1234-01-01"), "date4": date("-1980-11-09"), "date5": date("-1990-11-09"), "date6": date("2013-08-19"), "data7": date("2013-08-19"), "time1": time("08:23:49.000Z"), "time2": time("08:19:23.320Z"), "time3": time("20:19:23.320Z"), "time4": time("10:30:40.948Z"), "time5": time("10:30:40.948Z"), "datetime1": datetime("-1203-12-30T15:48:27.000Z"), "datetime2": datetime("-1203-12-30T23:48:27.392Z"), "datetime3": datetime("1723-12-03T23:59:23.392Z"), "datetime4": datetime("1723-12-04T03:59:23.392Z"), "datetime5": datetime("1723-12-04T03:59:23.392Z"), "datetime6": datetime("1970-01-02T03:59:23.392Z"), "datetime7": datetime("1723-12-04T03:59:23.392Z") }
+{ "date1": date("2013-08-23"), "date2": date("-0012-08-12"), "date3": date("-1234-01-01"), "date4": date("-1980-11-09"), "date5": date("-1990-11-09"), "date6": date("2013-08-19"), "data7": date("2013-08-19"), "time1": time("08:23:49.000"), "time2": time("08:19:23.320"), "time3": time("20:19:23.320"), "time4": time("18:30:40.948"), "time5": time("18:30:40.948"), "datetime1": datetime("-1203-12-30T15:48:27.000"), "datetime2": datetime("-1203-12-30T15:48:27.392"), "datetime3": datetime("1723-12-03T23:59:23.392"), "datetime4": datetime("1723-12-03T23:59:23.392"), "datetime5": datetime("1723-12-03T23:59:23.392"), "datetime6": datetime("1970-01-01T23:59:23.392"), "datetime7": datetime("1723-12-03T23:59:23.392") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm
index 68f22cc..8cd34d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_02/parse_02.1.adm
@@ -1 +1 @@
-{ "date-string-1": "-123/1/30", "date-string-2": "JAN 30, -0123", "date-string-3": "-0123/01/30", "time-string-1": "8.7.29.03 AM Z", "time-string-2": "08.07.29.030 AM Z", "datetime-string-1": "DEC 31 3:59:59.999 PM 137 Z", "datetime-string-2": "0137/DEC/31 3:59:59.999Z PM", "datetime-string-3": "0137-12-31T15:59:59.999Z", "datetime-string-4": "1000-10-10T10:10:10.100Z" }
+{ "date-string-1": "-123/1/30", "date-string-2": "JAN 30, -0123", "date-string-3": "-0123/01/30", "time-string-1": "8.7.29.03 AM", "time-string-2": "08.07.29.030 AM", "datetime-string-1": "DEC 31 11:59:59.999 PM 137", "datetime-string-2": "0137/DEC/31 11:59:59.999 PM", "datetime-string-3": "0137-12-31T23:59:59.999", "datetime-string-4": "1000-10-10T10:10:10.100" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.1.adm
new file mode 100644
index 0000000..cbc02e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.1.adm
@@ -0,0 +1,9 @@
+{ "dt": "1970-01-01T00:00:00.000", "dt2": "1970-01-01T00:00:00.000", "t": "00:00:00.000", "t2": "00:00:00.000" }
+{ "dt": "1970-01-01T00:00:00.001", "dt2": "1970-01-01T00:00:00.001", "t": "00:00:00.001", "t2": "00:00:00.001" }
+{ "dt": "1970-01-01T00:00:00.009", "dt2": "1970-01-01T00:00:00.009", "t": "00:00:00.009", "t2": "00:00:00.009" }
+{ "dt": "1970-01-01T00:00:00.010", "dt2": "1970-01-01T00:00:00.010", "t": "00:00:00.010", "t2": "00:00:00.010" }
+{ "dt": "1970-01-01T00:00:00.099", "dt2": "1970-01-01T00:00:00.099", "t": "00:00:00.099", "t2": "00:00:00.099" }
+{ "dt": "1970-01-01T00:00:00.100", "dt2": "1970-01-01T00:00:00.100", "t": "00:00:00.100", "t2": "00:00:00.100" }
+{ "dt": "1970-01-01T00:00:00.999", "dt2": "1970-01-01T00:00:00.999", "t": "00:00:00.999", "t2": "00:00:00.999" }
+{ "dt": "1970-01-01T00:00:01.999", "dt2": "1970-01-01T00:00:01.999", "t": "00:00:01.999", "t2": "00:00:01.999" }
+{ "dt": "1970-01-01T00:00:10.999", "dt2": "1970-01-01T00:00:10.999", "t": "00:00:10.999", "t2": "00:00:10.999" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.2.adm
new file mode 100644
index 0000000..4dd895b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.2.adm
@@ -0,0 +1,7 @@
+{ "s3": "2021-07-20 Sun", "d3": date("2021-07-20"), "s4": "2021-07-20 Sunday", "d4": date("2021-07-20") }
+{ "s3": "2021-07-21 Mon", "d3": date("2021-07-21"), "s4": "2021-07-21 Monday", "d4": date("2021-07-21") }
+{ "s3": "2021-07-22 Tue", "d3": date("2021-07-22"), "s4": "2021-07-22 Tuesday", "d4": date("2021-07-22") }
+{ "s3": "2021-07-23 Wed", "d3": date("2021-07-23"), "s4": "2021-07-23 Wednesday", "d4": date("2021-07-23") }
+{ "s3": "2021-07-24 Thu", "d3": date("2021-07-24"), "s4": "2021-07-24 Thursday", "d4": date("2021-07-24") }
+{ "s3": "2021-07-25 Fri", "d3": date("2021-07-25"), "s4": "2021-07-25 Friday", "d4": date("2021-07-25") }
+{ "s3": "2021-07-26 Sat", "d3": date("2021-07-26"), "s4": "2021-07-26 Saturday", "d4": date("2021-07-26") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.3.adm
new file mode 100644
index 0000000..8d26ff5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.3.adm
@@ -0,0 +1,7 @@
+{ "s3": "Sun 2021-07-20", "d3": date("2021-07-20"), "s4": "Sunday 2021-07-20", "d4": date("2021-07-20") }
+{ "s3": "Mon 2021-07-21", "d3": date("2021-07-21"), "s4": "Monday 2021-07-21", "d4": date("2021-07-21") }
+{ "s3": "Tue 2021-07-22", "d3": date("2021-07-22"), "s4": "Tuesday 2021-07-22", "d4": date("2021-07-22") }
+{ "s3": "Wed 2021-07-23", "d3": date("2021-07-23"), "s4": "Wednesday 2021-07-23", "d4": date("2021-07-23") }
+{ "s3": "Thu 2021-07-24", "d3": date("2021-07-24"), "s4": "Thursday 2021-07-24", "d4": date("2021-07-24") }
+{ "s3": "Fri 2021-07-25", "d3": date("2021-07-25"), "s4": "Friday 2021-07-25", "d4": date("2021-07-25") }
+{ "s3": "Sat 2021-07-26", "d3": date("2021-07-26"), "s4": "Saturday 2021-07-26", "d4": date("2021-07-26") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.4.adm
new file mode 100644
index 0000000..35d1aed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.4.adm
@@ -0,0 +1,12 @@
+{ "s3m": "2020-Jan", "d3m": date("2020-01-01"), "s3d": "2020-Jan-02", "d3d": date("2020-01-02"), "s4m": "2020-January", "d4m": date("2020-01-01"), "s4d": "2020-January-02", "d4d": date("2020-01-02") }
+{ "s3m": "2020-Feb", "d3m": date("2020-02-01"), "s3d": "2020-Feb-02", "d3d": date("2020-02-02"), "s4m": "2020-February", "d4m": date("2020-02-01"), "s4d": "2020-February-02", "d4d": date("2020-02-02") }
+{ "s3m": "2020-Mar", "d3m": date("2020-03-01"), "s3d": "2020-Mar-02", "d3d": date("2020-03-02"), "s4m": "2020-March", "d4m": date("2020-03-01"), "s4d": "2020-March-02", "d4d": date("2020-03-02") }
+{ "s3m": "2020-Apr", "d3m": date("2020-04-01"), "s3d": "2020-Apr-02", "d3d": date("2020-04-02"), "s4m": "2020-April", "d4m": date("2020-04-01"), "s4d": "2020-April-02", "d4d": date("2020-04-02") }
+{ "s3m": "2020-May", "d3m": date("2020-05-01"), "s3d": "2020-May-02", "d3d": date("2020-05-02"), "s4m": "2020-May", "d4m": date("2020-05-01"), "s4d": "2020-May-02", "d4d": date("2020-05-02") }
+{ "s3m": "2020-Jun", "d3m": date("2020-06-01"), "s3d": "2020-Jun-02", "d3d": date("2020-06-02"), "s4m": "2020-June", "d4m": date("2020-06-01"), "s4d": "2020-June-02", "d4d": date("2020-06-02") }
+{ "s3m": "2020-Jul", "d3m": date("2020-07-01"), "s3d": "2020-Jul-02", "d3d": date("2020-07-02"), "s4m": "2020-July", "d4m": date("2020-07-01"), "s4d": "2020-July-02", "d4d": date("2020-07-02") }
+{ "s3m": "2020-Aug", "d3m": date("2020-08-01"), "s3d": "2020-Aug-02", "d3d": date("2020-08-02"), "s4m": "2020-August", "d4m": date("2020-08-01"), "s4d": "2020-August-02", "d4d": date("2020-08-02") }
+{ "s3m": "2020-Sep", "d3m": date("2020-09-01"), "s3d": "2020-Sep-02", "d3d": date("2020-09-02"), "s4m": "2020-September", "d4m": date("2020-09-01"), "s4d": "2020-September-02", "d4d": date("2020-09-02") }
+{ "s3m": "2020-Oct", "d3m": date("2020-10-01"), "s3d": "2020-Oct-02", "d3d": date("2020-10-02"), "s4m": "2020-October", "d4m": date("2020-10-01"), "s4d": "2020-October-02", "d4d": date("2020-10-02") }
+{ "s3m": "2020-Nov", "d3m": date("2020-11-01"), "s3d": "2020-Nov-02", "d3d": date("2020-11-02"), "s4m": "2020-November", "d4m": date("2020-11-01"), "s4d": "2020-November-02", "d4d": date("2020-11-02") }
+{ "s3m": "2020-Dec", "d3m": date("2020-12-01"), "s3d": "2020-Dec-02", "d3d": date("2020-12-02"), "s4m": "2020-December", "d4m": date("2020-12-01"), "s4d": "2020-December-02", "d4d": date("2020-12-02") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.5.adm
new file mode 100644
index 0000000..d78e574
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.5.adm
@@ -0,0 +1 @@
+{ "t1": date("2020-01-01"), "t2": datetime("2020-01-01T00:00:00.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.6.adm
new file mode 100644
index 0000000..5a7c485
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/parse_03/parse_03.6.adm
@@ -0,0 +1,4 @@
+{ "r": 1, "d": date("2020-01-01"), "dt": datetime("2021-01-01T00:00:00.000") }
+{ "r": 2, "d": date("2020-04-01"), "dt": datetime("2021-04-01T00:00:00.000") }
+{ "r": 3, "d": date("2020-07-01"), "dt": datetime("2021-07-01T00:00:00.000") }
+{ "r": 4, "d": date("2020-10-01"), "dt": datetime("2021-10-01T00:00:00.000") }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.1.adm
new file mode 100644
index 0000000..83e0610
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.1.adm
@@ -0,0 +1,4 @@
+{ "p1": "2020-1", "p2": "2020-01", "cnt": 91 }
+{ "p1": "2020-2", "p2": "2020-02", "cnt": 91 }
+{ "p1": "2020-3", "p2": "2020-03", "cnt": 92 }
+{ "p1": "2020-4", "p2": "2020-04", "cnt": 92 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.2.adm
new file mode 100644
index 0000000..143423c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.2.adm
@@ -0,0 +1,12 @@
+{ "p": "2020-01-JANUARY", "cnt": 31 }
+{ "p": "2020-02-FEBRUARY", "cnt": 29 }
+{ "p": "2020-03-MARCH", "cnt": 31 }
+{ "p": "2020-04-APRIL", "cnt": 30 }
+{ "p": "2020-05-MAY", "cnt": 31 }
+{ "p": "2020-06-JUNE", "cnt": 30 }
+{ "p": "2020-07-JULY", "cnt": 31 }
+{ "p": "2020-08-AUGUST", "cnt": 31 }
+{ "p": "2020-09-SEPTEMBER", "cnt": 30 }
+{ "p": "2020-10-OCTOBER", "cnt": 31 }
+{ "p": "2020-11-NOVEMBER", "cnt": 30 }
+{ "p": "2020-12-DECEMBER", "cnt": 31 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.3.adm
new file mode 100644
index 0000000..1b6c2e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.3.adm
@@ -0,0 +1 @@
+{ "mn": 1, "mx": 366, "sm": 67161 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.4.adm
new file mode 100644
index 0000000..51db528
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.4.adm
@@ -0,0 +1,7 @@
+{ "r": 0, "d3": "Sun", "d4": "Sunday", "dt3": "Sun", "dt4": "Sunday" }
+{ "r": 1, "d3": "Mon", "d4": "Monday", "dt3": "Mon", "dt4": "Monday" }
+{ "r": 2, "d3": "Tue", "d4": "Tuesday", "dt3": "Tue", "dt4": "Tuesday" }
+{ "r": 3, "d3": "Wed", "d4": "Wednesday", "dt3": "Wed", "dt4": "Wednesday" }
+{ "r": 4, "d3": "Thu", "d4": "Thursday", "dt3": "Thu", "dt4": "Thursday" }
+{ "r": 5, "d3": "Fri", "d4": "Friday", "dt3": "Fri", "dt4": "Friday" }
+{ "r": 6, "d3": "Sat", "d4": "Saturday", "dt3": "Sat", "dt4": "Saturday" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.5.adm
new file mode 100644
index 0000000..54f1685
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/print_01/print_01.5.adm
@@ -0,0 +1,7 @@
+{ "ms": 0, "dt1": "0.000", "dt2": "0.000", "dt3": "0.000", "t1": "0.000", "t2": "0.000", "t3": "0.000" }
+{ "ms": 1, "dt1": "0.001", "dt2": "0.001", "dt3": "0.001", "t1": "0.001", "t2": "0.001", "t3": "0.001" }
+{ "ms": 9, "dt1": "0.009", "dt2": "0.009", "dt3": "0.009", "t1": "0.009", "t2": "0.009", "t3": "0.009" }
+{ "ms": 10, "dt1": "0.01", "dt2": "0.01", "dt3": "0.010", "t1": "0.01", "t2": "0.01", "t3": "0.010" }
+{ "ms": 99, "dt1": "0.099", "dt2": "0.099", "dt3": "0.099", "t1": "0.099", "t2": "0.099", "t3": "0.099" }
+{ "ms": 100, "dt1": "0.1", "dt2": "0.10", "dt3": "0.100", "t1": "0.1", "t2": "0.10", "t3": "0.100" }
+{ "ms": 999, "dt1": "0.999", "dt2": "0.999", "dt3": "0.999", "t1": "0.999", "t2": "0.999", "t3": "0.999" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/quarter_of_year_01/quarter_of_year_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/quarter_of_year_01/quarter_of_year_01.1.adm
new file mode 100644
index 0000000..9537fbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/quarter_of_year_01/quarter_of_year_01.1.adm
@@ -0,0 +1,12 @@
+{ "month": 1, "qoy1": 1, "qoy2": 1, "cnt": 31 }
+{ "month": 2, "qoy1": 1, "qoy2": 1, "cnt": 29 }
+{ "month": 3, "qoy1": 1, "qoy2": 1, "cnt": 31 }
+{ "month": 4, "qoy1": 2, "qoy2": 2, "cnt": 30 }
+{ "month": 5, "qoy1": 2, "qoy2": 2, "cnt": 31 }
+{ "month": 6, "qoy1": 2, "qoy2": 2, "cnt": 30 }
+{ "month": 7, "qoy1": 3, "qoy2": 3, "cnt": 31 }
+{ "month": 8, "qoy1": 3, "qoy2": 3, "cnt": 31 }
+{ "month": 9, "qoy1": 3, "qoy2": 3, "cnt": 30 }
+{ "month": 10, "qoy1": 4, "qoy2": 4, "cnt": 31 }
+{ "month": 11, "qoy1": 4, "qoy2": 4, "cnt": 30 }
+{ "month": 12, "qoy1": 4, "qoy2": 4, "cnt": 31 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/time_functions/time_functions.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/time_functions/time_functions.1.adm
index 303e762..0926827 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/time_functions/time_functions.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/time_functions/time_functions.1.adm
@@ -1 +1 @@
-{ "time1": time("00:26:00.074Z"), "time2": time("23:35:49.938Z"), "time3": time("23:30:23.000Z"), "time4": time("18:26:00.074Z"), "time5": time("00:11:49.938Z"), "unix1": 1560074, "duration1": duration("-PT23H24M"), "duration2": duration("PT18H"), "c1": true, "c2": true, "null1": null, "nullunix1": null, "null2": null, "null3": null, "null4": null, "null5": null, "null6": null }
+{ "time1": time("00:26:00.074"), "time2": time("23:35:49.938"), "time3": time("02:30:23.000"), "time4": time("18:26:00.074"), "time5": time("00:11:49.938"), "unix1": 1560074, "duration1": duration("-PT23H24M"), "duration2": duration("PT18H"), "c1": true, "c2": true, "null1": null, "nullunix1": null, "null2": null, "null3": null, "null4": null, "null5": null, "null6": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.1.adm
new file mode 100644
index 0000000..1b97f5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.1.adm
@@ -0,0 +1,63 @@
+{ "month": 1, "woy1": 1, "woy2": 1, "cnt": 2 }
+{ "month": 12, "woy1": 1, "woy2": 1, "cnt": 6 }
+{ "month": 1, "woy1": 2, "woy2": 2, "cnt": 7 }
+{ "month": 1, "woy1": 3, "woy2": 3, "cnt": 7 }
+{ "month": 1, "woy1": 4, "woy2": 4, "cnt": 7 }
+{ "month": 1, "woy1": 5, "woy2": 5, "cnt": 7 }
+{ "month": 1, "woy1": 6, "woy2": 6, "cnt": 1 }
+{ "month": 2, "woy1": 6, "woy2": 6, "cnt": 6 }
+{ "month": 2, "woy1": 7, "woy2": 7, "cnt": 7 }
+{ "month": 2, "woy1": 8, "woy2": 8, "cnt": 7 }
+{ "month": 2, "woy1": 9, "woy2": 9, "cnt": 7 }
+{ "month": 2, "woy1": 10, "woy2": 10, "cnt": 1 }
+{ "month": 3, "woy1": 10, "woy2": 10, "cnt": 6 }
+{ "month": 3, "woy1": 11, "woy2": 11, "cnt": 7 }
+{ "month": 3, "woy1": 12, "woy2": 12, "cnt": 7 }
+{ "month": 3, "woy1": 13, "woy2": 13, "cnt": 7 }
+{ "month": 3, "woy1": 14, "woy2": 14, "cnt": 4 }
+{ "month": 4, "woy1": 14, "woy2": 14, "cnt": 3 }
+{ "month": 4, "woy1": 15, "woy2": 15, "cnt": 7 }
+{ "month": 4, "woy1": 16, "woy2": 16, "cnt": 7 }
+{ "month": 4, "woy1": 17, "woy2": 17, "cnt": 7 }
+{ "month": 4, "woy1": 18, "woy2": 18, "cnt": 6 }
+{ "month": 5, "woy1": 18, "woy2": 18, "cnt": 1 }
+{ "month": 5, "woy1": 19, "woy2": 19, "cnt": 7 }
+{ "month": 5, "woy1": 20, "woy2": 20, "cnt": 7 }
+{ "month": 5, "woy1": 21, "woy2": 21, "cnt": 7 }
+{ "month": 5, "woy1": 22, "woy2": 22, "cnt": 7 }
+{ "month": 5, "woy1": 23, "woy2": 23, "cnt": 2 }
+{ "month": 6, "woy1": 23, "woy2": 23, "cnt": 5 }
+{ "month": 6, "woy1": 24, "woy2": 24, "cnt": 7 }
+{ "month": 6, "woy1": 25, "woy2": 25, "cnt": 7 }
+{ "month": 6, "woy1": 26, "woy2": 26, "cnt": 7 }
+{ "month": 6, "woy1": 27, "woy2": 27, "cnt": 4 }
+{ "month": 7, "woy1": 27, "woy2": 27, "cnt": 3 }
+{ "month": 7, "woy1": 28, "woy2": 28, "cnt": 7 }
+{ "month": 7, "woy1": 29, "woy2": 29, "cnt": 7 }
+{ "month": 7, "woy1": 30, "woy2": 30, "cnt": 7 }
+{ "month": 7, "woy1": 31, "woy2": 31, "cnt": 7 }
+{ "month": 8, "woy1": 32, "woy2": 32, "cnt": 7 }
+{ "month": 8, "woy1": 33, "woy2": 33, "cnt": 7 }
+{ "month": 8, "woy1": 34, "woy2": 34, "cnt": 7 }
+{ "month": 8, "woy1": 35, "woy2": 35, "cnt": 7 }
+{ "month": 8, "woy1": 36, "woy2": 36, "cnt": 3 }
+{ "month": 9, "woy1": 36, "woy2": 36, "cnt": 4 }
+{ "month": 9, "woy1": 37, "woy2": 37, "cnt": 7 }
+{ "month": 9, "woy1": 38, "woy2": 38, "cnt": 7 }
+{ "month": 9, "woy1": 39, "woy2": 39, "cnt": 7 }
+{ "month": 9, "woy1": 40, "woy2": 40, "cnt": 5 }
+{ "month": 10, "woy1": 40, "woy2": 40, "cnt": 2 }
+{ "month": 10, "woy1": 41, "woy2": 41, "cnt": 7 }
+{ "month": 10, "woy1": 42, "woy2": 42, "cnt": 7 }
+{ "month": 10, "woy1": 43, "woy2": 43, "cnt": 7 }
+{ "month": 10, "woy1": 44, "woy2": 44, "cnt": 7 }
+{ "month": 10, "woy1": 45, "woy2": 45, "cnt": 1 }
+{ "month": 11, "woy1": 45, "woy2": 45, "cnt": 6 }
+{ "month": 11, "woy1": 46, "woy2": 46, "cnt": 7 }
+{ "month": 11, "woy1": 47, "woy2": 47, "cnt": 7 }
+{ "month": 11, "woy1": 48, "woy2": 48, "cnt": 7 }
+{ "month": 11, "woy1": 49, "woy2": 49, "cnt": 3 }
+{ "month": 12, "woy1": 49, "woy2": 49, "cnt": 4 }
+{ "month": 12, "woy1": 50, "woy2": 50, "cnt": 7 }
+{ "month": 12, "woy1": 51, "woy2": 51, "cnt": 7 }
+{ "month": 12, "woy1": 52, "woy2": 52, "cnt": 7 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.2.adm
new file mode 100644
index 0000000..952440e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.2.adm
@@ -0,0 +1,61 @@
+{ "month": 1, "woy1": 1, "woy2": 1, "cnt": 3 }
+{ "month": 12, "woy1": 1, "woy2": 1, "cnt": 5 }
+{ "month": 1, "woy1": 2, "woy2": 2, "cnt": 7 }
+{ "month": 1, "woy1": 3, "woy2": 3, "cnt": 7 }
+{ "month": 1, "woy1": 4, "woy2": 4, "cnt": 7 }
+{ "month": 1, "woy1": 5, "woy2": 5, "cnt": 7 }
+{ "month": 2, "woy1": 6, "woy2": 6, "cnt": 7 }
+{ "month": 2, "woy1": 7, "woy2": 7, "cnt": 7 }
+{ "month": 2, "woy1": 8, "woy2": 8, "cnt": 7 }
+{ "month": 2, "woy1": 9, "woy2": 9, "cnt": 7 }
+{ "month": 3, "woy1": 10, "woy2": 10, "cnt": 7 }
+{ "month": 3, "woy1": 11, "woy2": 11, "cnt": 7 }
+{ "month": 3, "woy1": 12, "woy2": 12, "cnt": 7 }
+{ "month": 3, "woy1": 13, "woy2": 13, "cnt": 7 }
+{ "month": 3, "woy1": 14, "woy2": 14, "cnt": 3 }
+{ "month": 4, "woy1": 14, "woy2": 14, "cnt": 4 }
+{ "month": 4, "woy1": 15, "woy2": 15, "cnt": 7 }
+{ "month": 4, "woy1": 16, "woy2": 16, "cnt": 7 }
+{ "month": 4, "woy1": 17, "woy2": 17, "cnt": 7 }
+{ "month": 4, "woy1": 18, "woy2": 18, "cnt": 5 }
+{ "month": 5, "woy1": 18, "woy2": 18, "cnt": 2 }
+{ "month": 5, "woy1": 19, "woy2": 19, "cnt": 7 }
+{ "month": 5, "woy1": 20, "woy2": 20, "cnt": 7 }
+{ "month": 5, "woy1": 21, "woy2": 21, "cnt": 7 }
+{ "month": 5, "woy1": 22, "woy2": 22, "cnt": 7 }
+{ "month": 5, "woy1": 23, "woy2": 23, "cnt": 1 }
+{ "month": 6, "woy1": 23, "woy2": 23, "cnt": 6 }
+{ "month": 6, "woy1": 24, "woy2": 24, "cnt": 7 }
+{ "month": 6, "woy1": 25, "woy2": 25, "cnt": 7 }
+{ "month": 6, "woy1": 26, "woy2": 26, "cnt": 7 }
+{ "month": 6, "woy1": 27, "woy2": 27, "cnt": 3 }
+{ "month": 7, "woy1": 27, "woy2": 27, "cnt": 4 }
+{ "month": 7, "woy1": 28, "woy2": 28, "cnt": 7 }
+{ "month": 7, "woy1": 29, "woy2": 29, "cnt": 7 }
+{ "month": 7, "woy1": 30, "woy2": 30, "cnt": 7 }
+{ "month": 7, "woy1": 31, "woy2": 31, "cnt": 6 }
+{ "month": 8, "woy1": 31, "woy2": 31, "cnt": 1 }
+{ "month": 8, "woy1": 32, "woy2": 32, "cnt": 7 }
+{ "month": 8, "woy1": 33, "woy2": 33, "cnt": 7 }
+{ "month": 8, "woy1": 34, "woy2": 34, "cnt": 7 }
+{ "month": 8, "woy1": 35, "woy2": 35, "cnt": 7 }
+{ "month": 8, "woy1": 36, "woy2": 36, "cnt": 2 }
+{ "month": 9, "woy1": 36, "woy2": 36, "cnt": 5 }
+{ "month": 9, "woy1": 37, "woy2": 37, "cnt": 7 }
+{ "month": 9, "woy1": 38, "woy2": 38, "cnt": 7 }
+{ "month": 9, "woy1": 39, "woy2": 39, "cnt": 7 }
+{ "month": 9, "woy1": 40, "woy2": 40, "cnt": 4 }
+{ "month": 10, "woy1": 40, "woy2": 40, "cnt": 3 }
+{ "month": 10, "woy1": 41, "woy2": 41, "cnt": 7 }
+{ "month": 10, "woy1": 42, "woy2": 42, "cnt": 7 }
+{ "month": 10, "woy1": 43, "woy2": 43, "cnt": 7 }
+{ "month": 10, "woy1": 44, "woy2": 44, "cnt": 7 }
+{ "month": 11, "woy1": 45, "woy2": 45, "cnt": 7 }
+{ "month": 11, "woy1": 46, "woy2": 46, "cnt": 7 }
+{ "month": 11, "woy1": 47, "woy2": 47, "cnt": 7 }
+{ "month": 11, "woy1": 48, "woy2": 48, "cnt": 7 }
+{ "month": 11, "woy1": 49, "woy2": 49, "cnt": 2 }
+{ "month": 12, "woy1": 49, "woy2": 49, "cnt": 5 }
+{ "month": 12, "woy1": 50, "woy2": 50, "cnt": 7 }
+{ "month": 12, "woy1": 51, "woy2": 51, "cnt": 7 }
+{ "month": 12, "woy1": 52, "woy2": 52, "cnt": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.3.adm
new file mode 100644
index 0000000..10c40e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/week_of_year_01/week_of_year_01.3.adm
@@ -0,0 +1,7 @@
+{ "r": 1, "mn": 3, "mx": 3 }
+{ "r": 2, "mn": 2, "mx": 2 }
+{ "r": 3, "mn": 2, "mx": 2 }
+{ "r": 4, "mn": 2, "mx": 2 }
+{ "r": 5, "mn": 2, "mx": 2 }
+{ "r": 6, "mn": 2, "mx": 2 }
+{ "r": 7, "mn": 3, "mx": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.12.adm
index ae2549f..d9e9c05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.12.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.12.adm
@@ -1,7 +1,7 @@
-{ "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000Z"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06") } ] }
-{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
-{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
-{ "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000Z"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27") } ] }
-{ "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000Z"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15") } ] }
-{ "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000Z"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19") } ] }
-{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
+{ "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06") } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
+{ "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27") } ] }
+{ "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15") } ] }
+{ "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19") } ] }
+{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.13.adm
index 2df77f6..88e2886 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.13.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.13.adm
@@ -1,3 +1,3 @@
-{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
-{ "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000Z"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] }
-{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] }
+{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.18.adm
index b59a814..f2bec61 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.18.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.18.adm
@@ -1,12 +1,12 @@
-{ "tweet": { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, "similar-tweets": [ {{ "t-mobile", "shortcut-menu" }} ] }
-{ "tweet": { "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, "similar-tweets": [ {{ "verizon", "shortcut-menu" }}, {{ "iphone", "voice-clarity" }}, {{ "verizon", "voicemail-service" }} ] }
-{ "tweet": { "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "samsung", "platform" }} ] }
-{ "tweet": { "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, "similar-tweets": [ {{ "sprint", "voice-command" }}, {{ "samsung", "platform" }} ] }
-{ "tweet": { "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "t-mobile", "shortcut-menu" }}, {{ "verizon", "voicemail-service" }} ] }
-{ "tweet": { "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
-{ "tweet": { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, "similar-tweets": [ {{ "samsung", "voice-command" }} ] }
-{ "tweet": { "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
-{ "tweet": { "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "iphone", "platform" }} ] }
-{ "tweet": { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, "similar-tweets": [ {{ "iphone", "platform" }}, {{ "samsung", "voice-command" }} ] }
-{ "tweet": { "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "t-mobile", "customization" }}, {{ "verizon", "shortcut-menu" }} ] }
-{ "tweet": { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "verizon", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, "similar-tweets": [ {{ "t-mobile", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, "similar-tweets": [ {{ "verizon", "shortcut-menu" }}, {{ "iphone", "voice-clarity" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, "similar-tweets": [ {{ "sprint", "voice-command" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "t-mobile", "shortcut-menu" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, "similar-tweets": [ {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "iphone", "platform" }} ] }
+{ "tweet": { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, "similar-tweets": [ {{ "iphone", "platform" }}, {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "t-mobile", "customization" }}, {{ "verizon", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "verizon", "shortcut-menu" }} ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.20.adm
index 5c8c013..040d91f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.20.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.20.adm
@@ -1,13 +1,13 @@
-{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
-{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
-{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
-{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
-{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000Z"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
-{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
-{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
-{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
-{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
-{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
-{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
-{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
-{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.4.adm
index 79b7b4a..594c2a3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.4.adm
@@ -1 +1 @@
-{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
+{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.5.adm
index 4e39206..3cbc2cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.5.adm
@@ -1,3 +1,3 @@
-{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
-{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
-{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.6.adm
index 6b6b555..c43c2fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/tinysocial-suite.6.adm
@@ -1,4 +1,4 @@
-{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
-{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
-{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
-{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
+{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.12.adm
index ae2549f..d9e9c05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.12.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.12.adm
@@ -1,7 +1,7 @@
-{ "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000Z"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06") } ] }
-{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
-{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
-{ "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000Z"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27") } ] }
-{ "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000Z"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15") } ] }
-{ "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000Z"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19") } ] }
-{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
+{ "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06") } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
+{ "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27") } ] }
+{ "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15") } ] }
+{ "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19") } ] }
+{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.13.adm
index 2df77f6..88e2886 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.13.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.13.adm
@@ -1,3 +1,3 @@
-{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
-{ "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000Z"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] }
-{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] }
+{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.18.adm
index b59a814..f2bec61 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.18.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.18.adm
@@ -1,12 +1,12 @@
-{ "tweet": { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, "similar-tweets": [ {{ "t-mobile", "shortcut-menu" }} ] }
-{ "tweet": { "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, "similar-tweets": [ {{ "verizon", "shortcut-menu" }}, {{ "iphone", "voice-clarity" }}, {{ "verizon", "voicemail-service" }} ] }
-{ "tweet": { "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "samsung", "platform" }} ] }
-{ "tweet": { "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, "similar-tweets": [ {{ "sprint", "voice-command" }}, {{ "samsung", "platform" }} ] }
-{ "tweet": { "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "t-mobile", "shortcut-menu" }}, {{ "verizon", "voicemail-service" }} ] }
-{ "tweet": { "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
-{ "tweet": { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, "similar-tweets": [ {{ "samsung", "voice-command" }} ] }
-{ "tweet": { "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
-{ "tweet": { "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "iphone", "platform" }} ] }
-{ "tweet": { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, "similar-tweets": [ {{ "iphone", "platform" }}, {{ "samsung", "voice-command" }} ] }
-{ "tweet": { "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "t-mobile", "customization" }}, {{ "verizon", "shortcut-menu" }} ] }
-{ "tweet": { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "verizon", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, "similar-tweets": [ {{ "t-mobile", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, "similar-tweets": [ {{ "verizon", "shortcut-menu" }}, {{ "iphone", "voice-clarity" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, "similar-tweets": [ {{ "sprint", "voice-command" }}, {{ "samsung", "platform" }} ] }
+{ "tweet": { "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "t-mobile", "shortcut-menu" }}, {{ "verizon", "voicemail-service" }} ] }
+{ "tweet": { "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, "similar-tweets": [ {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+{ "tweet": { "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "iphone", "platform" }} ] }
+{ "tweet": { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, "similar-tweets": [ {{ "iphone", "platform" }}, {{ "samsung", "voice-command" }} ] }
+{ "tweet": { "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "t-mobile", "customization" }}, {{ "verizon", "shortcut-menu" }} ] }
+{ "tweet": { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "verizon", "shortcut-menu" }} ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.20.adm
index 5c8c013..040d91f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.20.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.20.adm
@@ -1,13 +1,13 @@
-{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
-{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
-{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
-{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
-{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000Z"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
-{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
-{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
-{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
-{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
-{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
-{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
-{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
-{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+{ "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }
+{ "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }
+{ "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }
+{ "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
+{ "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }
+{ "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }
+{ "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+{ "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }
+{ "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }
+{ "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+{ "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }
+{ "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.4.adm
index 79b7b4a..594c2a3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.4.adm
@@ -1 +1 @@
-{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
+{ "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.5.adm
index 4e39206..3cbc2cf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.5.adm
@@ -1,3 +1,3 @@
-{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
-{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
-{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.6.adm
index 6b6b555..c43c2fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite/tinysocial-suite.6.adm
@@ -1,4 +1,4 @@
-{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
-{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
-{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
-{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
+{ "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27") } ] }
+{ "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+{ "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08") } ] }
+{ "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/gettype/gettype.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/gettype/gettype.2.adm
new file mode 100644
index 0000000..ad79b3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/gettype/gettype.2.adm
@@ -0,0 +1,3 @@
+{ "dType": "string" }
+{ "dType": "string" }
+{ "dType": "string" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/gettype/gettype.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/gettype/gettype.3.adm
new file mode 100644
index 0000000..5154d50
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/gettype/gettype.3.adm
@@ -0,0 +1,2 @@
+{ "t1": "array", "t2": "binary", "t3": "boolean", "t4": "circle", "t5": "date", "t6": "datetime", "t7": "duration", "t8": "interval", "t9": "line", "t10": "multiset", "t11": "point", "t12": "polygon", "t13": "rectangle", "t14": "string", "t15": "time", "t16": "uuid" }
+{ "t1": "array", "t2": "binary", "t3": "boolean", "t4": "circle", "t5": "date", "t6": "datetime", "t7": "duration", "t8": "interval", "t9": "line", "t10": "multiset", "t11": "point", "t12": "polygon", "t13": "rectangle", "t14": "string", "t15": "time", "t16": "uuid" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/opentype_orderby_01/opentype_orderby_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/opentype_orderby_01/opentype_orderby_01.1.adm
index 0c76235..78b19bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/opentype_orderby_01/opentype_orderby_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/opentype_orderby_01/opentype_orderby_01.1.adm
@@ -48,7 +48,7 @@
{ "emp.id": 34, "emp.supvrid": "7" }
{ "emp.id": 39, "emp.supvrid": "8" }
{ "emp.id": 44, "emp.supvrid": "9" }
-{ "emp.id": 54, "emp.supvrid": time("12:54:54.039Z") }
+{ "emp.id": 54, "emp.supvrid": time("12:54:54.039") }
{ "emp.id": 55, "emp.supvrid": duration("P101YT12M") }
{ "emp.id": 52, "emp.supvrid": point("5.1E-10,-1000000.0") }
{ "emp.id": 51, "emp.supvrid": point("80.1,-1000000.0") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_bigint_02/to_bigint_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_bigint_02/to_bigint_02.1.adm
new file mode 100644
index 0000000..ef236da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_bigint_02/to_bigint_02.1.adm
@@ -0,0 +1 @@
+{ "t": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_boolean_02/to_boolean_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_boolean_02/to_boolean_02.1.adm
new file mode 100644
index 0000000..ef236da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_boolean_02/to_boolean_02.1.adm
@@ -0,0 +1 @@
+{ "t": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_double_02/to_double_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_double_02/to_double_02.1.adm
new file mode 100644
index 0000000..ef236da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_double_02/to_double_02.1.adm
@@ -0,0 +1 @@
+{ "t": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_string_02/to_string_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_string_02/to_string_02.1.adm
new file mode 100644
index 0000000..97101cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_string_02/to_string_02.1.adm
@@ -0,0 +1 @@
+{ "t": "2017-06-30" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/filtered-dataset/filtered-dataset.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/filtered-dataset/filtered-dataset.1.adm
index 2e4795e..e69edfc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/filtered-dataset/filtered-dataset.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/upsert/filtered-dataset/filtered-dataset.1.adm
@@ -1,9 +1,9 @@
-{ "message-id": 11, "author-id": 232, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000Z") }
-{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
-{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000Z") }
-{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000Z") }
-{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
-{ "message-id": 19, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000Z") }
-{ "message-id": 9, "author-id": 65, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
-{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
-{ "message-id": 17, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
+{ "message-id": 11, "author-id": 232, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000") }
+{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000") }
+{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000") }
+{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000") }
+{ "message-id": 19, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000") }
+{ "message-id": 9, "author-id": 65, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000") }
+{ "message-id": 17, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1317/query-ASTERIXDB-1317.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1317/query-ASTERIXDB-1317.1.adm
index bb04c02..ec9f346 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1317/query-ASTERIXDB-1317.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/query-ASTERIXDB-1317/query-ASTERIXDB-1317.1.adm
@@ -1,54 +1,54 @@
-{ "row_id": 12, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:07.000Z") }
-{ "row_id": 14, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:08.000Z") }
-{ "row_id": 21, "avg_RR_clipped": 0.6896551847457886, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:13.000Z") }
-{ "row_id": 23, "avg_RR_clipped": 0.6896551847457886, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:14.000Z") }
-{ "row_id": 26, "avg_RR_clipped": 0.6185566782951355, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:16.000Z") }
-{ "row_id": 30, "avg_RR_clipped": 0.6060606241226196, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:19.000Z") }
-{ "row_id": 33, "avg_RR_clipped": 0.5454545617103577, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:21.000Z") }
-{ "row_id": 38, "avg_RR_clipped": 0.47999998927116394, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:23.000Z") }
-{ "row_id": 45, "avg_RR_clipped": 0.508474588394165, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:27.000Z") }
-{ "row_id": 47, "avg_RR_clipped": 0.517241358757019, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:28.000Z") }
-{ "row_id": 49, "avg_RR_clipped": 0.5309734344482422, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:29.000Z") }
-{ "row_id": 51, "avg_RR_clipped": 0.5357142686843872, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:30.000Z") }
-{ "row_id": 58, "avg_RR_clipped": 0.5714285969734192, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:33.000Z") }
-{ "row_id": 60, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:35.000Z") }
-{ "row_id": 13, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:08.000Z") }
-{ "row_id": 15, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:09.000Z") }
-{ "row_id": 16, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:10.000Z") }
-{ "row_id": 19, "avg_RR_clipped": 0.6741573214530945, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:12.000Z") }
-{ "row_id": 20, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:12.000Z") }
-{ "row_id": 22, "avg_RR_clipped": 0.6976743936538696, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:14.000Z") }
-{ "row_id": 24, "avg_RR_clipped": 0.6521739363670349, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:15.000Z") }
-{ "row_id": 29, "avg_RR_clipped": 0.6185566782951355, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:18.000Z") }
-{ "row_id": 31, "avg_RR_clipped": 0.594059407711029, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:19.000Z") }
-{ "row_id": 35, "avg_RR_clipped": 0.5454545617103577, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:22.000Z") }
-{ "row_id": 46, "avg_RR_clipped": 0.5128205418586731, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:27.000Z") }
-{ "row_id": 48, "avg_RR_clipped": 0.52173912525177, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:28.000Z") }
-{ "row_id": 52, "avg_RR_clipped": 0.5405405163764954, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:30.000Z") }
-{ "row_id": 55, "avg_RR_clipped": 0.5263158082962036, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:32.000Z") }
-{ "row_id": 56, "avg_RR_clipped": 0.5309734344482422, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:32.000Z") }
-{ "row_id": 57, "avg_RR_clipped": 0.5357142686843872, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:33.000Z") }
-{ "row_id": 59, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:34.000Z") }
-{ "row_id": 62, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:36.000Z") }
-{ "row_id": 63, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:36.000Z") }
-{ "row_id": 64, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:37.000Z") }
-{ "row_id": 65, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:38.000Z") }
-{ "row_id": 25, "avg_RR_clipped": 0.6382978558540344, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:16.000Z") }
-{ "row_id": 28, "avg_RR_clipped": 0.6315789222717285, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:17.000Z") }
-{ "row_id": 34, "avg_RR_clipped": 0.5769230723381042, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:21.000Z") }
-{ "row_id": 37, "avg_RR_clipped": 0.4958677589893341, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:23.000Z") }
-{ "row_id": 40, "avg_RR_clipped": 0.4878048896789551, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:24.000Z") }
-{ "row_id": 41, "avg_RR_clipped": 0.49180328845977783, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:25.000Z") }
-{ "row_id": 43, "avg_RR_clipped": 0.5, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:26.000Z") }
-{ "row_id": 17, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:10.000Z") }
-{ "row_id": 18, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:11.000Z") }
-{ "row_id": 27, "avg_RR_clipped": 0.6185566782951355, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:17.000Z") }
-{ "row_id": 32, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:20.000Z") }
-{ "row_id": 36, "avg_RR_clipped": 0.5128205418586731, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:22.000Z") }
-{ "row_id": 39, "avg_RR_clipped": 0.4838709533214569, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:24.000Z") }
-{ "row_id": 42, "avg_RR_clipped": 0.4958677589893341, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:25.000Z") }
-{ "row_id": 44, "avg_RR_clipped": 0.5042017102241516, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:26.000Z") }
-{ "row_id": 50, "avg_RR_clipped": 0.5309734344482422, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:29.000Z") }
-{ "row_id": 53, "avg_RR_clipped": 0.5128205418586731, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:31.000Z") }
-{ "row_id": 54, "avg_RR_clipped": 0.52173912525177, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:31.000Z") }
-{ "row_id": 61, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:35.000Z") }
+{ "row_id": 12, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:07.000") }
+{ "row_id": 14, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:08.000") }
+{ "row_id": 21, "avg_RR_clipped": 0.6896551847457886, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:13.000") }
+{ "row_id": 23, "avg_RR_clipped": 0.6896551847457886, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:14.000") }
+{ "row_id": 26, "avg_RR_clipped": 0.6185566782951355, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:16.000") }
+{ "row_id": 30, "avg_RR_clipped": 0.6060606241226196, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:19.000") }
+{ "row_id": 33, "avg_RR_clipped": 0.5454545617103577, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:21.000") }
+{ "row_id": 38, "avg_RR_clipped": 0.47999998927116394, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:23.000") }
+{ "row_id": 45, "avg_RR_clipped": 0.508474588394165, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:27.000") }
+{ "row_id": 47, "avg_RR_clipped": 0.517241358757019, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:28.000") }
+{ "row_id": 49, "avg_RR_clipped": 0.5309734344482422, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:29.000") }
+{ "row_id": 51, "avg_RR_clipped": 0.5357142686843872, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:30.000") }
+{ "row_id": 58, "avg_RR_clipped": 0.5714285969734192, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:33.000") }
+{ "row_id": 60, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:35.000") }
+{ "row_id": 13, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:08.000") }
+{ "row_id": 15, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:09.000") }
+{ "row_id": 16, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:10.000") }
+{ "row_id": 19, "avg_RR_clipped": 0.6741573214530945, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:12.000") }
+{ "row_id": 20, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:12.000") }
+{ "row_id": 22, "avg_RR_clipped": 0.6976743936538696, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:14.000") }
+{ "row_id": 24, "avg_RR_clipped": 0.6521739363670349, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:15.000") }
+{ "row_id": 29, "avg_RR_clipped": 0.6185566782951355, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:18.000") }
+{ "row_id": 31, "avg_RR_clipped": 0.594059407711029, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:19.000") }
+{ "row_id": 35, "avg_RR_clipped": 0.5454545617103577, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:22.000") }
+{ "row_id": 46, "avg_RR_clipped": 0.5128205418586731, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:27.000") }
+{ "row_id": 48, "avg_RR_clipped": 0.52173912525177, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:28.000") }
+{ "row_id": 52, "avg_RR_clipped": 0.5405405163764954, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:30.000") }
+{ "row_id": 55, "avg_RR_clipped": 0.5263158082962036, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:32.000") }
+{ "row_id": 56, "avg_RR_clipped": 0.5309734344482422, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:32.000") }
+{ "row_id": 57, "avg_RR_clipped": 0.5357142686843872, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:33.000") }
+{ "row_id": 59, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:34.000") }
+{ "row_id": 62, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:36.000") }
+{ "row_id": 63, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:36.000") }
+{ "row_id": 64, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:37.000") }
+{ "row_id": 65, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:38.000") }
+{ "row_id": 25, "avg_RR_clipped": 0.6382978558540344, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:16.000") }
+{ "row_id": 28, "avg_RR_clipped": 0.6315789222717285, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:17.000") }
+{ "row_id": 34, "avg_RR_clipped": 0.5769230723381042, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:21.000") }
+{ "row_id": 37, "avg_RR_clipped": 0.4958677589893341, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:23.000") }
+{ "row_id": 40, "avg_RR_clipped": 0.4878048896789551, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:24.000") }
+{ "row_id": 41, "avg_RR_clipped": 0.49180328845977783, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:25.000") }
+{ "row_id": 43, "avg_RR_clipped": 0.5, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:26.000") }
+{ "row_id": 17, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:10.000") }
+{ "row_id": 18, "avg_RR_clipped": 0.6666666865348816, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:11.000") }
+{ "row_id": 27, "avg_RR_clipped": 0.6185566782951355, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:17.000") }
+{ "row_id": 32, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:20.000") }
+{ "row_id": 36, "avg_RR_clipped": 0.5128205418586731, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:22.000") }
+{ "row_id": 39, "avg_RR_clipped": 0.4838709533214569, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:24.000") }
+{ "row_id": 42, "avg_RR_clipped": 0.4958677589893341, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:25.000") }
+{ "row_id": 44, "avg_RR_clipped": 0.5042017102241516, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:26.000") }
+{ "row_id": 50, "avg_RR_clipped": 0.5309734344482422, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:29.000") }
+{ "row_id": 53, "avg_RR_clipped": 0.5128205418586731, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:31.000") }
+{ "row_id": 54, "avg_RR_clipped": 0.52173912525177, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:31.000") }
+{ "row_id": 61, "avg_RR_clipped": 0.582524299621582, "sid": 55, "gdate": date("2014-01-08"), "gday": 1, "timebin": time("11:45:35.000") }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.11.adm
new file mode 100644
index 0000000..8d802cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.11.adm
@@ -0,0 +1,2 @@
+{ "r": 3 }
+{ "r": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.12.adm
new file mode 100644
index 0000000..42561b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.12.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test1", "DatasetName": "vA", "DatatypeDataverseName": "Metadata", "DatatypeName": "AnyObject", "ViewDetails": { "Definition": "select r from range(3,4) r", "Dependencies": [ [ ], [ ], [ ] ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.2.adm
new file mode 100644
index 0000000..f9bc681
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.2.adm
@@ -0,0 +1 @@
+{ "d1": 2, "s2": 2, "s1": 2, "f2": 2, "f1": 2, "v2": 3, "v1": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.3.adm
new file mode 100644
index 0000000..6ceac5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.3.adm
@@ -0,0 +1,3 @@
+{ "DataverseName": "test1", "DatasetName": "v1", "DatatypeDataverseName": "Metadata", "DatatypeName": "AnyObject", "ViewDetails": { "Definition": "select sum(r) as v1 from range(1,3) r", "Dependencies": [ [ ], [ ], [ ] ] } }
+{ "DataverseName": "test1", "DatasetName": "vA", "DatatypeDataverseName": "Metadata", "DatatypeName": "AnyObject", "ViewDetails": { "Definition": "select v1.*, v2.*, f1.*, f2.*, s1.*, s2.*, d1.*\n from\n v1 v1,\n test2.v2 v2,\n f1() f1,\n test2.f2() f2,\n (\n select count(*) as s1 from DataverseSyn1\n where DataverseName like \"te%\"\n ) s1,\n (\n select count(*) as s2 from test2.DataverseSyn2\n where DataverseName like \"tes%\"\n ) s2,\n (\n select count(*) as d1 from Metadata.`Dataverse`\n where DataverseName like \"test%\"\n ) d1", "Dependencies": [ [ [ "test1", "v1" ], [ "test2", "v2" ], [ "Metadata", "Dataverse" ] ], [ [ "test1", "f1", "0" ], [ "test2", "f2", "0" ] ], [ ], [ [ "test1", "DataverseSyn1" ], [ "test2", "DataverseSyn2" ] ] ] } }
+{ "DataverseName": "test2", "DatasetName": "v2", "DatatypeDataverseName": "Metadata", "DatatypeName": "AnyObject", "ViewDetails": { "Definition": "select sum(r) as v2 from range(1,2) r", "Dependencies": [ [ ], [ ], [ ] ] } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.5.adm
new file mode 100644
index 0000000..f9bc681
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.5.adm
@@ -0,0 +1 @@
+{ "d1": 2, "s2": 2, "s1": 2, "f2": 2, "f1": 2, "v2": 3, "v1": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.6.adm
new file mode 100644
index 0000000..ccb58cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.6.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test1", "DatasetName": "vA", "DatatypeDataverseName": "Metadata", "DatatypeName": "AnyObject", "ViewDetails": { "Definition": "select v1.*, v2.*, f1.*, f2.*, s1.*, s2.*, d1.*\n from\n v1 v1,\n test2.v2 v2,\n f1() f1,\n test2.f2() f2,\n (\n select count(*) as s1 from DataverseSyn1\n where DataverseName like \"te%\"\n ) s1,\n (\n select count(*) as s2 from test2.DataverseSyn2\n where DataverseName like \"tes%\"\n ) s2,\n (\n select count(*) as d1 from Metadata.`Dataverse`\n where DataverseName like \"test%\"\n ) d1", "Dependencies": [ [ [ "test1", "v1" ], [ "test2", "v2" ], [ "Metadata", "Dataverse" ] ], [ [ "test1", "f1", "0" ], [ "test2", "f2", "0" ] ], [ ], [ [ "test1", "DataverseSyn1" ], [ "test2", "DataverseSyn2" ] ] ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.8.adm
new file mode 100644
index 0000000..7b4ad0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.8.adm
@@ -0,0 +1,2 @@
+{ "r": 1 }
+{ "r": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.9.adm
new file mode 100644
index 0000000..991315f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-1/create-view-1.9.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test1", "DatasetName": "vB", "DatatypeDataverseName": "Metadata", "DatatypeName": "AnyObject", "ViewDetails": { "Definition": "select r from range(1,2) r", "Dependencies": [ [ ], [ ], [ ] ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.10.adm
new file mode 100644
index 0000000..af0378d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.10.adm
@@ -0,0 +1,3 @@
+{ "c_id": 2, "c_x": 0, "c_y": 0 }
+{ "c_id": 3, "c_x": 0, "c_y": 0 }
+{ "c_id": 4, "c_x": 0, "c_y": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.11.adm
new file mode 100644
index 0000000..c32dff1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.11.adm
@@ -0,0 +1,5 @@
+{ "c_id1": 0, "c_id2": 0 }
+{ "c_id1": 1, "c_id2": -1 }
+{ "c_id1": 2, "c_id2": -2 }
+{ "c_id1": 3, "c_id2": -3 }
+{ "c_id1": 4, "c_id2": -4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.12.adm
new file mode 100644
index 0000000..dffdb6e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.12.adm
@@ -0,0 +1,9 @@
+{ "DataverseName": "test1", "DatasetName": "v1", "DatatypeFields": [ { "FieldName": "c_id", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i8", "FieldType": "int8", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i16", "FieldType": "int16", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i32", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i64", "FieldType": "int64", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_f", "FieldType": "float", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_d", "FieldType": "double", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_b", "FieldType": "boolean", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_s", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_datetime", "FieldType": "datetime", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_date", "FieldType": "date", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_time", "FieldType": "time", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_dur", "FieldType": "duration", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_ymdur", "FieldType": "year-month-duration", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_dtdur", "FieldType": "day-time-duration", "IsNullable": true, "IsMissable": true } ], "ViewDetails": { "Definition": "t1", "Dependencies": [ [ [ "test1", "t1" ] ], [ ], [ ] ], "Default": null } }
+{ "DataverseName": "test1", "DatasetName": "v2_ref_type", "DatatypeFields": [ { "FieldName": "c_id", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i8", "FieldType": "int8", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i16", "FieldType": "int16", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i32", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_i64", "FieldType": "int64", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_f", "FieldType": "float", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_d", "FieldType": "double", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_b", "FieldType": "boolean", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_s", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_datetime", "FieldType": "datetime", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_date", "FieldType": "date", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_time", "FieldType": "time", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_dur", "FieldType": "duration", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_ymdur", "FieldType": "year-month-duration", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_dtdur", "FieldType": "day-time-duration", "IsNullable": true, "IsMissable": true } ], "ViewDetails": { "Definition": "select c_id,\n c_i8, c_i16, c_i32, c_i64, c_f, c_d,\n c_b, c_s,\n c_datetime, c_date, c_time,\n c_dur, c_ymdur, c_dtdur\n from t1", "Dependencies": [ [ [ "test1", "t1" ] ], [ ], [ ] ], "Default": null } }
+{ "DataverseName": "test1", "DatasetName": "v3_datetime_format", "DatatypeFields": [ { "FieldName": "c_id", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_datetime", "FieldType": "datetime", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_date", "FieldType": "date", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_time", "FieldType": "time", "IsNullable": true, "IsMissable": true } ], "ViewDetails": { "Definition": "t2", "Dependencies": [ [ [ "test1", "t2" ] ], [ ], [ ] ], "Default": null, "DataFormat": [ "MM/DD/YYYY hh:mm:ss.nnna", "MM/DD/YYYY", "hh:mm:ss.nnna" ] } }
+{ "DataverseName": "test1", "DatasetName": "v4_date_format_only", "DatatypeFields": [ { "FieldName": "c_id", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_datetime", "FieldType": "datetime", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_date", "FieldType": "date", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_time", "FieldType": "time", "IsNullable": true, "IsMissable": true } ], "ViewDetails": { "Definition": "t2", "Dependencies": [ [ [ "test1", "t2" ] ], [ ], [ ] ], "Default": null, "DataFormat": [ null, "MM/DD/YYYY", null ] } }
+{ "DataverseName": "test1", "DatasetName": "v5_pk", "DatatypeFields": [ { "FieldName": "c_id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_datetime", "FieldType": "datetime", "IsNullable": true, "IsMissable": true } ], "ViewDetails": { "Definition": "t2", "Dependencies": [ [ [ "test1", "t2" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "c_id" ] ], "PrimaryKeyEnforced": false, "DataFormat": [ "MM/DD/YYYY hh:mm:ss.nnna", null, null ] } }
+{ "DataverseName": "test1", "DatasetName": "v6_pk_no_nulls", "DatatypeFields": [ { "FieldName": "c_i64", "FieldType": "int64", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_id", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ], "ViewDetails": { "Definition": "t1", "Dependencies": [ [ [ "test1", "t1" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "c_i64" ] ], "PrimaryKeyEnforced": false } }
+{ "DataverseName": "test1", "DatasetName": "v7_no_nulls", "DatatypeFields": [ { "FieldName": "c_i64", "FieldType": "int64", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_id", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ], "ViewDetails": { "Definition": "t1", "Dependencies": [ [ [ "test1", "t1" ] ], [ ], [ ] ], "Default": null } }
+{ "DataverseName": "test1", "DatasetName": "v8_no_nulls_multi", "DatatypeFields": [ { "FieldName": "c_id", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_x", "FieldType": "int64", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int64", "IsNullable": false, "IsMissable": false } ], "ViewDetails": { "Definition": "select\n c_id,\n case when to_bigint(c_i32) >= 0 then to_bigint(c_i32) when to_bigint(c_i32) < 0 then null else 0 end as c_x,\n case when to_bigint(c_i64) >= 0 then null when to_bigint(c_i64) < 0 then to_bigint(c_i64) else 0 end as c_y\n from t1", "Dependencies": [ [ [ "test1", "t1" ] ], [ ], [ ] ], "Default": null } }
+{ "DataverseName": "test1", "DatasetName": "v9_pk_composite", "DatatypeFields": [ { "FieldName": "c_id1", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_id2", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ], "ViewDetails": { "Definition": "select c_id as c_id1, -c_id as c_id2\n from t1", "Dependencies": [ [ [ "test1", "t1" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "c_id1" ], [ "c_id2" ] ], "PrimaryKeyEnforced": false } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.3.adm
new file mode 100644
index 0000000..00b95d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.3.adm
@@ -0,0 +1,5 @@
+{ "c_id": 0, "c_i8": 8, "c_i16": 16, "c_i32": 32, "c_i64": 64, "c_f": 1.5, "c_d": 2.25, "c_b": false, "c_s": "abc", "c_datetime": datetime("2020-02-03T10:11:12.001"), "c_date": date("2020-02-03"), "c_time": time("10:11:12.001"), "c_dur": duration("P30Y10M25DT13H12M50S"), "c_ymdur": year-month-duration("P30Y10M"), "c_dtdur": day-time-duration("P25DT13H12M50S") }
+{ "c_id": 1, "c_i8": -8, "c_i16": -16, "c_i32": -32, "c_i64": -64, "c_f": -1.5, "c_d": -2.25, "c_b": true, "c_s": "xyz", "c_datetime": datetime("2021-04-05T01:02:03.999"), "c_date": date("2021-04-05"), "c_time": time("01:02:03.999"), "c_dur": duration("P1Y2M3DT4H5M6S"), "c_ymdur": year-month-duration("P1Y2M"), "c_dtdur": day-time-duration("P3DT4H5M6S") }
+{ "c_id": 2, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": null, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
+{ "c_id": 3, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": null, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
+{ "c_id": 4, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": true, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.4.adm
new file mode 100644
index 0000000..00b95d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.4.adm
@@ -0,0 +1,5 @@
+{ "c_id": 0, "c_i8": 8, "c_i16": 16, "c_i32": 32, "c_i64": 64, "c_f": 1.5, "c_d": 2.25, "c_b": false, "c_s": "abc", "c_datetime": datetime("2020-02-03T10:11:12.001"), "c_date": date("2020-02-03"), "c_time": time("10:11:12.001"), "c_dur": duration("P30Y10M25DT13H12M50S"), "c_ymdur": year-month-duration("P30Y10M"), "c_dtdur": day-time-duration("P25DT13H12M50S") }
+{ "c_id": 1, "c_i8": -8, "c_i16": -16, "c_i32": -32, "c_i64": -64, "c_f": -1.5, "c_d": -2.25, "c_b": true, "c_s": "xyz", "c_datetime": datetime("2021-04-05T01:02:03.999"), "c_date": date("2021-04-05"), "c_time": time("01:02:03.999"), "c_dur": duration("P1Y2M3DT4H5M6S"), "c_ymdur": year-month-duration("P1Y2M"), "c_dtdur": day-time-duration("P3DT4H5M6S") }
+{ "c_id": 2, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": null, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
+{ "c_id": 3, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": null, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
+{ "c_id": 4, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": true, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.5.adm
new file mode 100644
index 0000000..2745452
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.5.adm
@@ -0,0 +1,3 @@
+{ "c_id": 0, "c_datetime": datetime("2020-02-20T23:40:41.001"), "c_date": date("2020-02-20"), "c_time": time("23:40:41.001") }
+{ "c_id": 1, "c_datetime": datetime("2021-11-25T11:50:51.999"), "c_date": date("2021-11-25"), "c_time": time("11:50:51.999") }
+{ "c_id": 2, "c_datetime": null, "c_date": null, "c_time": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.6.adm
new file mode 100644
index 0000000..4eab90a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.6.adm
@@ -0,0 +1,3 @@
+{ "c_id": 0, "c_date": date("2020-02-20") }
+{ "c_id": 1, "c_date": date("2021-11-25") }
+{ "c_id": 2, "c_date": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.7.adm
new file mode 100644
index 0000000..83cff58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.7.adm
@@ -0,0 +1,3 @@
+{ "c_id": 0 }
+{ "c_id": 1 }
+{ "c_id": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.8.adm
new file mode 100644
index 0000000..6136866
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.8.adm
@@ -0,0 +1,2 @@
+{ "c_id": 0, "c_i64": 64 }
+{ "c_id": 1, "c_i64": -64 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.9.adm
new file mode 100644
index 0000000..6136866
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-3-typed/create-view-3-typed.9.adm
@@ -0,0 +1,2 @@
+{ "c_id": 0, "c_i64": 64 }
+{ "c_id": 1, "c_i64": -64 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-4-typed-warn/create-view-4-typed-warn.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-4-typed-warn/create-view-4-typed-warn.3.adm
new file mode 100644
index 0000000..3c45dd1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-4-typed-warn/create-view-4-typed-warn.3.adm
@@ -0,0 +1,2 @@
+{ "c_id": 0, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": null, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
+{ "c_id": 1, "c_i8": null, "c_i16": null, "c_i32": null, "c_i64": null, "c_f": null, "c_d": null, "c_b": null, "c_s": null, "c_datetime": null, "c_date": null, "c_time": null, "c_dur": null, "c_ymdur": null, "c_dtdur": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-5-typed-warn/create-view-5-typed-warn.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-5-typed-warn/create-view-5-typed-warn.3.adm
new file mode 100644
index 0000000..74333a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-5-typed-warn/create-view-5-typed-warn.3.adm
@@ -0,0 +1 @@
+{ "c_id": 0, "c_datetime": null, "c_date": null, "c_time": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-7-foreign-key/create-view-7-foreign-key.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-7-foreign-key/create-view-7-foreign-key.2.adm
new file mode 100644
index 0000000..3af588c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-7-foreign-key/create-view-7-foreign-key.2.adm
@@ -0,0 +1,3 @@
+{ "DataverseName": "test1", "DatasetName": "employee_v1", "ViewDetails": { "Definition": "employee", "Dependencies": [ [ [ "test1", "employee" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "e_id" ] ], "PrimaryKeyEnforced": false, "ForeignKeys": [ { "ForeignKey": [ [ "e_mgr_id" ] ], "RefDataverseName": "test1", "RefDatasetName": "employee_v1", "IsEnforced": false } ] } }
+{ "DataverseName": "test1", "DatasetName": "employee_v2", "ViewDetails": { "Definition": "employee", "Dependencies": [ [ [ "test1", "employee" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "e_id" ] ], "PrimaryKeyEnforced": false, "ForeignKeys": [ { "ForeignKey": [ [ "e_mgr_id" ] ], "RefDataverseName": "test1", "RefDatasetName": "employee_v2", "IsEnforced": false }, { "ForeignKey": [ [ "e_hrr_id" ] ], "RefDataverseName": "test1", "RefDatasetName": "employee_v2", "IsEnforced": false } ] } }
+{ "DataverseName": "test2", "DatasetName": "employee2_v1", "ViewDetails": { "Definition": "employee2", "Dependencies": [ [ [ "test2", "employee2" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "e_id1" ], [ "e_id2" ] ], "PrimaryKeyEnforced": false, "ForeignKeys": [ { "ForeignKey": [ [ "e_mgr_id1" ], [ "e_mgr_id2" ] ], "RefDataverseName": "test2", "RefDatasetName": "employee2_v1", "IsEnforced": false }, { "ForeignKey": [ [ "e_hrr_id1" ], [ "e_hrr_id2" ] ], "RefDataverseName": "test2", "RefDatasetName": "employee2_v1", "IsEnforced": false } ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-7-foreign-key/create-view-7-foreign-key.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-7-foreign-key/create-view-7-foreign-key.4.adm
new file mode 100644
index 0000000..1391c82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/create-view-7-foreign-key/create-view-7-foreign-key.4.adm
@@ -0,0 +1,3 @@
+{ "DataverseName": "test1", "DatasetName": "orders_v", "ViewDetails": { "Definition": "test2.orders", "Dependencies": [ [ [ "test2", "orders" ], [ "test2", "customers_v" ], [ "test1", "stores_v" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "o_id" ] ], "PrimaryKeyEnforced": false, "ForeignKeys": [ { "ForeignKey": [ [ "o_cid" ] ], "RefDataverseName": "test2", "RefDatasetName": "customers_v", "IsEnforced": false }, { "ForeignKey": [ [ "o_sidX" ], [ "o_sidY" ] ], "RefDataverseName": "test1", "RefDatasetName": "stores_v", "IsEnforced": false } ] } }
+{ "DataverseName": "test1", "DatasetName": "stores_v", "ViewDetails": { "Definition": "test2.stores", "Dependencies": [ [ [ "test2", "stores" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "s_id1" ], [ "s_id2" ] ], "PrimaryKeyEnforced": false } }
+{ "DataverseName": "test2", "DatasetName": "customers_v", "ViewDetails": { "Definition": "customers", "Dependencies": [ [ [ "test2", "customers" ] ], [ ], [ ] ], "Default": null, "PrimaryKey": [ [ "c_id" ] ], "PrimaryKeyEnforced": false } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-dataverse-1/drop-dataverse-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-dataverse-1/drop-dataverse-1.1.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-dataverse-1/drop-dataverse-1.1.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.2.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.2.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.4.adm
new file mode 100644
index 0000000..bacb60c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.4.adm
@@ -0,0 +1 @@
+{ "cnt": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.5.adm
new file mode 100644
index 0000000..0b280d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-1/drop-view-1.5.adm
@@ -0,0 +1 @@
+{ "DatatypeName": "t3" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.10.adm
new file mode 100644
index 0000000..7b4ad0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.10.adm
@@ -0,0 +1,2 @@
+{ "r": 1 }
+{ "r": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.12.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.12.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.14.adm
new file mode 100644
index 0000000..7b4ad0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.14.adm
@@ -0,0 +1,2 @@
+{ "r": 1 }
+{ "r": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.16.adm
new file mode 100644
index 0000000..7b4ad0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.16.adm
@@ -0,0 +1,2 @@
+{ "r": 1 }
+{ "r": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.18.adm
new file mode 100644
index 0000000..feec5ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.18.adm
@@ -0,0 +1 @@
+{ "DatatypeName": "t1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.4.adm
new file mode 100644
index 0000000..7b4ad0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.4.adm
@@ -0,0 +1,2 @@
+{ "r": 1 }
+{ "r": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.6.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.6.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.8.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/drop-view-2-negative/drop-view-2-negative.8.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.3.adm
new file mode 100644
index 0000000..a42eb92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.3.adm
@@ -0,0 +1,10 @@
+{ "c1": 0, "c2": 9 }
+{ "c1": 1, "c2": 8 }
+{ "c1": 2, "c2": 7 }
+{ "c1": 3, "c2": 6 }
+{ "c1": 4, "c2": 5 }
+{ "c1": 5, "c2": 4 }
+{ "c1": 6, "c2": 3 }
+{ "c1": 7, "c2": 2 }
+{ "c1": 8, "c2": 1 }
+{ "c1": 9, "c2": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.4.adm
new file mode 100644
index 0000000..0c0c564
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.4.adm
@@ -0,0 +1,3 @@
+{ "Kind": "FUNCTION", "Name": "f5", "Definition": "select c1, c2 from s4", "Dependencies": [ [ ], [ ], [ ], [ [ "test1", "s4" ] ] ] }
+{ "Kind": "VIEW", "Name": "v3", "Definition": "select c1, c2 from s2", "Dependencies": [ [ ], [ ], [ ], [ [ "test1", "s2" ] ] ] }
+{ "Kind": "VIEW", "Name": "v6", "Definition": "select c1, c2 from f5() f5", "Dependencies": [ [ ], [ [ "test1", "f5", "0" ] ], [ ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.7.adm
new file mode 100644
index 0000000..554bf72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.7.adm
@@ -0,0 +1 @@
+{ "c": 36, "c1": [ 0, 1, 2 ], "c2": [ 0, 1, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.8.adm
new file mode 100644
index 0000000..98d2942
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/view/view-1/view-1.8.adm
@@ -0,0 +1,5 @@
+{ "Kind": "FUNCTION", "Name": "f5", "Dependencies": [ [ [ "test2", "v3" ] ], [ ], [ ] ] }
+{ "Kind": "FUNCTION", "Name": "f6", "Dependencies": [ [ [ "test2", "v4" ] ], [ ], [ ] ] }
+{ "Kind": "VIEW", "Name": "v3", "Dependencies": [ [ [ "test2", "ds1" ], [ "test2", "ds2" ] ], [ ], [ ] ] }
+{ "Kind": "VIEW", "Name": "v4", "Dependencies": [ [ [ "test2", "ds1" ], [ "test2", "ds2" ] ], [ ], [ ] ] }
+{ "Kind": "VIEW", "Name": "v9", "Dependencies": [ [ [ "test2", "ds1" ], [ "test2", "ds2" ], [ "test2", "v3" ], [ "test2", "v4" ] ], [ [ "test2", "f5", "0" ], [ "test2", "f6", "0" ] ], [ ], [ [ "test2", "s7" ], [ "test2", "s8" ] ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm
new file mode 100644
index 0000000..8e98bdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm
@@ -0,0 +1,4 @@
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 1 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 2 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 3 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.113.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.113.adm
index 3b7f6d7..9a55a1e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.113.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.113.adm
@@ -1,10 +1,10 @@
-{ "id": 1, "f_timetz": time("11:00:00.000Z"), "first_value": 1, "last_value": 3 }
-{ "id": 2, "f_timetz": time("12:00:00.000Z"), "first_value": 1, "last_value": 4 }
-{ "id": 3, "f_timetz": time("13:00:00.000Z"), "first_value": 2, "last_value": 5 }
-{ "id": 4, "f_timetz": time("14:00:00.000Z"), "first_value": 3, "last_value": 6 }
-{ "id": 5, "f_timetz": time("15:00:00.000Z"), "first_value": 4, "last_value": 7 }
-{ "id": 6, "f_timetz": time("16:00:00.000Z"), "first_value": 5, "last_value": 8 }
-{ "id": 7, "f_timetz": time("17:00:00.000Z"), "first_value": 6, "last_value": 9 }
-{ "id": 8, "f_timetz": time("18:00:00.000Z"), "first_value": 7, "last_value": 10 }
-{ "id": 9, "f_timetz": time("19:00:00.000Z"), "first_value": 8, "last_value": 10 }
-{ "id": 10, "f_timetz": time("20:00:00.000Z"), "first_value": 9, "last_value": 10 }
\ No newline at end of file
+{ "id": 1, "f_timetz": time("11:00:00.000"), "first_value": 1, "last_value": 3 }
+{ "id": 2, "f_timetz": time("12:00:00.000"), "first_value": 1, "last_value": 4 }
+{ "id": 3, "f_timetz": time("13:00:00.000"), "first_value": 2, "last_value": 5 }
+{ "id": 4, "f_timetz": time("14:00:00.000"), "first_value": 3, "last_value": 6 }
+{ "id": 5, "f_timetz": time("15:00:00.000"), "first_value": 4, "last_value": 7 }
+{ "id": 6, "f_timetz": time("16:00:00.000"), "first_value": 5, "last_value": 8 }
+{ "id": 7, "f_timetz": time("17:00:00.000"), "first_value": 6, "last_value": 9 }
+{ "id": 8, "f_timetz": time("18:00:00.000"), "first_value": 7, "last_value": 10 }
+{ "id": 9, "f_timetz": time("19:00:00.000"), "first_value": 8, "last_value": 10 }
+{ "id": 10, "f_timetz": time("20:00:00.000"), "first_value": 9, "last_value": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.114.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.114.adm
index b8fe968..1ffcc90 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.114.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.114.adm
@@ -1,10 +1,10 @@
-{ "id": 10, "f_timetz": time("20:00:00.000Z"), "first_value": 10, "last_value": 8 }
-{ "id": 9, "f_timetz": time("19:00:00.000Z"), "first_value": 10, "last_value": 7 }
-{ "id": 8, "f_timetz": time("18:00:00.000Z"), "first_value": 9, "last_value": 6 }
-{ "id": 7, "f_timetz": time("17:00:00.000Z"), "first_value": 8, "last_value": 5 }
-{ "id": 6, "f_timetz": time("16:00:00.000Z"), "first_value": 7, "last_value": 4 }
-{ "id": 5, "f_timetz": time("15:00:00.000Z"), "first_value": 6, "last_value": 3 }
-{ "id": 4, "f_timetz": time("14:00:00.000Z"), "first_value": 5, "last_value": 2 }
-{ "id": 3, "f_timetz": time("13:00:00.000Z"), "first_value": 4, "last_value": 1 }
-{ "id": 2, "f_timetz": time("12:00:00.000Z"), "first_value": 3, "last_value": 1 }
-{ "id": 1, "f_timetz": time("11:00:00.000Z"), "first_value": 2, "last_value": 1 }
\ No newline at end of file
+{ "id": 10, "f_timetz": time("20:00:00.000"), "first_value": 10, "last_value": 8 }
+{ "id": 9, "f_timetz": time("19:00:00.000"), "first_value": 10, "last_value": 7 }
+{ "id": 8, "f_timetz": time("18:00:00.000"), "first_value": 9, "last_value": 6 }
+{ "id": 7, "f_timetz": time("17:00:00.000"), "first_value": 8, "last_value": 5 }
+{ "id": 6, "f_timetz": time("16:00:00.000"), "first_value": 7, "last_value": 4 }
+{ "id": 5, "f_timetz": time("15:00:00.000"), "first_value": 6, "last_value": 3 }
+{ "id": 4, "f_timetz": time("14:00:00.000"), "first_value": 5, "last_value": 2 }
+{ "id": 3, "f_timetz": time("13:00:00.000"), "first_value": 4, "last_value": 1 }
+{ "id": 2, "f_timetz": time("12:00:00.000"), "first_value": 3, "last_value": 1 }
+{ "id": 1, "f_timetz": time("11:00:00.000"), "first_value": 2, "last_value": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.115.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.115.adm
index 1e26eb8..3b822e0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.115.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.115.adm
@@ -1,10 +1,10 @@
-{ "id": 1, "f_timestamptz": datetime("2000-10-19T10:23:54.000Z"), "first_value": 1, "last_value": 2 }
-{ "id": 2, "f_timestamptz": datetime("2001-10-19T10:23:54.000Z"), "first_value": 1, "last_value": 3 }
-{ "id": 3, "f_timestamptz": datetime("2002-10-19T10:23:54.000Z"), "first_value": 2, "last_value": 4 }
-{ "id": 4, "f_timestamptz": datetime("2003-10-19T10:23:54.000Z"), "first_value": 3, "last_value": 5 }
-{ "id": 5, "f_timestamptz": datetime("2004-10-19T10:23:54.000Z"), "first_value": 4, "last_value": 6 }
-{ "id": 6, "f_timestamptz": datetime("2005-10-19T10:23:54.000Z"), "first_value": 5, "last_value": 7 }
-{ "id": 7, "f_timestamptz": datetime("2006-10-19T10:23:54.000Z"), "first_value": 6, "last_value": 8 }
-{ "id": 8, "f_timestamptz": datetime("2007-10-19T10:23:54.000Z"), "first_value": 7, "last_value": 9 }
-{ "id": 9, "f_timestamptz": datetime("2008-10-19T10:23:54.000Z"), "first_value": 8, "last_value": 10 }
-{ "id": 10, "f_timestamptz": datetime("2009-10-19T10:23:54.000Z"), "first_value": 9, "last_value": 10 }
\ No newline at end of file
+{ "id": 1, "f_timestamptz": datetime("2000-10-19T10:23:54.000"), "first_value": 1, "last_value": 2 }
+{ "id": 2, "f_timestamptz": datetime("2001-10-19T10:23:54.000"), "first_value": 1, "last_value": 3 }
+{ "id": 3, "f_timestamptz": datetime("2002-10-19T10:23:54.000"), "first_value": 2, "last_value": 4 }
+{ "id": 4, "f_timestamptz": datetime("2003-10-19T10:23:54.000"), "first_value": 3, "last_value": 5 }
+{ "id": 5, "f_timestamptz": datetime("2004-10-19T10:23:54.000"), "first_value": 4, "last_value": 6 }
+{ "id": 6, "f_timestamptz": datetime("2005-10-19T10:23:54.000"), "first_value": 5, "last_value": 7 }
+{ "id": 7, "f_timestamptz": datetime("2006-10-19T10:23:54.000"), "first_value": 6, "last_value": 8 }
+{ "id": 8, "f_timestamptz": datetime("2007-10-19T10:23:54.000"), "first_value": 7, "last_value": 9 }
+{ "id": 9, "f_timestamptz": datetime("2008-10-19T10:23:54.000"), "first_value": 8, "last_value": 10 }
+{ "id": 10, "f_timestamptz": datetime("2009-10-19T10:23:54.000"), "first_value": 9, "last_value": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.116.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.116.adm
index ae6f718..db69f8f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.116.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/pg_win/pg_win.116.adm
@@ -1,10 +1,10 @@
-{ "id": 10, "f_timestamptz": datetime("2009-10-19T10:23:54.000Z"), "first_value": 10, "last_value": 9 }
-{ "id": 9, "f_timestamptz": datetime("2008-10-19T10:23:54.000Z"), "first_value": 10, "last_value": 8 }
-{ "id": 8, "f_timestamptz": datetime("2007-10-19T10:23:54.000Z"), "first_value": 9, "last_value": 7 }
-{ "id": 7, "f_timestamptz": datetime("2006-10-19T10:23:54.000Z"), "first_value": 8, "last_value": 6 }
-{ "id": 6, "f_timestamptz": datetime("2005-10-19T10:23:54.000Z"), "first_value": 7, "last_value": 5 }
-{ "id": 5, "f_timestamptz": datetime("2004-10-19T10:23:54.000Z"), "first_value": 6, "last_value": 4 }
-{ "id": 4, "f_timestamptz": datetime("2003-10-19T10:23:54.000Z"), "first_value": 5, "last_value": 3 }
-{ "id": 3, "f_timestamptz": datetime("2002-10-19T10:23:54.000Z"), "first_value": 4, "last_value": 2 }
-{ "id": 2, "f_timestamptz": datetime("2001-10-19T10:23:54.000Z"), "first_value": 3, "last_value": 1 }
-{ "id": 1, "f_timestamptz": datetime("2000-10-19T10:23:54.000Z"), "first_value": 2, "last_value": 1 }
\ No newline at end of file
+{ "id": 10, "f_timestamptz": datetime("2009-10-19T10:23:54.000"), "first_value": 10, "last_value": 9 }
+{ "id": 9, "f_timestamptz": datetime("2008-10-19T10:23:54.000"), "first_value": 10, "last_value": 8 }
+{ "id": 8, "f_timestamptz": datetime("2007-10-19T10:23:54.000"), "first_value": 9, "last_value": 7 }
+{ "id": 7, "f_timestamptz": datetime("2006-10-19T10:23:54.000"), "first_value": 8, "last_value": 6 }
+{ "id": 6, "f_timestamptz": datetime("2005-10-19T10:23:54.000"), "first_value": 7, "last_value": 5 }
+{ "id": 5, "f_timestamptz": datetime("2004-10-19T10:23:54.000"), "first_value": 6, "last_value": 4 }
+{ "id": 4, "f_timestamptz": datetime("2003-10-19T10:23:54.000"), "first_value": 5, "last_value": 3 }
+{ "id": 3, "f_timestamptz": datetime("2002-10-19T10:23:54.000"), "first_value": 4, "last_value": 2 }
+{ "id": 2, "f_timestamptz": datetime("2001-10-19T10:23:54.000"), "first_value": 3, "last_value": 1 }
+{ "id": 1, "f_timestamptz": datetime("2000-10-19T10:23:54.000"), "first_value": 2, "last_value": 1 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_null_missing/win_null_missing.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_null_missing/win_null_missing.4.adm
new file mode 100644
index 0000000..9818aa3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_null_missing/win_null_missing.4.adm
@@ -0,0 +1,6 @@
+{ "w0": null, "w1": "m", "w2": "n", "w3": "i", "w4": "s", "w5": "a", "w6": "o", "y": "m" }
+{ "w0": null, "w1": "m", "w2": "n", "w3": "i", "w4": "s", "w5": "a", "w6": "o", "x": null, "y": "n" }
+{ "w0": null, "w1": "m", "w2": "n", "w3": "i", "w4": "s", "w5": "a", "w6": "o", "x": 1, "y": "i" }
+{ "w0": null, "w1": "m", "w2": "n", "w3": "i", "w4": "s", "w5": "a", "w6": "o", "x": "a", "y": "s" }
+{ "w0": null, "w1": "m", "w2": "n", "w3": "i", "w4": "s", "w5": "a", "w6": "o", "x": [ "b" ], "y": "a" }
+{ "w0": null, "w1": "m", "w2": "n", "w3": "i", "w4": "s", "w5": "a", "w6": "o", "x": { "c": 1 }, "y": "o" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_null_missing/win_null_missing.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_null_missing/win_null_missing.5.adm
new file mode 100644
index 0000000..6aac76f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/win_null_missing/win_null_missing.5.adm
@@ -0,0 +1,6 @@
+{ "w0": null, "w1": "i", "w2": "s", "w3": "a", "w4": "o", "w5": "m", "w6": "n", "y": "m" }
+{ "w0": null, "w1": "i", "w2": "s", "w3": "a", "w4": "o", "w5": "m", "w6": "n", "x": null, "y": "n" }
+{ "w0": null, "w1": "i", "w2": "s", "w3": "a", "w4": "o", "w5": "m", "w6": "n", "x": 1, "y": "i" }
+{ "w0": null, "w1": "i", "w2": "s", "w3": "a", "w4": "o", "w5": "m", "w6": "n", "x": "a", "y": "s" }
+{ "w0": null, "w1": "i", "w2": "s", "w3": "a", "w4": "o", "w5": "m", "w6": "n", "x": [ "b" ], "y": "a" }
+{ "w0": null, "w1": "i", "w2": "s", "w3": "a", "w4": "o", "w5": "m", "w6": "n", "x": { "c": 1 }, "y": "o" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/comparison/datetime_tzeq/datetime_tzeq.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/comparison/datetime_tzeq/datetime_tzeq.4.ast
new file mode 100644
index 0000000..e8b3eac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/comparison/datetime_tzeq/datetime_tzeq.4.ast
@@ -0,0 +1,55 @@
+Query:
+RecordConstructor [
+ (
+ LiteralExpr [STRING] [result1]
+ :
+ OperatorExpr [
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00+04:00]
+ ]
+ =
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00+05:00]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result2]
+ :
+ OperatorExpr [
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00+04:00]
+ ]
+ =
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00Z]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result3]
+ :
+ OperatorExpr [
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00+04:00]
+ ]
+ =
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result4]
+ :
+ OperatorExpr [
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00Z]
+ ]
+ =
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2000-03-01T02:00:00]
+ ]
+ ]
+ )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/binary_01/binary_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/binary_01/binary_01.3.ast
index a4d112a..62a4c3a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/binary_01/binary_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/binary_01/binary_01.3.ast
@@ -1,44 +1,157 @@
-DataverseUse test
Query:
-OrderedListConstructor [
- FunctionCall asterix.hex@1[
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
LiteralExpr [STRING] [ABCDEF0123456789]
- ]
- FunctionCall asterix.hex@1[
LiteralExpr [STRING] [abcdef0123456789]
- ]
- FunctionCall asterix.hex@1[
LiteralExpr [STRING] [0A0B0C0D0E0F]
- ]
- FunctionCall asterix.hex@1[
LiteralExpr [STRING] [01020304050607080900]
- ]
- FunctionCall asterix.hex@1[
LiteralExpr [STRING] []
- ]
- FunctionCall asterix.hex@1[
FunctionCall asterix.hex@1[
LiteralExpr [STRING] [ABCDEF0123456789]
]
]
- FunctionCall asterix.base64@1[
- LiteralExpr [STRING] [0123456789qwertyuiopasdfghjklzxcvbnmQWERTYUIOPLKJHGFDSAZXCVBNM+/]
- ]
- FunctionCall asterix.base64@1[
- LiteralExpr [STRING] []
- ]
- FunctionCall asterix.base64@1[
- LiteralExpr [STRING] [QXN0ZXJpeA==]
- ]
- FunctionCall asterix.base64@1[
- LiteralExpr [STRING] [QXN0ZXJpeAE=]
- ]
- FunctionCall asterix.base64@1[
- LiteralExpr [STRING] [QXN0ZXJpeAE8]
- ]
- FunctionCall asterix.base64@1[
- FunctionCall asterix.base64@1[
- LiteralExpr [STRING] [QXN0ZXJpeAE8]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-01]
+ ]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2020-01-01T00:00:00Z]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [00:00:00]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
]
]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.hex@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.hex@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.hex@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/binary_01/binary_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/binary_01/binary_01.4.ast
new file mode 100644
index 0000000..0705c45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/binary_01/binary_01.4.ast
@@ -0,0 +1,157 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [0123456789qwertyuiopasdfghjklzxcvbnmQWERTYUIOPLKJHGFDSAZXCVBNM+/]
+ LiteralExpr [STRING] []
+ LiteralExpr [STRING] [QXN0ZXJpeA==]
+ LiteralExpr [STRING] [QXN0ZXJpeAE=]
+ LiteralExpr [STRING] [QXN0ZXJpeAE8]
+ FunctionCall asterix.base64@1[
+ LiteralExpr [STRING] [QXN0ZXJpeAE8]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-01]
+ ]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2020-01-01T00:00:00Z]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [00:00:00]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.base64@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.base64@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.base64@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/boolean_01/boolean_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/boolean_01/boolean_01.3.ast
index f2b6571..70dceef 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/boolean_01/boolean_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/boolean_01/boolean_01.3.ast
@@ -1,27 +1,259 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [boolean1]
- :
- FunctionCall asterix.boolean@1[
- LiteralExpr [STRING] [true]
- ]
- )
- (
- LiteralExpr [STRING] [boolean2]
- :
+Let Variable [ Name=$testFalse ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [FALSE]
FunctionCall asterix.boolean@1[
LiteralExpr [STRING] [false]
]
- )
- (
- LiteralExpr [STRING] [boolean3]
- :
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [STRING] [NaN]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [STRING] [NaN]
+ ]
+ LiteralExpr [STRING] [false]
+ ]
+Let Variable [ Name=$testTrue ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [TRUE]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [1]
+ ]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [2]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [1]
+ ]
+ FunctionCall asterix.int16@1[
+ - LiteralExpr [LONG] [1]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [1]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [2]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [1]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [3]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [1]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [STRING] [INF]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [STRING] [-INF]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [1]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [STRING] [INF]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [STRING] [-INF]
+ ]
+ LiteralExpr [STRING] [true]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [TRUE]
+ LiteralExpr [STRING] [FALSE]
+ LiteralExpr [STRING] [abc]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [1970-01-01]
+ ]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1970-01-01T00:00:00Z]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [00:00:00]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ ]
+ RecordConstructor [
+ ]
+ RecordConstructor [
+ (
+ LiteralExpr [STRING] [a]
+ :
+ LiteralExpr [NULL]
+ )
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [FALSE]
+ expected
FunctionCall asterix.boolean@1[
- FunctionCall asterix.boolean@1[
- LiteralExpr [STRING] [false]
+ IndexAccessor [
+ Variable [ Name=$testFalse ]
+ Index: Variable [ Name=$i ]
]
]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testFalse ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall asterix.boolean@1[
+ IndexAccessor [
+ Variable [ Name=$testTrue ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testTrue ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.boolean@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [3]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.boolean@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
)
+ AS Variable [ Name=#1 ]
]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.3.ast
index 93eda8c..38dce16 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.3.ast
@@ -1,83 +1,170 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [date1]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [2010-10-30]
- ]
- )
- (
- LiteralExpr [STRING] [date2]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [1987-11-19]
- ]
- )
- (
- LiteralExpr [STRING] [date3]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [-1987-11-19]
- ]
- )
- (
- LiteralExpr [STRING] [date4]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [0001-12-27]
- ]
- )
- (
- LiteralExpr [STRING] [date5]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [-1951-12-27]
- ]
- )
- (
- LiteralExpr [STRING] [date6]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [-2043-11-19]
- ]
- )
- (
- LiteralExpr [STRING] [date7]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [-19280329]
- ]
- )
- (
- LiteralExpr [STRING] [date8]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [19280329]
- ]
- )
- (
- LiteralExpr [STRING] [date9]
- :
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [19000228]
- ]
- )
- (
- LiteralExpr [STRING] [date10]
- :
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [2010-10-30]
+ LiteralExpr [STRING] [1987-11-19]
+ LiteralExpr [STRING] [-1987-11-19]
+ LiteralExpr [STRING] [0001-12-27]
+ LiteralExpr [STRING] [-1951-12-27]
+ LiteralExpr [STRING] [-2043-11-19]
+ LiteralExpr [STRING] [-19280329]
+ LiteralExpr [STRING] [19280329]
+ LiteralExpr [STRING] [19000228]
+ LiteralExpr [STRING] [20000229]
FunctionCall asterix.date@1[
LiteralExpr [STRING] [20000229]
]
- )
- (
- LiteralExpr [STRING] [date11]
- :
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2010-10-30T01:02:03Z]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
FunctionCall asterix.date@1[
- FunctionCall asterix.date@1[
- LiteralExpr [STRING] [20000229]
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
]
]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.date@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.date@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
)
+ AS Variable [ Name=#1 ]
]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast
new file mode 100644
index 0000000..9aa4f45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/date_01/date_01.4.ast
@@ -0,0 +1,44 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [10/30/2010]
+ LiteralExpr [STRING] [31-12-2020]
+ ]
+Let Variable [ Name=$fmt ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [MM/DD/YYYY]
+ LiteralExpr [STRING] [DD-MM-YYYY]
+ ]
+SELECT [
+Variable [ Name=$i ]
+i
+FunctionCall asterix.date@2[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ IndexAccessor [
+ Variable [ Name=$fmt ]
+ Index: Variable [ Name=$i ]
+ ]
+]
+actual
+]
+FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+]
+Orderby
+ Variable [ Name=$i ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.3.ast
index cbe87ec..b348fda 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.3.ast
@@ -1,111 +1,170 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [datetime1]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [2010-10-30T10:50:56.999+05:45]
- ]
- )
- (
- LiteralExpr [STRING] [datetime2]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [2010-10-30T10:30:56.250-10:00]
- ]
- )
- (
- LiteralExpr [STRING] [datetime3]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [1987-11-19T09:20:00.200Z]
- ]
- )
- (
- LiteralExpr [STRING] [datetime4]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [1987-11-19T10:50:56Z]
- ]
- )
- (
- LiteralExpr [STRING] [datetime5]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-1987-11-19T10:50:56.099-05:30]
- ]
- )
- (
- LiteralExpr [STRING] [datetime6]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-0001-11-19T10:50:56.719Z]
- ]
- )
- (
- LiteralExpr [STRING] [datetime7]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [1951-12-27T12:20:15Z]
- ]
- )
- (
- LiteralExpr [STRING] [datetime8]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [2043-11-19T10:50:56.719Z]
- ]
- )
- (
- LiteralExpr [STRING] [datetime9]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-19280329T174937374-0630]
- ]
- )
- (
- LiteralExpr [STRING] [datetime10]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-19280329T174937374+0630]
- ]
- )
- (
- LiteralExpr [STRING] [datetime11]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-19280329T174937374]
- ]
- )
- (
- LiteralExpr [STRING] [datetime12]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-19280329T174937374+0630]
- ]
- )
- (
- LiteralExpr [STRING] [datetime13]
- :
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-19280329T17493737+0630]
- ]
- )
- (
- LiteralExpr [STRING] [datetime14]
- :
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [2010-10-30T10:50:56.999+05:45]
+ LiteralExpr [STRING] [2010-10-30T10:30:56.250-10:00]
+ LiteralExpr [STRING] [1987-11-19T09:20:00.200Z]
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ LiteralExpr [STRING] [-1987-11-19T10:50:56.099-05:30]
+ LiteralExpr [STRING] [-0001-11-19T10:50:56.719Z]
+ LiteralExpr [STRING] [1951-12-27T12:20:15Z]
+ LiteralExpr [STRING] [2043-11-19T10:50:56.719Z]
+ LiteralExpr [STRING] [-19280329T174937374-0630]
+ LiteralExpr [STRING] [-19280329T174937374+0630]
+ LiteralExpr [STRING] [-19280329T174937374]
+ LiteralExpr [STRING] [-19280329T174937374+0630]
+ LiteralExpr [STRING] [-19280329T17493737+0630]
+ LiteralExpr [STRING] [-19280301T05493737+0630]
FunctionCall asterix.datetime@1[
LiteralExpr [STRING] [-19280301T05493737+0630]
]
- )
- (
- LiteralExpr [STRING] [datetime15]
- :
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
FunctionCall asterix.datetime@1[
- FunctionCall asterix.datetime@1[
- LiteralExpr [STRING] [-19280301T05493737+0630]
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
]
]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.datetime@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.datetime@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
)
+ AS Variable [ Name=#1 ]
]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast
new file mode 100644
index 0000000..8e644d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/datetime_01/datetime_01.4.ast
@@ -0,0 +1,44 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [10/30/2010 58-57-10]
+ LiteralExpr [STRING] [31-12-2020 11/58/59]
+ ]
+Let Variable [ Name=$fmt ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [MM/DD/YYYY ss-mm-hh]
+ LiteralExpr [STRING] [DD-MM-YYYY hh/mm/ss]
+ ]
+SELECT [
+Variable [ Name=$i ]
+i
+FunctionCall asterix.datetime@2[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ IndexAccessor [
+ Variable [ Name=$fmt ]
+ Index: Variable [ Name=$i ]
+ ]
+]
+actual
+]
+FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+]
+Orderby
+ Variable [ Name=$i ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/double_01/double_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/double_01/double_01.3.ast
index ef36c04..70bfa5e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/double_01/double_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/double_01/double_01.3.ast
@@ -1,55 +1,166 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [double1]
- :
- FunctionCall asterix.double@1[
- LiteralExpr [STRING] [NaN]
- ]
- )
- (
- LiteralExpr [STRING] [double2]
- :
- FunctionCall asterix.double@1[
- LiteralExpr [STRING] [INF]
- ]
- )
- (
- LiteralExpr [STRING] [double3]
- :
- FunctionCall asterix.double@1[
- LiteralExpr [STRING] [-INF]
- ]
- )
- (
- LiteralExpr [STRING] [double4]
- :
- FunctionCall asterix.double@1[
- LiteralExpr [STRING] [-80.20d]
- ]
- )
- (
- LiteralExpr [STRING] [double5]
- :
- FunctionCall asterix.double@1[
- LiteralExpr [STRING] [-20.56e-30]
- ]
- )
- (
- LiteralExpr [STRING] [double6]
- :
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [NaN]
+ LiteralExpr [STRING] [INF]
+ LiteralExpr [STRING] [-INF]
+ LiteralExpr [STRING] [-80.20d]
+ LiteralExpr [STRING] [-20.56e-30]
+ LiteralExpr [STRING] [-20.56e-300]
FunctionCall asterix.double@1[
LiteralExpr [STRING] [-20.56e-300]
]
- )
- (
- LiteralExpr [STRING] [double7]
- :
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [8]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [16]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [32]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [64]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [DOUBLE] [2.5]
+ ]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
FunctionCall asterix.double@1[
- FunctionCall asterix.double@1[
- LiteralExpr [STRING] [-20.56e-300]
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
]
]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.double@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.double@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
)
+ AS Variable [ Name=#1 ]
]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.3.ast
index d65e33c..3b037bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.3.ast
@@ -1,97 +1,172 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [duration1]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [P30Y10M25DT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration2]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [P25DT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration3]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [PT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration4]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [P30YT12MS]
- ]
- )
- (
- LiteralExpr [STRING] [duration5]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [PT13H]
- ]
- )
- (
- LiteralExpr [STRING] [duration6]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [-P30Y10M25DT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration7]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [-P25DT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration8]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [-PT13H50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration9]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [P120D]
- ]
- )
- (
- LiteralExpr [STRING] [duration10]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [-P28M]
- ]
- )
- (
- LiteralExpr [STRING] [duration11]
- :
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [PT29M90.937S]
- ]
- )
- (
- LiteralExpr [STRING] [duration12]
- :
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [P30Y10M25DT13H12M50S]
+ LiteralExpr [STRING] [P25DT13H12M50S]
+ LiteralExpr [STRING] [PT13H12M50S]
+ LiteralExpr [STRING] [P30YT12MS]
+ LiteralExpr [STRING] [PT13H]
+ LiteralExpr [STRING] [-P30Y10M25DT13H12M50S]
+ LiteralExpr [STRING] [-P25DT13H12M50S]
+ LiteralExpr [STRING] [-PT13H50S]
+ LiteralExpr [STRING] [P120D]
+ LiteralExpr [STRING] [-P28M]
+ LiteralExpr [STRING] [PT29M90.937S]
+ LiteralExpr [STRING] [P300Y15M60DT300H98M482.435S]
FunctionCall asterix.duration@1[
LiteralExpr [STRING] [P300Y15M60DT300H98M482.435S]
]
- )
- (
- LiteralExpr [STRING] [duration13]
- :
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P30Y10M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P25DT13H12M50S]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
FunctionCall asterix.duration@1[
- FunctionCall asterix.duration@1[
- LiteralExpr [STRING] [P300Y15M60DT300H98M482.435S]
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
]
]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.duration@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.duration@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
)
+ AS Variable [ Name=#1 ]
]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.4.ast
new file mode 100644
index 0000000..d1f7bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.4.ast
@@ -0,0 +1,165 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [P30Y10M]
+ LiteralExpr [STRING] [P30Y]
+ LiteralExpr [STRING] [-P30Y10M]
+ LiteralExpr [STRING] [-P28M]
+ LiteralExpr [STRING] [P300Y15M]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P300Y15M]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [P300Y16M60DT300H98M482.435S]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P25DT13H12M50S]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.year-month-duration@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.year-month-duration@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.year-month-duration@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.5.ast
new file mode 100644
index 0000000..7d8f323
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_01/duration_01.5.ast
@@ -0,0 +1,167 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [P25DT13H12M50S]
+ LiteralExpr [STRING] [PT13H12M50S]
+ LiteralExpr [STRING] [PT13H]
+ LiteralExpr [STRING] [-P25DT13H12M50S]
+ LiteralExpr [STRING] [-PT13H50S]
+ LiteralExpr [STRING] [P120D]
+ LiteralExpr [STRING] [PT29M90.937S]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [PT14H]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [P1Y2M3DT4H5M6S]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P1Y]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.day-time-duration@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.day-time-duration@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.day-time-duration@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.2.ast
deleted file mode 100644
index e69de29..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.2.ast
+++ /dev/null
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.3.ast
deleted file mode 100644
index e615b56..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/duration_02/duration_02.3.ast
+++ /dev/null
@@ -1,97 +0,0 @@
-DataverseUse test
-Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [duration1]
- :
- FunctionCall asterix.year-month-duration@1[
- LiteralExpr [STRING] [P30Y10M]
- ]
- )
- (
- LiteralExpr [STRING] [duration2]
- :
- FunctionCall asterix.day-time-duration@1[
- LiteralExpr [STRING] [P25DT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration3]
- :
- FunctionCall asterix.day-time-duration@1[
- LiteralExpr [STRING] [PT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration4]
- :
- FunctionCall asterix.year-month-duration@1[
- LiteralExpr [STRING] [P30Y]
- ]
- )
- (
- LiteralExpr [STRING] [duration5]
- :
- FunctionCall asterix.day-time-duration@1[
- LiteralExpr [STRING] [PT13H]
- ]
- )
- (
- LiteralExpr [STRING] [duration6]
- :
- FunctionCall asterix.year-month-duration@1[
- LiteralExpr [STRING] [-P30Y10M]
- ]
- )
- (
- LiteralExpr [STRING] [duration7]
- :
- FunctionCall asterix.day-time-duration@1[
- LiteralExpr [STRING] [-P25DT13H12M50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration8]
- :
- FunctionCall asterix.day-time-duration@1[
- LiteralExpr [STRING] [-PT13H50S]
- ]
- )
- (
- LiteralExpr [STRING] [duration9]
- :
- FunctionCall asterix.day-time-duration@1[
- LiteralExpr [STRING] [P120D]
- ]
- )
- (
- LiteralExpr [STRING] [duration10]
- :
- FunctionCall asterix.year-month-duration@1[
- LiteralExpr [STRING] [-P28M]
- ]
- )
- (
- LiteralExpr [STRING] [duration11]
- :
- FunctionCall asterix.day-time-duration@1[
- LiteralExpr [STRING] [PT29M90.937S]
- ]
- )
- (
- LiteralExpr [STRING] [duration12]
- :
- FunctionCall asterix.year-month-duration@1[
- LiteralExpr [STRING] [P300Y15M]
- ]
- )
- (
- LiteralExpr [STRING] [duration13]
- :
- FunctionCall asterix.year-month-duration@1[
- FunctionCall asterix.year-month-duration@1[
- LiteralExpr [STRING] [P300Y15M]
- ]
- ]
- )
-]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/float_01/float_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/float_01/float_01.3.ast
index 6a8e42b..bf2f7e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/float_01/float_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/float_01/float_01.3.ast
@@ -1,48 +1,165 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [float1]
- :
- FunctionCall asterix.float@1[
- LiteralExpr [STRING] [NaN]
- ]
- )
- (
- LiteralExpr [STRING] [float2]
- :
- FunctionCall asterix.float@1[
- LiteralExpr [STRING] [INF]
- ]
- )
- (
- LiteralExpr [STRING] [float3]
- :
- FunctionCall asterix.float@1[
- LiteralExpr [STRING] [-INF]
- ]
- )
- (
- LiteralExpr [STRING] [float4]
- :
- FunctionCall asterix.float@1[
- LiteralExpr [STRING] [-80.20]
- ]
- )
- (
- LiteralExpr [STRING] [float5]
- :
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [NaN]
+ LiteralExpr [STRING] [INF]
+ LiteralExpr [STRING] [-INF]
+ LiteralExpr [STRING] [-80.20]
+ LiteralExpr [STRING] [-20.56e-30]
FunctionCall asterix.float@1[
LiteralExpr [STRING] [-20.56e-30]
]
- )
- (
- LiteralExpr [STRING] [float6]
- :
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [8]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [16]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [32]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [64]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [DOUBLE] [2.5]
+ ]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
FunctionCall asterix.float@1[
- FunctionCall asterix.float@1[
- LiteralExpr [STRING] [-20.56e-30]
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
]
]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.float@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.float@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
)
+ AS Variable [ Name=#1 ]
]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.3.ast
index e346cec..c84907e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.3.ast
@@ -1,103 +1,188 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [int8]
- :
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [+80i8]
+ LiteralExpr [STRING] [-80]
FunctionCall asterix.int8@1[
LiteralExpr [STRING] [+80i8]
]
- )
- (
- LiteralExpr [STRING] [int16]
- :
FunctionCall asterix.int16@1[
- LiteralExpr [STRING] [160]
+ LiteralExpr [LONG] [16]
]
- )
- (
- LiteralExpr [STRING] [int32]
- :
FunctionCall asterix.int32@1[
- LiteralExpr [STRING] [+320i32]
+ LiteralExpr [LONG] [32]
]
- )
- (
- LiteralExpr [STRING] [int64]
- :
FunctionCall asterix.int64@1[
- LiteralExpr [STRING] [640]
+ LiteralExpr [LONG] [64]
]
- )
- (
- LiteralExpr [STRING] [int8_2]
- :
- FunctionCall asterix.int8@1[
- LiteralExpr [STRING] [-80]
+ FunctionCall asterix.float@1[
+ LiteralExpr [DOUBLE] [1.25]
]
- )
- (
- LiteralExpr [STRING] [int16_2]
- :
+ FunctionCall asterix.double@1[
+ LiteralExpr [DOUBLE] [2.25]
+ ]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ LiteralExpr [STRING] [80.25]
+ LiteralExpr [STRING] [80.5]
+ LiteralExpr [STRING] [80.75]
+ LiteralExpr [STRING] [-80.25]
+ LiteralExpr [STRING] [-80.5]
+ LiteralExpr [STRING] [-80.75]
+ LiteralExpr [STRING] [8.125e1]
+ LiteralExpr [STRING] [-812.5e-1]
FunctionCall asterix.int16@1[
- LiteralExpr [STRING] [-160i16]
+ LiteralExpr [LONG] [999]
]
- )
- (
- LiteralExpr [STRING] [int32_2]
- :
FunctionCall asterix.int32@1[
- LiteralExpr [STRING] [-320]
+ LiteralExpr [LONG] [999]
]
- )
- (
- LiteralExpr [STRING] [int64_2]
- :
FunctionCall asterix.int64@1[
- LiteralExpr [STRING] [-640i64]
+ LiteralExpr [LONG] [999]
]
- )
- (
- LiteralExpr [STRING] [int64_min]
- :
- FunctionCall asterix.int64@1[
- LiteralExpr [STRING] [-9223372036854775808]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [999]
]
- )
- (
- LiteralExpr [STRING] [int8_3]
- :
- FunctionCall asterix.int8@1[
- FunctionCall asterix.int8@1[
- LiteralExpr [STRING] [+80i8]
- ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [999]
]
- )
- (
- LiteralExpr [STRING] [int16_3]
- :
- FunctionCall asterix.int16@1[
- FunctionCall asterix.int16@1[
- LiteralExpr [STRING] [160]
- ]
+ LiteralExpr [STRING] [INF]
+ LiteralExpr [STRING] [-INF]
+ LiteralExpr [STRING] [NaN]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
]
- )
- (
- LiteralExpr [STRING] [int32_3]
- :
- FunctionCall asterix.int32@1[
- FunctionCall asterix.int32@1[
- LiteralExpr [STRING] [+320i32]
- ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
]
- )
- (
- LiteralExpr [STRING] [int64_3]
- :
- FunctionCall asterix.int64@1[
- FunctionCall asterix.int64@1[
- LiteralExpr [STRING] [640]
- ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
]
- )
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.int8@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.int8@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.int8@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.4.ast
new file mode 100644
index 0000000..f0ae023
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.4.ast
@@ -0,0 +1,185 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [160]
+ LiteralExpr [STRING] [-160i16]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [STRING] [160]
+ ]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [8]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [32]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [64]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [DOUBLE] [1.25]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [DOUBLE] [2.25]
+ ]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ LiteralExpr [STRING] [160.25]
+ LiteralExpr [STRING] [160.5]
+ LiteralExpr [STRING] [160.75]
+ LiteralExpr [STRING] [-160.25]
+ LiteralExpr [STRING] [-160.5]
+ LiteralExpr [STRING] [-160.75]
+ LiteralExpr [STRING] [16.125e1]
+ LiteralExpr [STRING] [-1612.5e-1]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [99999]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [99999]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [99999]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [99999]
+ ]
+ LiteralExpr [STRING] [INF]
+ LiteralExpr [STRING] [-INF]
+ LiteralExpr [STRING] [NaN]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.int16@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.int16@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.int16@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.5.ast
new file mode 100644
index 0000000..5190253
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.5.ast
@@ -0,0 +1,182 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [+320i32]
+ LiteralExpr [STRING] [-320]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [STRING] [+320i32]
+ ]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [8]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [16]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [64]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [DOUBLE] [1.25]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [DOUBLE] [2.25]
+ ]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ LiteralExpr [STRING] [320.25]
+ LiteralExpr [STRING] [320.5]
+ LiteralExpr [STRING] [320.75]
+ LiteralExpr [STRING] [-320.25]
+ LiteralExpr [STRING] [-320.5]
+ LiteralExpr [STRING] [-320.75]
+ LiteralExpr [STRING] [32.125e1]
+ LiteralExpr [STRING] [-3212.5e-1]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [9999999999]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [9999999999]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [9999999999]
+ ]
+ LiteralExpr [STRING] [INF]
+ LiteralExpr [STRING] [-INF]
+ LiteralExpr [STRING] [NaN]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.int32@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.int32@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.int32@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.6.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.6.ast
new file mode 100644
index 0000000..eba1e91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/int_01/int_01.6.ast
@@ -0,0 +1,180 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [640]
+ LiteralExpr [STRING] [-640i64]
+ LiteralExpr [STRING] [-9223372036854775808]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [STRING] [640]
+ ]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [8]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [16]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [32]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [DOUBLE] [1.25]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [DOUBLE] [2.25]
+ ]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ LiteralExpr [STRING] [640.25]
+ LiteralExpr [STRING] [640.5]
+ LiteralExpr [STRING] [640.75]
+ LiteralExpr [STRING] [-640.25]
+ LiteralExpr [STRING] [-640.5]
+ LiteralExpr [STRING] [-640.75]
+ LiteralExpr [STRING] [64.125e1]
+ LiteralExpr [STRING] [-6412.5e-1]
+ FunctionCall asterix.float@1[
+ LiteralExpr [STRING] [1e100]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [STRING] [1e100]
+ ]
+ LiteralExpr [STRING] [INF]
+ LiteralExpr [STRING] [-INF]
+ LiteralExpr [STRING] [NaN]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [1987-11-19T10:50:56Z]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.int64@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.int64@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.int64@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/string_01/string_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/string_01/string_01.3.ast
index 2696db5..5c6e26f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/string_01/string_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/string_01/string_01.3.ast
@@ -1,72 +1,167 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [string1]
- :
- FunctionCall asterix.string@1[
- LiteralExpr [STRING] [true]
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [true]
+ LiteralExpr [STRING] [false"]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [STRING] [8]
]
- )
- (
- LiteralExpr [STRING] [string2]
- :
- FunctionCall asterix.string@1[
- LiteralExpr [STRING] [false"]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [STRING] [16]
]
- )
- (
- LiteralExpr [STRING] [string3]
- :
- FunctionCall asterix.string@1[
- FunctionCall asterix.int8@1[
- LiteralExpr [STRING] [8]
- ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [STRING] [32]
]
- )
- (
- LiteralExpr [STRING] [string4]
- :
- FunctionCall asterix.string@1[
- FunctionCall asterix.int16@1[
- LiteralExpr [STRING] [16]
- ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [STRING] [64]
]
- )
- (
- LiteralExpr [STRING] [string5]
- :
- FunctionCall asterix.string@1[
- FunctionCall asterix.int32@1[
- LiteralExpr [STRING] [32]
- ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [STRING] [1.25]
]
- )
- (
- LiteralExpr [STRING] [string6]
- :
- FunctionCall asterix.string@1[
- FunctionCall asterix.int64@1[
- LiteralExpr [STRING] [64]
- ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [STRING] [2.5]
]
- )
- (
- LiteralExpr [STRING] [string7]
- :
- FunctionCall asterix.string@1[
- FunctionCall asterix.float@1[
- LiteralExpr [STRING] [1.25]
- ]
+ LiteralExpr [TRUE]
+ LiteralExpr [FALSE]
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2020-01-02T03:04:05Z]
]
- )
- (
- LiteralExpr [STRING] [string8]
- :
- FunctionCall asterix.string@1[
- FunctionCall asterix.double@1[
- LiteralExpr [STRING] [2.5]
- ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2021-01-02]
]
- )
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:02:03]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [P30Y10M25DT13H12M50S]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P2Y]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P4S]
+ ]
+ FunctionCall asterix.uuid@1[
+ LiteralExpr [STRING] [02a199ca-bf58-412e-bd9f-60a0c975a8ac]
+ ]
+ FunctionCall asterix.base64@1[
+ LiteralExpr [STRING] [0123456789qwertyuiopasdfghjklzxcvbnmQWERTYUIOPLKJHGFDSAZXCVBNM+/]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
+ FunctionCall asterix.string@1[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.string@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.string@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ )
+ AS Variable [ Name=#1 ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.3.ast
index 6c688cb..5eb4fb1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.3.ast
@@ -1,90 +1,171 @@
-DataverseUse test
Query:
-RecordConstructor [
- (
- LiteralExpr [STRING] [time1]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [10:50:56.200+05:00]
- ]
- )
- (
- LiteralExpr [STRING] [time2]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [10:50:56.200-10:15]
- ]
- )
- (
- LiteralExpr [STRING] [time3]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [10:50:56]
- ]
- )
- (
- LiteralExpr [STRING] [time4]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [10:50:56.200Z]
- ]
- )
- (
- LiteralExpr [STRING] [time5]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [23:59:59.999-13:30]
- ]
- )
- (
- LiteralExpr [STRING] [time6]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [00:00:00.000+14:45]
- ]
- )
- (
- LiteralExpr [STRING] [time7]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [12:59:00.019-01:00]
- ]
- )
- (
- LiteralExpr [STRING] [time8]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [12:59:00.01-01:00]
- ]
- )
- (
- LiteralExpr [STRING] [time9]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [12:59:00.019-01:00]
- ]
- )
- (
- LiteralExpr [STRING] [time10]
- :
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [12590001-0100]
- ]
- )
- (
- LiteralExpr [STRING] [time11]
- :
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [10:50:56.200+05:00]
+ LiteralExpr [STRING] [10:50:56.200-10:15]
+ LiteralExpr [STRING] [10:50:56]
+ LiteralExpr [STRING] [10:50:56.200Z]
+ LiteralExpr [STRING] [23:59:59.999-13:30]
+ LiteralExpr [STRING] [00:00:00.000+14:45]
+ LiteralExpr [STRING] [12:59:00.019-01:00]
+ LiteralExpr [STRING] [12:59:00.01-01:00]
+ LiteralExpr [STRING] [12:59:00.019-01:00]
+ LiteralExpr [STRING] [12590001-0100]
+ LiteralExpr [STRING] [125900019+0100]
FunctionCall asterix.time@1[
LiteralExpr [STRING] [125900019+0100]
]
- )
- (
- LiteralExpr [STRING] [time12]
- :
+ FunctionCall asterix.datetime@1[
+ LiteralExpr [STRING] [2010-10-30T01:02:03Z]
+ ]
+ ]
+Let Variable [ Name=$testNull ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [FALSE]
+ LiteralExpr [TRUE]
+ LiteralExpr [STRING] [@#!]
+ FunctionCall asterix.int8@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int16@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int32@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.int64@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.float@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.double@1[
+ LiteralExpr [LONG] [0]
+ ]
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2020-01-02]
+ ]
+ FunctionCall asterix.duration@1[
+ LiteralExpr [STRING] [PT0H]
+ ]
+ FunctionCall asterix.year-month-duration@1[
+ LiteralExpr [STRING] [P0Y0M]
+ ]
+ FunctionCall asterix.day-time-duration@1[
+ LiteralExpr [STRING] [P0D]
+ ]
+ OrderedListConstructor [
+ ]
+ RecordConstructor [
+ ]
+ ]
+Let Variable [ Name=$testMissing ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [MISSING]
+ ]
+SELECT ELEMENT [
+Variable [ Name=#1 ]
+]
+FROM [ (
+ SELECT [
+ LiteralExpr [LONG] [0]
+ g
+ Variable [ Name=$i ]
+ i
FunctionCall asterix.time@1[
- FunctionCall asterix.time@1[
- LiteralExpr [STRING] [125900019+0100]
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
]
]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [1]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [NULL]
+ expected
+ FunctionCall asterix.time@1[
+ IndexAccessor [
+ Variable [ Name=$testNull ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testNull ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
+ UNION
+ SELECT [
+ LiteralExpr [LONG] [2]
+ g
+ Variable [ Name=$i ]
+ i
+ LiteralExpr [TRUE]
+ expected
+ FunctionCall algebricks.is-missing@1[
+ FunctionCall asterix.time@1[
+ IndexAccessor [
+ Variable [ Name=$testMissing ]
+ Index: Variable [ Name=$i ]
+ ]
+ ]
+ ]
+ actual
+ ]
+ FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$testMissing ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+ ]
)
+ AS Variable [ Name=#1 ]
]
+Orderby
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=g
+ ]
+ ASC
+ FieldAccessor [
+ Variable [ Name=#1 ]
+ Field=i
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast
new file mode 100644
index 0000000..39c441c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/constructor/time_01/time_01.4.ast
@@ -0,0 +1,44 @@
+Query:
+Let Variable [ Name=$test ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [11/58/59]
+ LiteralExpr [STRING] [58-57-10]
+ ]
+Let Variable [ Name=$fmt ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [hh/mm/ss]
+ LiteralExpr [STRING] [ss-mm-hh]
+ ]
+SELECT [
+Variable [ Name=$i ]
+i
+FunctionCall asterix.time@2[
+ IndexAccessor [
+ Variable [ Name=$test ]
+ Index: Variable [ Name=$i ]
+ ]
+ IndexAccessor [
+ Variable [ Name=$fmt ]
+ Index: Variable [ Name=$i ]
+ ]
+]
+actual
+]
+FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ OperatorExpr [
+ FunctionCall asterix.len@1[
+ Variable [ Name=$test ]
+ ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$i ]
+]
+Orderby
+ Variable [ Name=$i ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/list/ordered-list-constructor_05/ordered-list-constructor_05.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/list/ordered-list-constructor_05/ordered-list-constructor_05.1.ast
new file mode 100644
index 0000000..cf6bb09
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/list/ordered-list-constructor_05/ordered-list-constructor_05.1.ast
@@ -0,0 +1,114 @@
+Query:
+SELECT [
+Variable [ Name=$r ]
+r
+OrderedListConstructor [
+ LiteralExpr [STRING] [foo]
+ LiteralExpr [STRING] [bar]
+ LiteralExpr [STRING] [foobar]
+]
+a1
+OrderedListConstructor [
+ Variable [ Name=$r ]
+ OrderedListConstructor [
+ Variable [ Name=$r ]
+ ]
+ LiteralExpr [NULL]
+]
+a2
+OrderedListConstructor [
+ (
+ SELECT ELEMENT [
+ FunctionCall asterix.sql-count@1[
+ (
+ SELECT ELEMENT [
+ FieldAccessor [
+ Variable [ Name=#3 ]
+ Field=x
+ ]
+ ]
+ FROM [ Variable [ Name=#1 ]
+ AS Variable [ Name=#3 ]
+ ]
+ )
+ ]
+ ]
+ FROM [ OrderedListConstructor [
+ Variable [ Name=$r ]
+ OperatorExpr [
+ Variable [ Name=$r ]
+ +
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$x ]
+ ]
+ Group All
+ GROUP AS Variable [ Name=#1 ]
+ (
+ x:=Variable [ Name=$x ]
+ )
+
+ )
+ (
+ SELECT ELEMENT [
+ FunctionCall asterix.sql-sum@1[
+ (
+ SELECT ELEMENT [
+ FieldAccessor [
+ Variable [ Name=#4 ]
+ Field=x
+ ]
+ ]
+ FROM [ Variable [ Name=#2 ]
+ AS Variable [ Name=#4 ]
+ ]
+ )
+ ]
+ ]
+ FROM [ OrderedListConstructor [
+ Variable [ Name=$r ]
+ OperatorExpr [
+ Variable [ Name=$r ]
+ +
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ AS Variable [ Name=$x ]
+ ]
+ Group All
+ GROUP AS Variable [ Name=#2 ]
+ (
+ x:=Variable [ Name=$x ]
+ )
+
+ )
+]
+a3
+OperatorExpr [
+ OrderedListConstructor [
+ Variable [ Name=$r ]
+ OrderedListConstructor [
+ Variable [ Name=$r ]
+ ]
+ ]
+ =
+ OrderedListConstructor [
+ Variable [ Name=$r ]
+ OrderedListConstructor [
+ Variable [ Name=$r ]
+ ]
+ ]
+]
+eq
+]
+FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [1]
+ LiteralExpr [LONG] [2]
+ ]
+ AS Variable [ Name=$r ]
+]
+Orderby
+ Variable [ Name=$r ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/scan/alltypes_01/alltypes_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/scan/alltypes_01/alltypes_01.4.ast
new file mode 100644
index 0000000..9cbf7f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/scan/alltypes_01/alltypes_01.4.ast
@@ -0,0 +1,20 @@
+DataverseUse test
+Query:
+RecordConstructor [
+ (
+ LiteralExpr [STRING] [t1_array_of_unknown]
+ :
+ OrderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [MISSING]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [t2_multiset_of_unknown]
+ :
+ UnorderedListConstructor [
+ LiteralExpr [NULL]
+ LiteralExpr [MISSING]
+ ]
+ )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/datetime_functions/datetime_functions.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/datetime_functions/datetime_functions.4.ast
new file mode 100644
index 0000000..c7cbe6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/datetime_functions/datetime_functions.4.ast
@@ -0,0 +1,110 @@
+Query:
+SELECT [
+Variable [ Name=$tz ]
+tz
+FunctionCall asterix.string@1[
+ Variable [ Name=$s1 ]
+]
+s1
+FunctionCall asterix.string@1[
+ Variable [ Name=$s2 ]
+]
+s2
+FunctionCall asterix.string@1[
+ Variable [ Name=$m1 ]
+]
+m1
+FunctionCall asterix.string@1[
+ Variable [ Name=$m2 ]
+]
+m2
+OperatorExpr [
+ FunctionCall asterix.unix-time-from-datetime-in-secs@1[
+ Variable [ Name=$s1 ]
+ ]
+ =
+ Variable [ Name=$ets ]
+]
+us1
+OperatorExpr [
+ FunctionCall asterix.unix-time-from-datetime-in-secs@2[
+ Variable [ Name=$s2 ]
+ Variable [ Name=$tz ]
+ ]
+ =
+ Variable [ Name=$ets ]
+]
+us2
+OperatorExpr [
+ FunctionCall asterix.unix-time-from-datetime-in-ms@1[
+ Variable [ Name=$m1 ]
+ ]
+ =
+ Variable [ Name=$etm ]
+]
+um1
+OperatorExpr [
+ FunctionCall asterix.unix-time-from-datetime-in-ms@2[
+ Variable [ Name=$m2 ]
+ Variable [ Name=$tz ]
+ ]
+ =
+ Variable [ Name=$etm ]
+]
+um2
+]
+FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [0]
+ LiteralExpr [LONG] [2]
+ ]
+ AS Variable [ Name=$r ]
+]
+Let Variable [ Name=$ets ]
+ :=
+ LiteralExpr [LONG] [1631671261]
+Let Variable [ Name=$etm ]
+ :=
+ OperatorExpr [
+ Variable [ Name=$ets ]
+ *
+ LiteralExpr [LONG] [1000]
+ ]
+Let Variable [ Name=$tzs ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [UTC]
+ LiteralExpr [STRING] [America/Los_Angeles]
+ LiteralExpr [STRING] [Asia/Tokyo]
+ ]
+Let Variable [ Name=$tz ]
+ :=
+ IndexAccessor [
+ Variable [ Name=$tzs ]
+ Index: Variable [ Name=$r ]
+ ]
+Let Variable [ Name=$s1 ]
+ :=
+ FunctionCall asterix.datetime-from-unix-time-in-secs@1[
+ Variable [ Name=$ets ]
+ ]
+Let Variable [ Name=$s2 ]
+ :=
+ FunctionCall asterix.datetime-from-unix-time-in-secs@2[
+ Variable [ Name=$ets ]
+ Variable [ Name=$tz ]
+ ]
+Let Variable [ Name=$m1 ]
+ :=
+ FunctionCall asterix.datetime-from-unix-time-in-ms@1[
+ Variable [ Name=$etm ]
+ ]
+Let Variable [ Name=$m2 ]
+ :=
+ FunctionCall asterix.datetime-from-unix-time-in-ms@2[
+ Variable [ Name=$etm ]
+ Variable [ Name=$tz ]
+ ]
+Orderby
+ Variable [ Name=$r ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/day_of_week_01/day_of_week_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/day_of_week_01/day_of_week_01.3.ast
index 3f525c1..eec1007 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/day_of_week_01/day_of_week_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/day_of_week_01/day_of_week_01.3.ast
@@ -4,46 +4,51 @@
(
LiteralExpr [STRING] [1970-01-01]
:
- FunctionCall asterix.day-of-week@1[
+ FunctionCall asterix.day-of-week@2[
FunctionCall asterix.date@1[
LiteralExpr [STRING] [1970-01-01]
]
+ LiteralExpr [STRING] [Mon]
]
)
(
LiteralExpr [STRING] [2013-08-06]
:
- FunctionCall asterix.day-of-week@1[
+ FunctionCall asterix.day-of-week@2[
FunctionCall asterix.date@1[
LiteralExpr [STRING] [2013-08-06]
]
+ LiteralExpr [STRING] [MON]
]
)
(
LiteralExpr [STRING] [-2013-08-06]
:
- FunctionCall asterix.day-of-week@1[
+ FunctionCall asterix.day-of-week@2[
FunctionCall asterix.date@1[
LiteralExpr [STRING] [-2013-08-06]
]
+ LiteralExpr [STRING] [Monday]
]
)
(
LiteralExpr [STRING] [1913-08-06T15:53:28Z]
:
- FunctionCall asterix.day-of-week@1[
+ FunctionCall asterix.day-of-week@2[
FunctionCall asterix.datetime@1[
LiteralExpr [STRING] [1913-08-06T15:53:28Z]
]
+ LiteralExpr [STRING] [MONDAY]
]
)
(
LiteralExpr [STRING] [-1913-08-10T15:53:28Z]
:
- FunctionCall asterix.day-of-week@1[
+ FunctionCall asterix.day-of-week@2[
FunctionCall asterix.datetime@1[
LiteralExpr [STRING] [-1913-08-10T15:53:28Z]
]
+ LiteralExpr [LONG] [2]
]
)
(
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/day_of_week_01/day_of_week_01.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/day_of_week_01/day_of_week_01.4.ast
new file mode 100644
index 0000000..cab93b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/day_of_week_01/day_of_week_01.4.ast
@@ -0,0 +1,173 @@
+Query:
+SELECT [
+Variable [ Name=$r ]
+r
+FunctionCall asterix.sql-min@1[
+ Variable [ Name=$dows ]
+]
+mn
+FunctionCall asterix.sql-max@1[
+ Variable [ Name=$dows ]
+]
+mx
+]
+FROM [ FunctionCall asterix.range@2[
+ LiteralExpr [LONG] [1]
+ LiteralExpr [LONG] [7]
+ ]
+ AS Variable [ Name=$r ]
+]
+Let Variable [ Name=$d ]
+ :=
+ FunctionCall asterix.date@1[
+ LiteralExpr [STRING] [2021-09-05]
+ ]
+Let Variable [ Name=$dt ]
+ :=
+ FunctionCall asterix.datetime-from-date-time@2[
+ Variable [ Name=$d ]
+ FunctionCall asterix.time@1[
+ LiteralExpr [STRING] [01:01:01]
+ ]
+ ]
+Let Variable [ Name=$days ]
+ :=
+ OrderedListConstructor [
+ LiteralExpr [STRING] [sunday]
+ LiteralExpr [STRING] [monday]
+ LiteralExpr [STRING] [tuesday]
+ LiteralExpr [STRING] [wednesday]
+ LiteralExpr [STRING] [thursday]
+ LiteralExpr [STRING] [friday]
+ LiteralExpr [STRING] [saturday]
+ ]
+Let Variable [ Name=$dows ]
+ :=
+ OrderedListConstructor [
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$d ]
+ FunctionCall asterix.int8@1[
+ Variable [ Name=$r ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$dt ]
+ FunctionCall asterix.int16@1[
+ Variable [ Name=$r ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$d ]
+ FunctionCall asterix.int32@1[
+ Variable [ Name=$r ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$dt ]
+ FunctionCall asterix.int64@1[
+ Variable [ Name=$r ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$d ]
+ FunctionCall asterix.float@1[
+ Variable [ Name=$r ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$dt ]
+ FunctionCall asterix.double@1[
+ Variable [ Name=$r ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$d ]
+ IndexAccessor [
+ Variable [ Name=$days ]
+ Index: OperatorExpr [
+ Variable [ Name=$r ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$dt ]
+ FunctionCall asterix.uppercase@1[
+ IndexAccessor [
+ Variable [ Name=$days ]
+ Index: OperatorExpr [
+ Variable [ Name=$r ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$d ]
+ FunctionCall asterix.lowercase@1[
+ IndexAccessor [
+ Variable [ Name=$days ]
+ Index: OperatorExpr [
+ Variable [ Name=$r ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$dt ]
+ FunctionCall asterix.substring@3[
+ IndexAccessor [
+ Variable [ Name=$days ]
+ Index: OperatorExpr [
+ Variable [ Name=$r ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ LiteralExpr [LONG] [0]
+ LiteralExpr [LONG] [3]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$d ]
+ FunctionCall asterix.uppercase@1[
+ FunctionCall asterix.substring@3[
+ IndexAccessor [
+ Variable [ Name=$days ]
+ Index: OperatorExpr [
+ Variable [ Name=$r ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ LiteralExpr [LONG] [0]
+ LiteralExpr [LONG] [3]
+ ]
+ ]
+ ]
+ FunctionCall asterix.day-of-week@2[
+ Variable [ Name=$dt ]
+ FunctionCall asterix.lowercase@1[
+ FunctionCall asterix.substring@3[
+ IndexAccessor [
+ Variable [ Name=$days ]
+ Index: OperatorExpr [
+ Variable [ Name=$r ]
+ -
+ LiteralExpr [LONG] [1]
+ ]
+ ]
+ LiteralExpr [LONG] [0]
+ LiteralExpr [LONG] [3]
+ ]
+ ]
+ ]
+ ]
+Orderby
+ Variable [ Name=$r ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_01/parse_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_01/parse_01.3.ast
index 08e0b89..39490af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_01/parse_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_01/parse_01.3.ast
@@ -47,7 +47,7 @@
:
FunctionCall asterix.parse-date@2[
LiteralExpr [STRING] [Mon Aug 19 2013]
- LiteralExpr [STRING] [W MMM D Y]
+ LiteralExpr [STRING] [EEE MMM D Y]
]
)
(
@@ -55,7 +55,7 @@
:
FunctionCall asterix.parse-date@2[
LiteralExpr [STRING] [SKIPMEPLEASE Mon Aug SKIPME1ALSO 19 2013]
- LiteralExpr [STRING] [O W MMM O D Y]
+ LiteralExpr [STRING] [O EEE MMM O D Y]
]
)
(
@@ -143,7 +143,7 @@
:
FunctionCall asterix.parse-datetime@2[
LiteralExpr [STRING] [1970-01-01 Thu 23:59:23.392-04:00]
- LiteralExpr [STRING] [MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD W hh:mm:ss.nnnz]
+ LiteralExpr [STRING] [MMM DD YYYY h:m:s a|MM/DD/YYY hh:mm:ss.nnn a z|YYYY-MM-DD EEE hh:mm:ss.nnnz]
]
)
(
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_02/parse_02.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_02/parse_02.3.ast
index ea8c109..6ac928a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_02/parse_02.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/parse_02/parse_02.3.ast
@@ -38,7 +38,7 @@
FunctionCall asterix.time@1[
LiteralExpr [STRING] [08:07:29.030Z]
]
- LiteralExpr [STRING] [h.m.s.nn a z]
+ LiteralExpr [STRING] [h.m.s.nn a]
]
)
(
@@ -48,7 +48,7 @@
FunctionCall asterix.time@1[
LiteralExpr [STRING] [08:07:29.030Z]
]
- LiteralExpr [STRING] [hh.mm.ss.nnn a z]
+ LiteralExpr [STRING] [hh.mm.ss.nnn a]
]
)
(
@@ -58,7 +58,7 @@
FunctionCall asterix.datetime@1[
LiteralExpr [STRING] [0137-12-31T23:59:59.999+08:00]
]
- LiteralExpr [STRING] [MMM DD h:m:s.nnn a YY z]
+ LiteralExpr [STRING] [MMM DD h:m:s.nnn a YY]
]
)
(
@@ -68,7 +68,7 @@
FunctionCall asterix.datetime@1[
LiteralExpr [STRING] [0137-12-31T23:59:59.999+08:00]
]
- LiteralExpr [STRING] [YYYY/MMM/DD h:m:s.nnnz a]
+ LiteralExpr [STRING] [YYYY/MMM/DD h:m:s.nnn a]
]
)
(
@@ -78,7 +78,7 @@
FunctionCall asterix.datetime@1[
LiteralExpr [STRING] [0137-12-31T23:59:59.999+08:00]
]
- LiteralExpr [STRING] [YYYY-MM-DDThh:mm:ss.nnnz]
+ LiteralExpr [STRING] [YYYY-MM-DDThh:mm:ss.nnn]
]
)
(
@@ -88,7 +88,7 @@
FunctionCall asterix.datetime@1[
LiteralExpr [STRING] [1000-10-10T10:10:10.100+00:00]
]
- LiteralExpr [STRING] [YYYY-MM-DDThh:mm:ss.nnnz]
+ LiteralExpr [STRING] [YYYY-MM-DDThh:mm:ss.nnn]
]
)
]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast
new file mode 100644
index 0000000..a6cbed7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast
@@ -0,0 +1,71 @@
+DataverseUse test
+Query:
+SELECT [
+FieldAccessor [
+ Variable [ Name=$y ]
+ Field=c2
+]
+c2
+(
+ SELECT [
+ WINDOW asterix.nth-value-impl@3[
+ (
+ SELECT ELEMENT [
+ FieldAccessor [
+ FieldAccessor [
+ Variable [ Name=#2 ]
+ Field=x
+ ]
+ Field=c2
+ ]
+ ]
+ FROM [ Variable [ Name=#1 ]
+ AS Variable [ Name=#2 ]
+ ]
+ )
+ LiteralExpr [LONG] [3]
+ FieldAccessor [
+ Variable [ Name=$x ]
+ Field=c2
+ ]
+ ]
+ AS Variable [ Name=#1 ]
+ (
+ x:=Variable [ Name=$x ]
+ )
+ OVER (
+ PARTITION BY
+ FieldAccessor [
+ Variable [ Name=$x ]
+ Field=one
+ ]
+ ORDER BY
+ FieldAccessor [
+ Variable [ Name=$x ]
+ Field=c2
+ ]
+ ASC
+ range between unbounded preceding and unbounded following exclude no others
+ )
+ nth
+ ]
+ FROM [ FunctionCall asterix.dataset@1[
+ LiteralExpr [STRING] [test.t1]
+ ]
+ AS Variable [ Name=$x ]
+ ]
+)
+q1
+]
+FROM [ FunctionCall asterix.dataset@1[
+ LiteralExpr [STRING] [test.t1]
+ ]
+ AS Variable [ Name=$y ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=$y ]
+ Field=c2
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
index fca1590..2e1a6bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
@@ -22,37 +22,37 @@
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
<compilation-unit name="valid-auth-methods">
<placeholder name="azureblob-credentialsname" value="accountKey" />
- <placeholder name="azureblob-credentialsvalue" value="%azureblob-accountkey%" />
+ <placeholder name="azureblob-credentialsvalue" value="%azure-accountkey%" />
<output-dir compare="Text">valid-auth-methods</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
<compilation-unit name="valid-auth-methods">
<placeholder name="azureblob-credentialsname" value="sharedAccessSignature" />
- <placeholder name="azureblob-credentialsvalue" value="%azureblob-sas%" />
+ <placeholder name="azureblob-credentialsvalue" value="%azure-sas%" />
<output-dir compare="Text">valid-auth-methods</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
<compilation-unit name="valid-auth-methods">
<placeholder name="azureblob-credentialsname" value="connectionString" />
- <placeholder name="azureblob-credentialsvalue" value="%azureblob-connectionstringaccountkey%" />
+ <placeholder name="azureblob-credentialsvalue" value="%azure-connectionstringaccountkey%" />
<output-dir compare="Text">valid-auth-methods</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
<compilation-unit name="valid-auth-methods">
<placeholder name="azureblob-credentialsname" value="connectionString" />
- <placeholder name="azureblob-credentialsvalue" value="%azureblob-connectionstringsas%" />
+ <placeholder name="azureblob-credentialsvalue" value="%azure-connectionstringsas%" />
<output-dir compare="Text">valid-auth-methods</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
<compilation-unit name="invalid-auth-methods">
<placeholder name="azureblob-credentialsname-1" value="accountKey" />
- <placeholder name="azureblob-credentialsvalue-1" value="%azureblob-accountkey%" />
+ <placeholder name="azureblob-credentialsvalue-1" value="%azure-accountkey%" />
<placeholder name="azureblob-credentialsname-2" value="connectionString" />
- <placeholder name="azureblob-credentialsvalue-2" value="%azureblob-connectionstringaccountkey%" />
+ <placeholder name="azureblob-credentialsvalue-2" value="%azure-connectionstringaccountkey%" />
<output-dir compare="Text">invalid-auth-methods</output-dir>
<expected-error>ASX1138: Only a single authentication method is allowed: connectionString, accountName & accountKey, or accountName & sharedAccessSignature</expected-error>
</compilation-unit>
@@ -60,17 +60,28 @@
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
<compilation-unit name="invalid-auth-methods">
<placeholder name="azureblob-credentialsname-1" value="sharedAccessSignature" />
- <placeholder name="azureblob-credentialsvalue-1" value="%azureblob-sas%" />
+ <placeholder name="azureblob-credentialsvalue-1" value="%azure-sas%" />
<placeholder name="azureblob-credentialsname-2" value="connectionString" />
- <placeholder name="azureblob-credentialsvalue-2" value="%azureblob-connectionstringaccountkey%" />
+ <placeholder name="azureblob-credentialsvalue-2" value="%azure-connectionstringaccountkey%" />
<output-dir compare="Text">invalid-auth-methods</output-dir>
<expected-error>ASX1138: Only a single authentication method is allowed: connectionString, accountName & accountKey, or accountName & sharedAccessSignature</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
- <compilation-unit name="invalid-no-auth">
- <output-dir compare="Text">invalid-no-auth</output-dir>
- <expected-error>ASX1139: No authentication parameters provided</expected-error>
+ <compilation-unit name="anonymous-no-auth-public-access-allowed">
+ <output-dir compare="Text">anonymous-no-auth-public-access-allowed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
+ <compilation-unit name="anonymous-no-auth-public-access-not-allowed">
+ <output-dir compare="Text">anonymous-no-auth-public-access-not-allowed</output-dir>
+ <expected-error>Server failed to authenticate the request. Make sure the value of the Authorization header is formed correctly including the signature.</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
+ <compilation-unit name="anonymous-no-auth-no-endpoint">
+ <output-dir compare="Text">anonymous-no-auth-no-endpoint</output-dir>
+ <expected-error>ASX1151: No authentication credentials provided, 'endpoint' field is required for anonymous access</expected-error>
</compilation-unit>
</test-case>
</test-group>
@@ -127,6 +138,14 @@
<output-dir compare="Text">common/tsv/mixed</output-dir>
</compilation-unit>
</test-case>
+ <!-- Parquet Tests Start -->
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/object-concat">
+ <placeholder name="adapter" value="AZUREBLOB" />
+ <output-dir compare="Text">common/parquet/object-concat</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- Parquet Tests End -->
<test-case FilePath="external-dataset">
<compilation-unit name="common/empty-string-definition">
<placeholder name="adapter" value="AZUREBLOB" />
@@ -159,7 +178,7 @@
<test-case FilePath="external-dataset">
<compilation-unit name="common/invalid-endpoint">
<placeholder name="adapter" value="AZUREBLOB" />
- <placeholder name="blobEndpoint" value="http://^invalid-endpoint^" />
+ <placeholder name="endpoint" value="http://^invalid-endpoint^" />
<output-dir compare="Text">common/invalid-endpoint</output-dir>
<expected-error>External source error. java.net.URISyntaxException: Illegal character in authority at index 7: http://^invalid-endpoint^</expected-error>
</compilation-unit>
@@ -168,7 +187,7 @@
<compilation-unit name="common/bucket-does-not-exist">
<placeholder name="adapter" value="AZUREBLOB" />
<output-dir compare="Text">common/bucket-does-not-exist</output-dir>
- <expected-error>External source error. Status code 404</expected-error>
+ <expected-error>External source error. com.azure.storage.blob.models.BlobStorageException: Status code 404</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset" check-warnings="true">
@@ -345,4 +364,24 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="bom">
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/byte_order_mark/json">
+ <placeholder name="adapter" value="AZUREBLOB" />
+ <output-dir compare="Text">common/byte_order_mark/json</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/byte_order_mark/csv">
+ <placeholder name="adapter" value="AZUREBLOB" />
+ <output-dir compare="Text">common/byte_order_mark/csv</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/byte_order_mark/tsv">
+ <placeholder name="adapter" value="AZUREBLOB" />
+ <output-dir compare="Text">common/byte_order_mark/tsv</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
index b354e65..bacc23b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
@@ -76,6 +76,103 @@
<output-dir compare="Text">common/tsv/mixed</output-dir>
</compilation-unit>
</test-case>
+ <!-- Parquet Tests Start -->
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/select-all-fields">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/select-all-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/select-count-one-field">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/select-count-one-field</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/multi-file-multi-schema">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/multi-file-multi-schema</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/field-access-pushdown">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/field-access-pushdown</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/array-access-pushdown">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/array-access-pushdown</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/missing-fields">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/missing-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/string-standard-utf8">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/string-standard-utf8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/object-concat">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/object-concat</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="common/parquet/type-mismatch">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/type-mismatch</output-dir>
+ <expected-warn>ASX0002: Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is array (in line 31, at column 29)</expected-warn>
+ <expected-warn>ASX0002: Type mismatch: function array-star expects its 1st input parameter to be of type array, but the actual input type is object (in line 31, at column 24)</expected-warn>
+ <expected-warn>ASX0002: Type mismatch: function field-access-by-name expects its 1st input parameter to be of type object, but the actual input type is array (in line 31, at column 52)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/heterogeneous-access-pushdown">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/heterogeneous-access-pushdown</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/pushdown-plans">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/pushdown-plans</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset/s3">
+ <compilation-unit name="parquet-anonymous-access">
+ <output-dir compare="Text">parquet-anonymous-access</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset/s3">
+ <compilation-unit name="parquet-temporary-access">
+ <output-dir compare="Text">parquet-temporary-access</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/parquet/invalid-type">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1161: Type 'ParquetType' contains declared fields, which is not supported for 'parquet' format</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="common/parquet/no-files">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/parquet/no-files</output-dir>
+ <source-location>false</source-location>
+ <expected-warn>The provided external dataset configuration returned no files from the external source</expected-warn>
+ <expected-warn>The provided external dataset configuration returned no files from the external source</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <!-- Parquet Tests End -->
<test-case FilePath="external-dataset">
<compilation-unit name="common/empty-string-definition">
<placeholder name="adapter" value="S3" />
@@ -117,7 +214,7 @@
<compilation-unit name="common/bucket-does-not-exist">
<placeholder name="adapter" value="S3" />
<output-dir compare="Text">common/bucket-does-not-exist</output-dir>
- <expected-error>External source error. The specified bucket does not exist (Service: S3, Status Code: 404, Request ID: null)</expected-error>
+ <expected-error>External source error. software.amazon.awssdk.services.s3.model.NoSuchBucketException: The specified bucket does not exist (Service: S3, Status Code: 404</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset" check-warnings="true">
@@ -146,6 +243,7 @@
<test-case FilePath="external-dataset/s3">
<compilation-unit name="non-s3-region">
<output-dir compare="Text">non-s3-region</output-dir>
+ <expected-error>Provided S3 region is not supported: 'some-new-region'</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/s3">
@@ -313,4 +411,24 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="bom">
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/byte_order_mark/json">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/byte_order_mark/json</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/byte_order_mark/csv">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/byte_order_mark/csv</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="common/byte_order_mark/tsv">
+ <placeholder name="adapter" value="S3" />
+ <output-dir compare="Text">common/byte_order_mark/tsv</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
index 35bec85..686ede2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
@@ -57,7 +57,7 @@
result[0].append(self.next_tuple(*arg, key=self.mid))
File "entrypoint.py", line 99, in next_tuple
return self.wrapped_fns[key](*args)
- File "site-packages/roundtrip.py", line 29, in warning
+ File "site-packages/roundtrip.py", line 32, in warning
raise ArithmeticError("oof")
ArithmeticError: oof
(in line 28, at column 1)</expected-warn>
@@ -80,5 +80,15 @@
<expected-warn>ASX0201: External UDF returned exception. Returned exception was: java.io.IOException: Python process exited with code: 1 (in line 23, at column 1)</expected-warn>
</compilation-unit>
</test-case>
+ <test-case FilePath="external-library">
+ <compilation-unit name="big_object_pyudf">
+ <output-dir compare="Text">big_object_pyudf</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-library">
+ <compilation-unit name="big_object_roundtrip_20M">
+ <output-dir compare="Text">big_object_roundtrip_20M</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
index 6c67216..7667bb4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
@@ -27,7 +27,7 @@
<compilation-unit name="bad-ext-function-ddl-1">
<output-dir compare="Text">none</output-dir>
<expected-error>ASX1079: Compilation error: Variable number of parameters is not supported for external functions</expected-error>
- <expected-error>ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 25, at column 1)</expected-error>
+ <expected-error>ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 25, at column 57)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="external-library">
@@ -108,6 +108,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="external-library">
+ <compilation-unit name="mixedudfs">
+ <output-dir compare="Text">mixedudfs</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-library">
<compilation-unit name="udf_metadata">
<output-dir compare="Text">udf_metadata</output-dir>
</compilation-unit>
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 5d1f9fd..0bfbcf2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -29,6 +29,11 @@
&TemporalQueries;
<test-group name="api">
<test-case FilePath="api">
+ <compilation-unit name="compileonly">
+ <output-dir compare="Text">compileonly</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
<compilation-unit name="multiple-param-values">
<output-dir compare="Text">multiple-param-values</output-dir>
</compilation-unit>
@@ -47,6 +52,13 @@
<expected-error>ASX0044: DELETE statement is not supported in read-only mode</expected-error>
</compilation-unit>
</test-case>
+ <test-case FilePath="api" check-warnings="true">
+ <compilation-unit name="request-dataverse">
+ <output-dir compare="Text">request-dataverse</output-dir>
+ <expected-warn>ASX1063: Cannot find dataverse with name testUnknown</expected-warn>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="api">
<compilation-unit name="request-param-validation">
<output-dir compare="Text">request-param-validation</output-dir>
@@ -133,6 +145,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="api">
+ <compilation-unit name="signature">
+ <output-dir compare="Text">signature</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
<compilation-unit name="ignore-body-for-get">
<output-dir compare="Text">ignore-body-for-get</output-dir>
</compilation-unit>
@@ -3431,6 +3448,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="comparison">
+ <compilation-unit name="is_distinct_01">
+ <output-dir compare="Text">is_distinct_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
<compilation-unit name="like">
<output-dir compare="Text">like</output-dir>
</compilation-unit>
@@ -3586,6 +3608,19 @@
<output-dir compare="Text">binary_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="binary_02">
+ <output-dir compare="Text">binary_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for binary in @#!1 (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: hex() cannot process input type date (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: hex() cannot process input type array (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: hex() cannot process input type object (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for binary in @#!2 (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: base64() cannot process input type date (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: base64() cannot process input type array (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: base64() cannot process input type object (in line 27, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="add-null">
<output-dir compare="Text">add-null</output-dir>
@@ -3596,6 +3631,16 @@
<output-dir compare="Text">boolean_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="boolean_02">
+ <output-dir compare="Text">boolean_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for boolean in FALSE (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for boolean in TRUE (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type array (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean() cannot process input type object (in line 28, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="circle_01">
<output-dir compare="Text">circle_01</output-dir>
@@ -3606,24 +3651,80 @@
<output-dir compare="Text">date_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="date_02">
+ <output-dir compare="Text">date_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for date in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for date in 12/31/2020 (in line 30, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="datetime_01">
<output-dir compare="Text">datetime_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="datetime_02">
+ <output-dir compare="Text">datetime_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for datetime in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for datetime in 1951-12-27T12:20:15Z (in line 30, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="double_01">
<output-dir compare="Text">double_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="double_02">
+ <output-dir compare="Text">double_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for double in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double() cannot process input type object (in line 29, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="duration_01">
<output-dir compare="Text">duration_01</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="constructor">
+ <test-case FilePath="constructor" check-warnings="true">
<compilation-unit name="duration_02">
<output-dir compare="Text">duration_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for duration in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for yearmonthduration in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for daytimeduration in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type bigint (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type datetime (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type time (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type array (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration() cannot process input type object (in line 30, at column 13)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="constructor">
@@ -3631,11 +3732,51 @@
<output-dir compare="Text">float_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="float_02">
+ <output-dir compare="Text">float_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for float in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float() cannot process input type object (in line 29, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="int_01">
<output-dir compare="Text">int_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="int_02">
+ <output-dir compare="Text">int_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for tinyint in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for smallint in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for integer in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for bigint in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type datetime (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type date (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type time (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64() cannot process input type object (in line 29, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="interval">
<output-dir compare="Text">interval</output-dir>
@@ -3686,26 +3827,40 @@
<output-dir compare="Text">string_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="string_02">
+ <output-dir compare="Text">string_02</output-dir>
+ <expected-warn>ASX0004: Unsupported type: string() cannot process input type array (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: string() cannot process input type object (in line 25, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor">
<compilation-unit name="time_01">
<output-dir compare="Text">time_01</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="constructor" check-warnings="true">
+ <compilation-unit name="time_02">
+ <output-dir compare="Text">time_02</output-dir>
+ <expected-warn>ASX0006: Invalid format for time in @#! (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type boolean (in line 25, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type bigint (in line 26, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type date (in line 27, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type array (in line 28, at column 13)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time() cannot process input type object (in line 29, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for time in 11:58:59 (in line 30, at column 13)</expected-warn>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="constructor/uuid">
<compilation-unit name="uuid_01">
<output-dir compare="Text">uuid_01</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="constructor/uuid">
+ <test-case FilePath="constructor/uuid" check-warnings="true">
<compilation-unit name="uuid_02">
<output-dir compare="Text">uuid_02</output-dir>
- <expected-error>Invalid format for uuid in 02a199ca-bf58-412e-bd9f-60a0c975a8a-</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor/uuid">
- <compilation-unit name="uuid_03">
- <output-dir compare="Text">uuid_03</output-dir>
- <expected-error>Invalid format for uuid in 12345</expected-error>
+ <expected-warn>ASX0006: Invalid format for uuid in 02a199ca-bf58-412e-bd9f-60a0c975a8a- (in line 24, at column 13)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for uuid in 12345 (in line 25, at column 13)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="constructor">
@@ -3988,6 +4143,13 @@
</compilation-unit>
</test-case>
<test-case FilePath="ddl/create-index">
+ <compilation-unit name="create-index-6">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1050: Cannot find dataset with name LineItemView1 in dataverse test (in line 55, at column 1)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name LineItemView2 in dataverse test (in line 60, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl/create-index">
<compilation-unit name="create-inverted-index-with-variable-length-primary-key">
<output-dir compare="Text">create-inverted-index-with-variable-length-primary-key</output-dir>
</compilation-unit>
@@ -3996,6 +4158,7 @@
<compilation-unit name="bad-type-ddl">
<output-dir compare="Text">none</output-dir>
<expected-error>ASX1079: Compilation error: Reserved type name $x</expected-error>
+ <expected-error>ASX0013: Duplicate field name "c" (in line 29, at column 19)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="ddl">
@@ -4009,6 +4172,7 @@
<expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust1</expected-error>
<expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust2</expected-error>
<expected-error>ASX1082: Cannot find datatype with name my_unknown_type</expected-error>
+ <expected-error>ASX0013: Duplicate field name "c_name" (in line 25, at column 22)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="ddl">
@@ -4109,6 +4273,16 @@
</compilation-unit>
</test-case>
<test-case FilePath="ddl">
+ <compilation-unit name="invalid-view-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "" (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a" (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " invalid" (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: "a/b" (in line 29, at column 13)</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: "c/d" (in line 32, at column 11)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
<compilation-unit name="dataset-and-index-same-dataverse">
<output-dir compare="Text">dataset-and-index-same-dataverse</output-dir>
</compilation-unit>
@@ -4136,6 +4310,28 @@
<output-dir compare="Text">index-bad-fields</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="index-cast-null">
+ <placeholder name="with" value="" />
+ <output-dir compare="Text">index-cast-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="index-cast-null">
+ <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
+ <output-dir compare="Text">index-cast-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="index-cast-null-negative">
+ <output-dir compare="Text">index-cast-null-negative</output-dir>
+ <expected-error>CAST modifier is only allowed for B-Tree indexes</expected-error>
+ <expected-error>CAST modifier cannot be specified together with ENFORCED</expected-error>
+ <expected-error>CAST modifier is used without specifying the type of the indexed field</expected-error>
+ <expected-error>Typed index on "[typed_f2]" field could be created only for open datatype</expected-error>
+ <expected-error>Parameter invalid_date cannot be set</expected-error>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="dml">
<test-case FilePath="dml">
@@ -4938,6 +5134,26 @@
<output-dir compare="Text">query-issue382</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="create-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="insert-into-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="upsert-into-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-into-index-unknown-key">
+ <output-dir compare="Text">index-unknown-key</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="employee">
<test-case FilePath="employee">
@@ -5627,6 +5843,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="group-by">
+ <compilation-unit name="gby-cross-join">
+ <output-dir compare="Text">gby-cross-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
<compilation-unit name="gby-nested-01">
<output-dir compare="Text">gby-nested-01</output-dir>
</compilation-unit>
@@ -5679,6 +5900,12 @@
</compilation-unit>
</test-case>
<test-case FilePath="group-by">
+ <compilation-unit name="core-07-error">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>GROUP BY x, y;<< Encountered "GROUP" at column 1]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
<compilation-unit name="sugar-01">
<output-dir compare="Text">core-01</output-dir>
</compilation-unit>
@@ -5963,6 +6190,11 @@
<output-dir compare="Text">rtree-spatial-self-intersect-point</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="index-join">
+ <compilation-unit name="btree-multiple-join">
+ <output-dir compare="Text">btree-multiple-join</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="index-selection">
<test-case FilePath="index-selection">
@@ -6305,6 +6537,18 @@
<output-dir compare="Text">word-jaccard-inline</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="cast-default-null">
+ <placeholder name="with" value="" />
+ <output-dir compare="Text">cast-default-null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="index-selection">
+ <compilation-unit name="cast-default-null">
+ <placeholder name="with" value="WITH {'merge-policy': {'name': 'correlated-prefix','parameters': { 'max-mergable-component-size': 16384, 'max-tolerance-component-count': 3 }}}" />
+ <output-dir compare="Text">cast-default-null</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="inverted-index-join-noeqjoin">
<test-case FilePath="inverted-index-join-noeqjoin">
@@ -6370,11 +6614,27 @@
</test-group>
<test-group name="join">
<test-case FilePath="join">
+ <compilation-unit name="cross-join-01">
+ <output-dir compare="Text">cross-join-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
+ <compilation-unit name="cross-join-02-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1077: Cannot find dataset x in dataverse Default nor an alias with name x (in line 26, at column 39)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
<compilation-unit name="hash_join_array">
<output-dir compare="Text">hash_join_array</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="join">
+ <compilation-unit name="hash_join_missing">
+ <output-dir compare="Text">hash_join_missing</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="join">
<compilation-unit name="hash_join_record">
<output-dir compare="Text">hash_join_record</output-dir>
</compilation-unit>
@@ -6487,6 +6747,22 @@
</compilation-unit>
</test-case>
<test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_04">
+ <output-dir compare="Text">ordered-list-constructor_04</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_05">
+ <output-dir compare="Text">ordered-list-constructor_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_06_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 26 >> from [r, r+1] x select value count(x),<< Encountered "," at column 42]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
<compilation-unit name="scan-collection_01">
<output-dir compare="Text">scan-collection_01</output-dir>
</compilation-unit>
@@ -7840,6 +8116,14 @@
<test-group name="array-index">
<test-group name="array-index/error-handling">
<test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-include-unknown-key">
+ <output-dir compare="Text">index-include-unknown-key</output-dir>
+ <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
+ <expected-error>ASX1079: Compilation error: Array indexes must specify EXCLUDE UNKNOWN KEY</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
<compilation-unit name="index-two-array-fields">
<output-dir compare="Text">index-two-array-fields</output-dir>
<expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
@@ -7865,14 +8149,7 @@
<test-case FilePath="array-index/error-handling">
<compilation-unit name="index-with-enforced-type">
<output-dir compare="Text">index-with-enforced-type</output-dir>
- <expected-error>ASX1154: Incompatible index type ARRAY</expected-error>
- <source-location>false</source-location>
- </compilation-unit>
- </test-case>
- <test-case FilePath="array-index/error-handling">
- <compilation-unit name="index-mixed-composite">
- <output-dir compare="Text">index-mixed-composite</output-dir>
- <expected-error>ASX1154: Incompatible index type ARRAY</expected-error>
+ <expected-error>ASX1155: Incompatible index type ARRAY</expected-error>
<source-location>false</source-location>
</compilation-unit>
</test-case>
@@ -7918,16 +8195,21 @@
<output-dir compare="Text">use-case-4</output-dir>
</compilation-unit>
</test-case>
- <!-- <test-case FilePath="array-index/metadata/closed">-->
- <!-- <compilation-unit name="with-composite-sk">-->
- <!-- <output-dir compare="Text">with-composite-sk</output-dir>-->
- <!-- </compilation-unit>-->
- <!-- </test-case>-->
- <!-- <test-case FilePath="array-index/metadata/open">-->
- <!-- <compilation-unit name="with-composite-sk">-->
- <!-- <output-dir compare="Text">with-composite-sk</output-dir>-->
- <!-- </compilation-unit>-->
- <!-- </test-case>-->
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="complex-structures">
+ <output-dir compare="Text">complex-structures</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="array-index/metadata/closed">
<compilation-unit name="with-composite-array-different-indicators">
<output-dir compare="Text">with-composite-array-different-indicators</output-dir>
@@ -8005,6 +8287,11 @@
<output-dir compare="Text">with-3-level-record-path</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="composite-atomic">
+ <output-dir compare="Text">composite-atomic</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="array-index/bulk-loading/after-index-creation">
<test-case FilePath="array-index/bulk-loading/after-index-creation">
@@ -8115,6 +8402,28 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="array-index/join-quantified-queries">
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-quantified-queries">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
<test-group name="array-index/join-unnest-queries">
<test-case FilePath="array-index/join-unnest-queries">
<compilation-unit name="use-case-1">
@@ -8141,6 +8450,11 @@
<output-dir compare="Text">with-open-index</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="loj-subquery">
+ <output-dir compare="Text">loj-subquery</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="array-index/select-unnest-queries">
<test-case FilePath="array-index/select-unnest-queries/closed">
@@ -8213,6 +8527,16 @@
<output-dir compare="Text">with-filter-fields</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="using-feed-new-index">
+ <output-dir compare="Text">using-feed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="using-feed-old-index">
+ <output-dir compare="Text">using-feed</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="array-index/select-quantified-queries">
<test-case FilePath="array-index/select-quantified-queries">
@@ -8246,6 +8570,13 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="array-index/composite-index-queries">
+ <test-case FilePath="array-index">
+ <compilation-unit name="composite-index-queries">
+ <output-dir compare="Text">composite-index-queries</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
</test-group>
<test-group name="nestrecords">
<test-case FilePath="nestrecords">
@@ -8317,11 +8648,28 @@
</compilation-unit>
</test-case>
<test-case FilePath="null-missing">
+ <compilation-unit name="order-by-2">
+ <output-dir compare="Text">order-by-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
+ <compilation-unit name="order-by-3-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 33 >>ORDER BY d.a NULLS;<< Encountered ";" at column 19]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: Unexpected token: NULLS (in line 33, at column 20)]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
<compilation-unit name="order-by-from-dataset">
<output-dir compare="Text">order-by-from-dataset</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="null-missing">
+ <compilation-unit name="order-by-from-dataset-2">
+ <output-dir compare="Text">order-by-from-dataset-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="null-missing">
<compilation-unit name="scan-collection">
<output-dir compare="Text">scan-collection</output-dir>
</compilation-unit>
@@ -8669,6 +9017,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="numeric">
+ <compilation-unit name="round-half-up">
+ <output-dir compare="Text">round-half-up</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
<compilation-unit name="round0">
<output-dir compare="Text">round0</output-dir>
</compilation-unit>
@@ -9203,6 +9556,11 @@
</test-group>
<test-group name="quantifiers">
<test-case FilePath="quantifiers">
+ <compilation-unit name="any_and_every_01">
+ <output-dir compare="Text">any_and_every_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="quantifiers">
<compilation-unit name="anysat_01">
<output-dir compare="Text">somesat_01</output-dir>
</compilation-unit>
@@ -9232,6 +9590,11 @@
<output-dir compare="Text">query-ASTERIXDB-2696</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="quantifiers">
+ <compilation-unit name="query-ASTERIXDB-2947">
+ <output-dir compare="Text">query-ASTERIXDB-2947</output-dir>
+ </compilation-unit>
+ </test-case>
<!--
<test-case FilePath="quantifiers">
<compilation-unit name="everysat_02">
@@ -9427,7 +9790,17 @@
</test-case>
<test-case FilePath="scan">
<compilation-unit name="alltypes_01">
- <output-dir compare="Text">alltypes_01</output-dir>
+ <output-dir compare="Clean-JSON">alltypes_01-cleanjson</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="alltypes_01">
+ <output-dir compare="Lossless-JSON">alltypes_01-losslessjson</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="scan">
+ <compilation-unit name="alltypes_01">
+ <output-dir compare="Lossless-ADM-JSON">alltypes_01-losslessadmjson</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="scan">
@@ -9678,6 +10051,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="spatial">
+ <compilation-unit name="hints_spatial_partitioning">
+ <output-dir compare="Text">hints_spatial_partitioning</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
<compilation-unit name="line_accessor">
<output-dir compare="Text">line_accessor</output-dir>
</compilation-unit>
@@ -9777,6 +10155,128 @@
<output-dir compare="Text">spatial-large-data</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_join_dynamic_partitioning">
+ <output-dir compare="Text">spatial_join_dynamic_partitioning</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_join_static_partitioning">
+ <output-dir compare="Text">spatial_join_static_partitioning</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_left_outer_join_st_intersects">
+ <output-dir compare="Text">spatial_left_outer_join_st_intersects</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="spatial">
+ <compilation-unit name="spatial_join_projection_check">
+ <output-dir compare="Text">spatial_join_projection_check</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="sql-compat">
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="in_non_list_01">
+ <output-dir compare="Text">in_non_list_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="outer_join_01">
+ <output-dir compare="Text">outer_join_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="outer_unnest_01">
+ <output-dir compare="Text">outer_unnest_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="select_star_01">
+ <output-dir compare="Text">select_star_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="select_star_02_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1168: Ambiguous projection in SELECT clause (in line 32, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_01_scalar">
+ <output-dir compare="Text">subquery_coercion_01_scalar</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_02_scalar_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 32, at column 29)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 32, at column 20)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 35, at column 4)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. (in line 34, at column 3)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_03_cmp">
+ <output-dir compare="Text">subquery_coercion_03_cmp</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_04_cmp_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 26)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 35, at column 21)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 37, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. (in line 36, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 35, at column 20)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 11)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 11)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_05_in">
+ <output-dir compare="Text">subquery_coercion_05_in</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_06_in_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 36, at column 27)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Subquery returns more than one field (in line 37, at column 17)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 38, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Both SELECT and SELECT VALUE are present (in line 39, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. (in line 37, at column 3)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 36, at column 21)</expected-error>
+ <expected-error>ASX1169: Unable to do subquery coercion. Unsupported projection kind (in line 37, at column 25)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_07_from">
+ <output-dir compare="Text">subquery_coercion_07_from</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="subquery_coercion_08_misc">
+ <output-dir compare="Text">subquery_coercion_08_misc</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="union_all_01">
+ <output-dir compare="Text">union_all_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="sql-compat">
+ <compilation-unit name="union_all_02_negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1171: Unable to process UNION clause. Both SELECT and SELECT VALUE are present (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unequal number of input fields (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 29, at column 8)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 30, at column 8)</expected-error>
+ <expected-error>ASX1171: Unable to process UNION clause. Unsupported projection kind (in line 31, at column 8)</expected-error>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="statement-params">
<test-case FilePath="statement-params">
@@ -10325,6 +10825,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="string">
+ <compilation-unit name="substr-ASTERIXDB-2949">
+ <output-dir compare="Text">substr-ASTERIXDB-2949</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
<compilation-unit name="regexp_position_with_flag/offset0/regex_position0_with_flag">
<output-dir compare="Text">regexp_position_with_flag/offset0/regex_position0_with_flag</output-dir>
</compilation-unit>
@@ -10465,6 +10970,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="string">
+ <compilation-unit name="string-literal1">
+ <output-dir compare="Text">string-literal1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
<compilation-unit name="string-to-codepoint">
<output-dir compare="Text">string-to-codepoint</output-dir>
</compilation-unit>
@@ -10864,6 +11374,11 @@
<output-dir compare="Text">query-ASTERIXDB-2845</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-3006">
+ <output-dir compare="Text">query-ASTERIXDB-3006</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="subset-collection">
<test-case FilePath="subset-collection">
@@ -12467,7 +12982,7 @@
<expected-error>ASX1001: Syntax error: Unexpected type declaration for parameter a in function myfn001</expected-error>
<expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn002</expected-error>
<expected-error>ASX1001: Syntax error: Unexpected return type declaration for function myfn003</expected-error>
- <expected-error>ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 29, at column 45)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="user-defined-functions">
@@ -12493,8 +13008,8 @@
<test-case FilePath="user-defined-functions">
<compilation-unit name="drop-dependency-2">
<output-dir compare="Text">drop-dependency-2</output-dir>
- <expected-error>ASX1147: Cannot drop dataverse: dataset C.TweetMessages being used by function B.f2(2)</expected-error>
- <expected-error>ASX1147: Cannot drop dataverse: dataset C.TweetMessages being used by function B.f3(...)</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f2(2)</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) C.TweetMessages being used by function B.f3(...)</expected-error>
<source-location>false</source-location>
</compilation-unit>
</test-case>
@@ -12777,14 +13292,14 @@
<test-case FilePath="user-defined-functions">
<compilation-unit name="udf37_recursion">
<output-dir compare="Text">none</output-dir>
- <expected-error>ASX1149: Illegal function recursion (in line 24, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function recursion (in line 25, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function recursion (in line 26, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function recursion (in line 27, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function recursion (in line 28, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function recursion (in line 29, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function recursion (in line 30, at column 1)</expected-error>
- <expected-error>ASX1149: Illegal function recursion (in line 31, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 24, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 25, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 26, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 27, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 28, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="user-defined-functions">
@@ -12830,6 +13345,162 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="view">
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-1">
+ <output-dir compare="Text">create-view-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-2-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 as select * from range(1,2) r order by;<< Encountered ";" at column 59]]></expected-error>
+ <expected-error>ASX1081: Cannot find function with signature test.undefined_range(2) (in line 25, at column 38)</expected-error>
+ <expected-error>ASX1160: A view with this name test.v1 already exists (in line 26, at column 1)</expected-error>
+ <expected-error>ASX1072: A dataset with name ds1 already exists in dataverse test (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1072: A dataset with name ds2 already exists in dataverse test (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1160: A view with this name test.ds1 already exists (in line 27, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: Unexpected IF NOT EXISTS (in line 25, at column 1)]]></expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 31, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 32, at column 1)</expected-error>
+ <expected-error>ASX1149: Illegal function or view recursion (in line 33, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1 primary key (r) not enforced as<< Encountered "primary" at column 21]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >>create view test.v2 foreign key (r) references v1 not enforced as<< Encountered <IDENTIFIER> "foreign" at column 21]]></expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-3-typed">
+ <output-dir compare="Text">create-view-3-typed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view" check-warnings="true">
+ <compilation-unit name="create-view-4-typed-warn">
+ <output-dir compare="Text">create-view-4-typed-warn</output-dir>
+ <expected-warn>ASX0006: Invalid format for tinyint in a (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for smallint in b (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for integer in c (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for bigint in d (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for float in e (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for double in f (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for datetime in g (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for date in h (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for time in j (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for duration in k (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for yearmonthduration in m (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for daytimeduration in n (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: boolean-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int8-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int16-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int32-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: int64-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: float-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: double-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: datetime-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: date-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: time-default-null() cannot process input type duration (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: year-month-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+ <expected-warn>ASX0004: Unsupported type: day-time-duration-default-null() cannot process input type date (in line 30, at column 6)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view" check-warnings="true">
+ <compilation-unit name="create-view-5-typed-warn">
+ <output-dir compare="Text">create-view-5-typed-warn</output-dir>
+ <expected-warn>ASX0006: Invalid format for datetime in a (in line 27, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for date in b (in line 27, at column 6)</expected-warn>
+ <expected-warn>ASX0006: Invalid format for time in c (in line 27, at column 6)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-6-typed-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1082: Cannot find datatype with name unknown_dv.t1</expected-error>
+ <expected-error>ASX1082: Cannot find datatype with name test.t1_unknown</expected-error>
+ <expected-error>ASX1079: Compilation error: view type cannot have open fields (in line 29, at column 1)</expected-error>
+ <expected-error>ASX1004: Unsupported type: view cannot process input type t1_a (in line 30, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint, a [bigint]) default null as<< Encountered "[" at column 33]]></expected-error>
+ <expected-error>ASX1092: Parameter date_illegal_property_name cannot be set (in line 25, at column 1)</expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 25 >>create view test.v1(r bigint) as<< Encountered "as" at column 31]]></expected-error>
+ <expected-error><![CDATA[ASX1014: Field "unknown_field" is not found (in line 25, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1014: Field "unknown_field_2" is not found (in line 25, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 28 >> as select r from range(1,2) r;<< Encountered "as" at column 3]]></expected-error>
+ <expected-error><![CDATA[ASX0013: Duplicate field name "r" (in line 25, at column 20)]]></expected-error>
+ <expected-error><![CDATA[ASX1167: Cannot change primary key of view test1.employee_v1 (in line 38, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 25, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1162: Invalid primary key definition (in line 26, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1001: Syntax error: In line 36 >> as employee;<< Encountered "as" at column 3]]></expected-error>
+ <expected-error><![CDATA[ASX1165: Invalid foreign key definition: view test1.employee_v1 does not have a primary key (in line 32, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 34, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v2 (in line 34, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 34, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1063: Cannot find dataverse with name test3 (in line 42, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee_v3 (in line 42, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1159: Cannot find view with name test1.employee (in line 43, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 43, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1164: Invalid foreign key definition (in line 39, at column 1)]]></expected-error>
+ <expected-error><![CDATA[ASX1166: Invalid foreign key definition: foreign key does not match primary key of view test1.employee_v1 (in line 39, at column 1)]]></expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="create-view-7-foreign-key">
+ <output-dir compare="Text">create-view-7-foreign-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-dataverse-1">
+ <output-dir compare="Text">drop-dataverse-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-dataverse-2-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.v2 being used by function test1.f1()</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.ds2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: function test2.f2() being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: synonym test2.s3 being used by view test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: type test2.t1 being used by dataset test1.v1</expected-error>
+ <expected-error>ASX1147: Cannot drop dataverse: dataset (or view) test2.employee_v2 being used by view test1.employee_v1</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-view-1">
+ <output-dir compare="Text">drop-view-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="drop-view-2-negative">
+ <output-dir compare="Text">drop-view-2-negative</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name test (in line 24, at column 1)</expected-error>
+ <expected-error>ASX1159: Cannot find view with name test.v1 (in line 25, at column 1)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test (in line 27, at column 1)</expected-error>
+ <expected-error>ASX1159: Cannot find view with name test.ds1 (in line 30, at column 1)</expected-error>
+ <expected-error>ASX1148: Cannot drop dataset test2.ds2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop function test2.f2() being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop synonym test2.s2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop view test2.v2 being used by view test1.v1</expected-error>
+ <expected-error>ASX1148: Cannot drop view test2.v2 being used by function test1.f1()</expected-error>
+ <expected-error>ASX1148: Cannot drop type test2.t1 being used by dataset test1.v1</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="view-1">
+ <output-dir compare="Text">view-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="view">
+ <compilation-unit name="view-2-negative">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1050: Cannot find dataset with name v1 in dataverse test1 (in line 24, at column 17)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name v2 in dataverse test1 (in line 24, at column 17)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name v3 in dataverse test1 (in line 24, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
<test-group name="load">
<test-case FilePath="load">
<compilation-unit name="load_non-empty_index">
@@ -13872,10 +14543,10 @@
<output-dir compare="Text">to_boolean_01</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="types">
+ <test-case FilePath="types" check-warnings="true">
<compilation-unit name="to_boolean_02">
<output-dir compare="Text">to_boolean_02</output-dir>
- <expected-error>ASX0002: Type mismatch</expected-error>
+ <expected-warn>ASX0004: Unsupported type: to-boolean() cannot process input type date (in line 24, at column 8)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="types">
@@ -13883,10 +14554,10 @@
<output-dir compare="Text">to_bigint_01</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="types">
+ <test-case FilePath="types" check-warnings="true">
<compilation-unit name="to_bigint_02">
<output-dir compare="Text">to_bigint_02</output-dir>
- <expected-error>ASX0002: Type mismatch</expected-error>
+ <expected-warn>ASX0004: Unsupported type: to-bigint() cannot process input type date (in line 24, at column 7)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="types">
@@ -13894,10 +14565,10 @@
<output-dir compare="Text">to_double_01</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="types">
+ <test-case FilePath="types" check-warnings="true">
<compilation-unit name="to_double_02">
<output-dir compare="Text">to_double_02</output-dir>
- <expected-error>ASX0002: Type mismatch</expected-error>
+ <expected-warn>ASX0004: Unsupported type: to-double() cannot process input type date (in line 24, at column 7)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="types">
@@ -13924,7 +14595,6 @@
<test-case FilePath="types">
<compilation-unit name="to_string_02">
<output-dir compare="Text">to_string_02</output-dir>
- <expected-error>ASX0004: Unsupported type</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="types">
@@ -13935,10 +14605,18 @@
<test-case FilePath="types">
<compilation-unit name="domain_boundaries_error">
<output-dir compare="Text">domain_boundaries_error</output-dir>
- <expected-error>ASX0006: Invalid format for int8 in tinyint</expected-error>
- <expected-error>ASX0006: Invalid format for int16 in smallint</expected-error>
- <expected-error>ASX0006: Invalid format for int32 in integer</expected-error>
- <expected-error>ASX0006: Invalid format for int64 in bigint</expected-error>
+ <expected-error>ASX0001: Field type null cannot be promoted to type tinyint</expected-error>
+ <expected-error>ASX0001: Field type null cannot be promoted to type smallint</expected-error>
+ <expected-error>ASX0001: Field type null cannot be promoted to type integer</expected-error>
+ <expected-error>ASX0001: Field type null cannot be promoted to type bigint</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="types">
+ <compilation-unit name="query-ASTERIXDB-2950">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 24, at column 21)</expected-error>
+ <expected-error>ASX1002: Type mismatch: function get-item expects its 1st input parameter to be of type multiset or array, but the actual input type is bigint (in line 25, at column 31)</expected-error>
</compilation-unit>
</test-case>
</test-group>
@@ -14492,6 +15170,14 @@
<source-location>false</source-location>
</compilation-unit>
</test-case>
+ <test-case FilePath="json-parser" check-warnings="true">
+ <compilation-unit name="parse-json-function">
+ <output-dir compare="Text">parse-json-function</output-dir>
+ <source-location>false</source-location>
+ <expected-warn>Type mismatch: function parse-json expects its 1st input parameter to be of type string, but the actual input type is bigint</expected-warn>
+ <expected-warn>Malformed input stream</expected-warn>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="composite-key">
<test-case FilePath="composite-key">
@@ -14658,7 +15344,7 @@
<expected-warn>ASX1107: Unexpected hint: indexnl. "hash" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: hash. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "skip-index", "use-index" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: hash. None expected at this location</expected-warn>
</compilation-unit>
</test-case>
@@ -14681,7 +15367,7 @@
<expected-warn>ASX1107: Unexpected hint: unknown_hint_groupby. "hash" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "skip-index", "use-index" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_elsewhere. None expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
index 5dfca79..4d50ca2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
@@ -124,6 +124,12 @@
<output-dir compare="Text">hdfs_03</output-dir>
</compilation-unit>
</test-case>
+ <!-- More Parquet tests can be found at testsuite_external_dataset_s3.xml -->
+ <test-case FilePath="hdfs">
+ <compilation-unit name="parquet">
+ <output-dir compare="Text">parquet</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="external-indexing">
<test-case FilePath="external-indexing">
@@ -137,11 +143,6 @@
</compilation-unit>
</test-case>
<test-case FilePath="external-indexing">
- <compilation-unit name="rc-format">
- <output-dir compare="Text">rc-format</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="external-indexing">
<compilation-unit name="rtree-index">
<output-dir compare="Text">rtree-index</output-dir>
</compilation-unit>
@@ -157,41 +158,4 @@
</compilation-unit>
</test-case>
</test-group>
- <test-group name="parquet">
- <test-case FilePath="parquet">
- <compilation-unit name="select-all-fields">
- <output-dir compare="Text">select-all-fields</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="parquet">
- <compilation-unit name="select-count-one-field">
- <output-dir compare="Text">select-count-one-field</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="parquet">
- <compilation-unit name="multi-file-multi-schema">
- <output-dir compare="Text">multi-file-multi-schema</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="parquet">
- <compilation-unit name="expression-pushdown">
- <output-dir compare="Text">expression-pushdown</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="parquet">
- <compilation-unit name="missing-fields">
- <output-dir compare="Text">missing-fields</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="parquet">
- <compilation-unit name="string-standard-utf8">
- <output-dir compare="Text">string-standard-utf8</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="parquet">
- <compilation-unit name="object-concat">
- <output-dir compare="Text">object-concat</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
index 7177797..a96c57e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
@@ -1123,11 +1123,6 @@
</compilation-unit>
</test-case>
<test-case FilePath="constructor">
- <compilation-unit name="duration_02">
- <output-dir compare="AST">duration_02</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
<compilation-unit name="float_01">
<output-dir compare="AST">float_01</output-dir>
</compilation-unit>
@@ -1138,12 +1133,6 @@
</compilation-unit>
</test-case>
<test-case FilePath="constructor">
- <compilation-unit name="int_02">
- <expected-error>Syntax error: Could not parse numeric literal</expected-error>
- <output-dir compare="AST">int_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
<compilation-unit name="interval">
<output-dir compare="AST">interval</output-dir>
</compilation-unit>
@@ -2815,6 +2804,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="list">
+ <compilation-unit name="ordered-list-constructor_05">
+ <output-dir compare="AST">ordered-list-constructor_05</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="list">
<compilation-unit name="scan-collection_01">
<output-dir compare="AST">scan-collection_01</output-dir>
</compilation-unit>
@@ -4421,11 +4415,6 @@
</compilation-unit>
</test-case>
<test-case FilePath="scan">
- <compilation-unit name="alltypes_01">
- <output-dir compare="AST">alltypes_01</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="scan">
<compilation-unit name="alltypes_02">
<output-dir compare="AST">alltypes_02</output-dir>
</compilation-unit>
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index ee4aa65..e76fb5b 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -161,7 +161,7 @@
</dependency>
<dependency>
<groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
+ <artifactId>netty-codec</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/IRecordTypeAnnotation.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/IRecordTypeAnnotation.java
index 5edef2f..ec0917e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/IRecordTypeAnnotation.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/IRecordTypeAnnotation.java
@@ -19,9 +19,10 @@
package org.apache.asterix.common.annotations;
public interface IRecordTypeAnnotation {
- public enum Kind {
- RECORD_DATA_GEN
+ enum Kind {
+ RECORD_DATA_GEN,
+ RECORD_FIELD_ORDER
}
- public Kind getKind();
+ Kind getKind();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/MissingNullInOutFunction.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/MissingNullInOutFunction.java
index 04be957..c2e4ce3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/MissingNullInOutFunction.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/MissingNullInOutFunction.java
@@ -25,10 +25,20 @@
import java.lang.annotation.Target;
/**
- * The functions to which this annotation is applied, respect the missing/null in -> missing/null out behaviour
+ * The functions to which this annotation is applied, specify what {@link MissingNullType} to return on MISSING or NULL
+ * input, and their runtimes respect the specification.
*/
@Documented
@Target(ElementType.TYPE)
@Retention(RetentionPolicy.RUNTIME)
public @interface MissingNullInOutFunction {
+
+ enum MissingNullType {
+ MISSING,
+ NULL
+ }
+
+ MissingNullType onMissing() default MissingNullType.MISSING;
+
+ MissingNullType onNull() default MissingNullType.NULL;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/RecordFieldOrderAnnotation.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/RecordFieldOrderAnnotation.java
new file mode 100644
index 0000000..04cf589
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/RecordFieldOrderAnnotation.java
@@ -0,0 +1,62 @@
+/*
+ * 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.common.annotations;
+
+import java.io.Serializable;
+import java.util.LinkedHashSet;
+import java.util.Objects;
+
+/**
+ * Contains an ordered set of fields of a record
+ */
+public final class RecordFieldOrderAnnotation implements IRecordTypeAnnotation, Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final LinkedHashSet<String> fieldNames;
+
+ public RecordFieldOrderAnnotation(LinkedHashSet<String> fieldNames) {
+ this.fieldNames = Objects.requireNonNull(fieldNames);
+ }
+
+ public LinkedHashSet<String> getFieldNames() {
+ return fieldNames;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.RECORD_FIELD_ORDER;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+ RecordFieldOrderAnnotation that = (RecordFieldOrderAnnotation) o;
+ return fieldNames.equals(that.fieldNames);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(fieldNames);
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SpatialJoinAnnotation.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SpatialJoinAnnotation.java
new file mode 100644
index 0000000..42262d6
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SpatialJoinAnnotation.java
@@ -0,0 +1,72 @@
+/*
+ * 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.common.annotations;
+
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+
+public final class SpatialJoinAnnotation implements IExpressionAnnotation {
+
+ public static final String HINT_STRING = "spatial-partitioning";
+
+ private final double minX;
+ private final double minY;
+ private final double maxX;
+ private final double maxY;
+ private final int numRows;
+ private final int numColumns;
+
+ public SpatialJoinAnnotation(double minX, double minY, double maxX, double maxY, int numRows, int numColumns) {
+ this.minX = minX;
+ this.minY = minY;
+ this.maxX = maxX;
+ this.maxY = maxY;
+ this.numRows = numRows;
+ this.numColumns = numColumns;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("%s:%f,%f,%f,%f,%d,%d", HINT_STRING, getMinX(), getMinY(), getMaxX(), getMaxY(),
+ getNumRows(), getNumColumns());
+ }
+
+ public double getMinX() {
+ return minX;
+ }
+
+ public double getMinY() {
+ return minY;
+ }
+
+ public double getMaxX() {
+ return maxX;
+ }
+
+ public double getMaxY() {
+ return maxY;
+ }
+
+ public int getNumRows() {
+ return numRows;
+ }
+
+ public int getNumColumns() {
+ return numColumns;
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
index b03af55..44b83d4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
@@ -169,4 +169,17 @@
* @return the current datasets io stats
*/
StorageIOStats getDatasetsIOStats();
+
+ /**
+ * Closes {@code resourcePath} if open
+ * @param resourcePath
+ * @throws HyracksDataException
+ */
+ void closeIfOpen(String resourcePath) throws HyracksDataException;
+
+ /**
+ * Removes all memory references of {@code partition}
+ * @param partitionId
+ */
+ void closePartition(int partitionId);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index 8c82979..5475b97 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -26,6 +26,7 @@
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.common.replication.IReplicationChannel;
import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.replication.IReplicationStrategyFactory;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.storage.IReplicaManager;
import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
@@ -71,7 +72,8 @@
IResourceIdFactory getResourceIdFactory();
void initialize(IRecoveryManagerFactory recoveryManagerFactory, IReceptionistFactory receptionistFactory,
- IConfigValidatorFactory configValidatorFactory, boolean initialRun) throws IOException, AlgebricksException;
+ IConfigValidatorFactory configValidatorFactory, IReplicationStrategyFactory replicationStrategyFactory,
+ boolean initialRun) throws IOException, AlgebricksException;
void setShuttingdown(boolean b);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/ClusterPartition.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/ClusterPartition.java
index cc99421..504155e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/ClusterPartition.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/ClusterPartition.java
@@ -71,7 +71,10 @@
@Override
public ClusterPartition clone() {
- return new ClusterPartition(partitionId, nodeId, ioDeviceNum);
+ ClusterPartition clone = new ClusterPartition(partitionId, nodeId, ioDeviceNum);
+ clone.setPendingActivation(pendingActivation);
+ clone.setActiveNodeId(activeNodeId);
+ return clone;
}
@Override
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
index a37e6e4..76802d9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
@@ -62,9 +62,11 @@
* @param nodeId
* @param active
* @param ncLocalCounters
+ * @param activePartitions
* @throws HyracksDataException
*/
- void updateNodeState(String nodeId, boolean active, NcLocalCounters ncLocalCounters) throws HyracksDataException;
+ void updateNodeState(String nodeId, boolean active, NcLocalCounters ncLocalCounters, Set<Integer> activePartitions)
+ throws HyracksDataException;
/**
* Updates the active node and active state of the cluster partition with id {@code partitionNum}
@@ -263,4 +265,19 @@
* @param rebalanceRequired
*/
void setRebalanceRequired(boolean rebalanceRequired) throws HyracksDataException;
+
+ /**
+ * Gets a map of the cluster partitions
+ *
+ * @return the cluster partitions map
+ */
+ Map<Integer, ClusterPartition> getClusterPartitions();
+
+ /**
+ * Returns true if any of the nodes in {@code nodesId} is currently inactive
+ *
+ * @param nodeIds
+ * @return true if any of the nodes is currently inactive, otherwise false
+ */
+ boolean nodesFailed(Set<String> nodeIds);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
index f673d24..b85c0be 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
@@ -30,7 +30,8 @@
*/
public enum DatasetType {
INTERNAL,
- EXTERNAL
+ EXTERNAL,
+ VIEW
}
public enum IndexType {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
index 7d5ec42..31708d3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
@@ -24,6 +24,7 @@
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.SortedMap;
import org.apache.asterix.common.cluster.ClusterPartition;
@@ -123,6 +124,10 @@
return accessor.getClusterPartitions();
}
+ public Set<Integer> getNodeActivePartitions(String nodeId) {
+ return accessor.getActivePartitions(nodeId);
+ }
+
public Map<String, String> getTransactionLogDirs() {
return accessor.getTransactionLogDirs();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java
index aaf6316..afb103d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/NodeProperties.java
@@ -46,7 +46,8 @@
STARTING_PARTITION_ID(
OptionTypes.INTEGER,
-1,
- "The first partition id to assign to iodevices on this node (-1 == auto-assign)");
+ "The first partition id to assign to iodevices on this node (-1 == auto-assign)"),
+ ACTIVE_PARTITIONS(OptionTypes.STRING_ARRAY, null, "List of node active partitions");
private final IOptionType type;
private final Object defaultValue;
@@ -95,7 +96,7 @@
@Override
public boolean hidden() {
- return this == INITIAL_RUN || this == STARTING_PARTITION_ID;
+ return this == INITIAL_RUN;
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java
index 3b6100c..5ba378d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/PropertiesAccessor.java
@@ -25,6 +25,7 @@
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -35,6 +36,7 @@
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Predicate;
+import java.util.stream.Collectors;
import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -192,6 +194,17 @@
return clusterPartitions;
}
+ public Set<Integer> getActivePartitions(String nodeId) {
+ // by default, node actives partitions are the partitions assigned to the node
+ String[] activePartitions = cfg.getStringArray(NodeProperties.Option.ACTIVE_PARTITIONS);
+ if (activePartitions == null) {
+ ClusterPartition[] nodeClusterPartitions = nodePartitionsMap.get(nodeId);
+ return Arrays.stream(nodeClusterPartitions).map(ClusterPartition::getPartitionId)
+ .collect(Collectors.toSet());
+ }
+ return Arrays.stream(activePartitions).map(Integer::parseInt).collect(Collectors.toSet());
+ }
+
public List<AsterixExtension> getExtensions() {
return extensions;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
index 72dacc4..7483a17 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
@@ -45,6 +45,7 @@
private int numActiveIOOps;
private int pendingFlushes;
private int pendingMerges;
+ private int pendingReplications;
private long lastAccess;
private boolean isExternal;
private boolean isRegistered;
@@ -82,6 +83,9 @@
case MERGE:
pendingMerges++;
break;
+ case REPLICATE:
+ pendingReplications++;
+ break;
default:
break;
}
@@ -96,6 +100,9 @@
case MERGE:
pendingMerges--;
break;
+ case REPLICATE:
+ pendingReplications--;
+ break;
default:
break;
}
@@ -251,4 +258,8 @@
public synchronized int getPendingMerges() {
return pendingMerges;
}
+
+ public synchronized int getPendingReplications() {
+ return pendingReplications;
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index b2f4034..c431dca 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -170,9 +170,15 @@
closeIndex(iInfo);
dsInfo.removeIndex(resourceID);
synchronized (dsInfo) {
- if (dsInfo.getReferenceCount() == 0 && dsInfo.isOpen() && dsInfo.getIndexes().isEmpty()
- && !dsInfo.isExternal()) {
+ int referenceCount = dsInfo.getReferenceCount();
+ boolean open = dsInfo.isOpen();
+ boolean empty = dsInfo.getIndexes().isEmpty();
+ if (referenceCount == 0 && open && empty && !dsInfo.isExternal()) {
+ LOGGER.debug("removing dataset {} from cache", dsInfo.getDatasetID());
removeDatasetFromCache(dsInfo.getDatasetID());
+ } else {
+ LOGGER.debug("keeping dataset {} in cache, ref count {}, open {}, indexes count: {}",
+ dsInfo.getDatasetID(), referenceCount, open, dsInfo.getIndexes().size());
}
}
}
@@ -414,9 +420,10 @@
logManager.log(waitLog);
for (PrimaryIndexOperationTracker primaryOpTracker : dsr.getOpTrackers()) {
// flush each partition one by one
- if (primaryOpTracker.getNumActiveOperations() > 0) {
- throw new IllegalStateException(
- "flushDatasetOpenIndexes is called on a dataset with currently active operations");
+ int numActiveOperations = primaryOpTracker.getNumActiveOperations();
+ if (numActiveOperations > 0) {
+ throw new IllegalStateException("flushDatasetOpenIndexes is called on dataset " + dsInfo.getDatasetID()
+ + " with currently " + "active operations, count=" + numActiveOperations);
}
primaryOpTracker.setFlushOnExit(true);
primaryOpTracker.flushIfNeeded();
@@ -555,10 +562,57 @@
for (DatasetResource dsr : datasets.values()) {
stats.addPendingFlushes(dsr.getDatasetInfo().getPendingFlushes());
stats.addPendingMerges(dsr.getDatasetInfo().getPendingMerges());
+ stats.addPendingReplications(dsr.getDatasetInfo().getPendingReplications());
}
return stats;
}
+ //TODO refactor this method with unregister method
+ @Override
+ public synchronized void closeIfOpen(String resourcePath) throws HyracksDataException {
+ validateDatasetLifecycleManagerState();
+ int did = getDIDfromResourcePath(resourcePath);
+ long resourceID = getResourceIDfromResourcePath(resourcePath);
+
+ DatasetResource dsr = datasets.get(did);
+ IndexInfo iInfo = dsr == null ? null : dsr.getIndexInfo(resourceID);
+
+ if (dsr == null || iInfo == null) {
+ return;
+ }
+
+ PrimaryIndexOperationTracker opTracker = dsr.getOpTracker(iInfo.getPartition());
+ if (iInfo.getReferenceCount() != 0 || (opTracker != null && opTracker.getNumActiveOperations() != 0)) {
+ if (LOGGER.isErrorEnabled()) {
+ final String logMsg = String.format(
+ "Failed to drop in-use index %s. Ref count (%d), Operation tracker active ops (%d)",
+ resourcePath, iInfo.getReferenceCount(), opTracker.getNumActiveOperations());
+ LOGGER.error(logMsg);
+ }
+ throw HyracksDataException.create(ErrorCode.CANNOT_DROP_IN_USE_INDEX,
+ StoragePathUtil.getIndexNameFromPath(resourcePath));
+ }
+
+ // TODO: use fine-grained counters, one for each index instead of a single counter per dataset.
+ DatasetInfo dsInfo = dsr.getDatasetInfo();
+ dsInfo.waitForIO();
+ closeIndex(iInfo);
+ dsInfo.removeIndex(resourceID);
+ synchronized (dsInfo) {
+ if (dsInfo.getReferenceCount() == 0 && dsInfo.isOpen() && dsInfo.getIndexes().isEmpty()
+ && !dsInfo.isExternal()) {
+ removeDatasetFromCache(dsInfo.getDatasetID());
+ }
+ }
+ }
+
+ @Override
+ public synchronized void closePartition(int partitionId) {
+ for (DatasetResource ds : datasets.values()) {
+ ds.removePartition(partitionId);
+ }
+ }
+
private void closeIndex(IndexInfo indexInfo) throws HyracksDataException {
if (indexInfo.isOpen()) {
ILSMOperationTracker opTracker = indexInfo.getIndex().getOperationTracker();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetResource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetResource.java
index 54e1976..db9eabb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetResource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetResource.java
@@ -182,4 +182,10 @@
public boolean isMetadataDataset() {
return MetadataIndexImmutableProperties.isMetadataDataset(getDatasetID());
}
+
+ public void removePartition(int partitionId) {
+ datasetPrimaryOpTrackers.remove(partitionId);
+ datasetComponentIdGenerators.remove(partitionId);
+ datasetRateLimiters.remove(partitionId);
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 806a6d4..fb001a0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -135,7 +135,7 @@
for (ILSMIndex lsmIndex : indexes) {
if (lsmIndex.isPrimaryIndex()) {
if (lsmIndex.isCurrentMutableComponentEmpty()) {
- LOGGER.info("Primary index on dataset {} and partition {} is empty... skipping flush",
+ LOGGER.debug("Primary index on dataset {} and partition {} is empty... skipping flush",
dsInfo.getDatasetID(), partition);
return;
}
@@ -145,8 +145,8 @@
}
}
if (primaryLsmIndex == null) {
- throw new IllegalStateException(
- "Primary index not found in dataset " + dsInfo.getDatasetID() + " and partition " + partition);
+ throw new IllegalStateException("Primary index not found in dataset " + dsInfo.getDatasetID()
+ + " and partition " + partition + " open indexes " + indexes);
}
for (ILSMIndex lsmIndex : indexes) {
ILSMOperationTracker opTracker = lsmIndex.getOperationTracker();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
index 7f448e3..249505b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/DatasetLocalResource.java
@@ -41,7 +41,7 @@
/**
* The resource partition
*/
- private final int partition;
+ private int partition;
private final IResource resource;
public DatasetLocalResource(int datasetId, int partition, IResource resource) {
@@ -68,6 +68,10 @@
resource.setPath(path);
}
+ public void setPartition(int partition) {
+ this.partition = partition;
+ }
+
@Override
public IIndex createInstance(INCServiceContext ncServiceCtx) throws HyracksDataException {
return resource.createInstance(ncServiceCtx);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
index ad90814..c22ac4c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.common.dataflow;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.common.api.ICoordinationService;
import org.apache.asterix.common.api.IMetadataLockManager;
@@ -149,4 +151,11 @@
* @return the adapter factory service
*/
IAdapterFactoryService getAdapterFactoryService();
+
+ /**
+ * Gets the cluster query compilation lock
+ *
+ * @return the cluster query compilation lock
+ */
+ ReentrantReadWriteLock getCompilationLock();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index c87cb7f..8939ce0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -85,6 +85,7 @@
EXTERNAL_UDF_RESULT_TYPE_ERROR(200),
EXTERNAL_UDF_EXCEPTION(201),
+ EXTERNAL_UDF_PROTO_RETURN_EXCEPTION(202),
// Compilation errors
PARSE_ERROR(1001),
@@ -105,10 +106,10 @@
COMPILATION_INDEX_TYPE_NOT_SUPPORTED_FOR_DATASET_TYPE(1016),
COMPILATION_FILTER_CANNOT_BE_NULLABLE(1017),
COMPILATION_ILLEGAL_FILTER_TYPE(1018),
- COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY(1019),
+ COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_KEY(1019),
COMPILATION_ILLEGAL_AUTOGENERATED_TYPE(1020),
- COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE(1021),
- COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE(1022),
+ COMPILATION_KEY_CANNOT_BE_NULLABLE(1021),
+ COMPILATION_ILLEGAL_KEY_TYPE(1022),
COMPILATION_CANT_DROP_ACTIVE_DATASET(1023),
COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX(1026),
COMPILATION_DATASET_TYPE_DOES_NOT_HAVE_PRIMARY_INDEX(1027),
@@ -233,14 +234,29 @@
UNKNOWN_FEED_POLICY(1146),
CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS(1147),
CANNOT_DROP_OBJECT_DEPENDENT_EXISTS(1148),
- ILLEGAL_FUNCTION_RECURSION(1149),
+ ILLEGAL_FUNCTION_OR_VIEW_RECURSION(1149),
ILLEGAL_FUNCTION_USE(1150),
- FULL_TEXT_CONFIG_NOT_FOUND(1151),
- FULL_TEXT_FILTER_NOT_FOUND(1152),
- FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED(1153),
- COMPILATION_INCOMPATIBLE_INDEX_TYPE(1154),
- FULL_TEXT_CONFIG_ALREADY_EXISTS(1155),
- FULL_TEXT_FILTER_ALREADY_EXISTS(1156),
+ NO_AUTH_PROVIDED_ENDPOINT_REQUIRED_FOR_ANONYMOUS_ACCESS(1151),
+ FULL_TEXT_CONFIG_NOT_FOUND(1152),
+ FULL_TEXT_FILTER_NOT_FOUND(1153),
+ FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED(1154),
+ COMPILATION_INCOMPATIBLE_INDEX_TYPE(1155),
+ FULL_TEXT_CONFIG_ALREADY_EXISTS(1156),
+ FULL_TEXT_FILTER_ALREADY_EXISTS(1157),
+ COMPILATION_BAD_VIEW_DEFINITION(1158),
+ UNKNOWN_VIEW(1159),
+ VIEW_EXISTS(1160),
+ UNSUPPORTED_TYPE_FOR_PARQUET(1161),
+ INVALID_PRIMARY_KEY_DEFINITION(1162),
+ UNSUPPORTED_AUTH_METHOD(1163),
+ INVALID_FOREIGN_KEY_DEFINITION(1164),
+ INVALID_FOREIGN_KEY_DEFINITION_REF_PK_NOT_FOUND(1165),
+ INVALID_FOREIGN_KEY_DEFINITION_REF_PK_MISMATCH(1166),
+ CANNOT_CHANGE_PRIMARY_KEY(1167),
+ AMBIGUOUS_PROJECTION(1168),
+ COMPILATION_SUBQUERY_COERCION_ERROR(1169),
+ S3_REGION_NOT_SUPPORTED(1170),
+ COMPILATION_SET_OPERATION_ERROR(1171),
// Feed errors
DATAFLOW_ILLEGAL_STATE(3001),
@@ -356,6 +372,8 @@
ACTIVE_ENTITY_NOT_RUNNING(3118),
REQUIRED_PARAM_IF_PARAM_IS_PRESENT(3119),
PARSER_DATA_PARSER_UNEXPECTED_TOKEN(3120),
+ REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT(3121),
+ PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT(3122),
// Lifecycle management errors
DUPLICATE_PARTITION_ID(4000),
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
index ce6d253..1189b51 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
@@ -294,6 +294,12 @@
// no op
}
+ @Override
+ public long getLastValidSequence() throws HyracksDataException {
+ ResourceReference resourceReference = ResourceReference.ofIndex(lsmIndex.getIndexIdentifier());
+ return indexCheckpointManagerProvider.get(resourceReference).getValidComponentSequence();
+ }
+
private boolean isMerge(ILSMIOOperation operation) {
return operation.getIOOpertionType() == LSMIOOperationType.MERGE
&& operation.getAccessor().getOpContext().getOperation() != IndexOperation.DELETE_COMPONENTS;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
index 761b2c6..f311655 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
@@ -51,4 +51,18 @@
* @param failure
*/
void notifyFailure(Exception failure);
+
+ /**
+ * Gets the current sync progress
+ *
+ * @return the current sync progress
+ */
+ double getSyncProgress();
+
+ /**
+ * Gets the last progress time of this replica based on System.nanoTime
+ *
+ * @return the last progress time
+ */
+ long getLastProgressTime();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategyFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategyFactory.java
new file mode 100644
index 0000000..384cd65
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategyFactory.java
@@ -0,0 +1,30 @@
+/*
+ * 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.common.replication;
+
+public interface IReplicationStrategyFactory {
+
+ /**
+ * Creates a replication strategy based on the provided {@code name}
+ *
+ * @param name
+ * @return the replication strategy
+ */
+ IReplicationStrategy create(String name);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
index 9c129d8..935b08f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
@@ -21,7 +21,7 @@
import java.util.HashMap;
import java.util.Map;
-public class ReplicationStrategyFactory {
+public class ReplicationStrategyFactory implements IReplicationStrategyFactory {
private static final Map<String, Class<? extends IReplicationStrategy>> BUILT_IN_REPLICATION_STRATEGY =
new HashMap<>();
@@ -32,11 +32,8 @@
BUILT_IN_REPLICATION_STRATEGY.put("metadata", MetadataOnlyReplicationStrategy.class);
}
- private ReplicationStrategyFactory() {
- throw new AssertionError();
- }
-
- public static IReplicationStrategy create(String name) {
+ @Override
+ public IReplicationStrategy create(String name) {
String strategyName = name.toLowerCase();
if (!BUILT_IN_REPLICATION_STRATEGY.containsKey(strategyName)) {
throw new IllegalStateException("Couldn't find strategy with name: " + name);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
index 801ca0b..beb8e07 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
@@ -28,9 +28,23 @@
* @param validComponentSequence
* @param lsn
* @param validComponentId
+ * @param masterNodeId
* @throws HyracksDataException
*/
- void init(long validComponentSequence, long lsn, long validComponentId) throws HyracksDataException;
+ void init(long validComponentSequence, long lsn, long validComponentId, String masterNodeId)
+ throws HyracksDataException;
+
+ /**
+ * Called when a new LSM disk component is flushed due to a replicated component.
+ * When called, the index checkpoint is updated with the latest valid {@code componentSequence}
+ * and low watermark {@code lsn}
+ *
+ * @param componentSequence
+ * @param lsn
+ * @param masterNodeId
+ * @throws HyracksDataException
+ */
+ void flushed(long componentSequence, long lsn, long componentId, String masterNodeId) throws HyracksDataException;
/**
* Called when a new LSM disk component is flushed. When called, the index checkpoint is updated
@@ -50,9 +64,11 @@
* @param componentSequence
* @param masterLsn
* @param componentId
+ * @param masterNodeId
* @throws HyracksDataException
*/
- void replicated(long componentSequence, long masterLsn, long componentId) throws HyracksDataException;
+ void replicated(long componentSequence, long masterLsn, long componentId, String masterNodeId)
+ throws HyracksDataException;
/**
* Called when a flush log is received and replicated from master. The mapping between
@@ -109,12 +125,13 @@
IndexCheckpoint getLatest() throws HyracksDataException;
/**
- * Advance the last valid component sequence. Used for replicated bulkloaded components
+ * Advance the last valid component. Used for replicated bulkloaded components
*
* @param componentSequence
+ * @param componentId
* @throws HyracksDataException
*/
- void advanceValidComponentSequence(long componentSequence) throws HyracksDataException;
+ void advanceValidComponent(long componentSequence, long componentId) throws HyracksDataException;
/**
* Set the last component id. Used during recovery or after component delete
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IReplicaManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IReplicaManager.java
index 1b8ec53..a4d56ce 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IReplicaManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IReplicaManager.java
@@ -57,6 +57,13 @@
Set<Integer> getPartitions();
/**
+ * Sets the node active partitions
+ *
+ * @param activePartitions
+ */
+ void setActivePartitions(Set<Integer> activePartitions);
+
+ /**
* Promotes a partition by making this node its master replica
*
* @param partition
@@ -73,10 +80,34 @@
void release(int partition) throws HyracksDataException;
/**
- * A lock that can be used to ensure a single replica is being synchronized at a time
+ * A lock that can be used to ensure a single partition replica is being synchronized at a time
* by this {@link IReplicaManager}
*
+ * @param partition partition
+ *
* @return the synchronization lock
*/
- Object getReplicaSyncLock();
+ Object getPartitionSyncLock(int partition);
+
+ /**
+ * Gets the partition replicas matching {@code id}
+ * @param id
+ * @return The partition replica if found. Otherwise, null.
+ */
+ IPartitionReplica getReplica(ReplicaIdentifier id);
+
+ /**
+ * Gets the list of replicas
+ *
+ * @return the list of replicas
+ */
+ List<IPartitionReplica> getReplicas();
+
+ /**
+ * Returns true if {@code partition} is owned by this node, otherwise false.
+ *
+ * @param partition
+ * @return true if the partition is owned by this node, otherwise false.
+ */
+ boolean isPartitionOwner(int partition);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
index 878c94e..a2cf531 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
@@ -38,14 +38,17 @@
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final long INITIAL_CHECKPOINT_ID = 0;
// TODO(mblow): remove this marker & related logic once we no longer are able to read indexes prior to the fix
- private static final long HAS_NULL_MISSING_VALUES_FIX = -2;
+ private static final long HAS_NULL_MISSING_VALUES_FIX = -3;
private long id;
private long validComponentSequence;
private long lowWatermark;
private long lastComponentId;
private Map<Long, Long> masterNodeFlushMap;
+ private String masterNodeId;
+ private long masterValidSeq;
- public static IndexCheckpoint first(long lastComponentSequence, long lowWatermark, long validComponentId) {
+ public static IndexCheckpoint first(long lastComponentSequence, long lowWatermark, long validComponentId,
+ String masterNodeId) {
IndexCheckpoint firstCheckpoint = new IndexCheckpoint();
firstCheckpoint.id = INITIAL_CHECKPOINT_ID;
firstCheckpoint.lowWatermark = lowWatermark;
@@ -53,11 +56,13 @@
firstCheckpoint.lastComponentId = validComponentId;
firstCheckpoint.masterNodeFlushMap = new HashMap<>();
firstCheckpoint.masterNodeFlushMap.put(HAS_NULL_MISSING_VALUES_FIX, HAS_NULL_MISSING_VALUES_FIX);
+ firstCheckpoint.masterNodeId = masterNodeId;
+ firstCheckpoint.masterValidSeq = lastComponentSequence;
return firstCheckpoint;
}
public static IndexCheckpoint next(IndexCheckpoint latest, long lowWatermark, long validComponentSequence,
- long lastComponentId) {
+ long lastComponentId, String masterNodeId) {
if (lowWatermark < latest.getLowWatermark()) {
if (LOGGER.isErrorEnabled()) {
LOGGER.error("low watermark {} less than the latest checkpoint low watermark {}", lowWatermark, latest);
@@ -70,8 +75,15 @@
next.lastComponentId = lastComponentId;
next.validComponentSequence = validComponentSequence;
next.masterNodeFlushMap = latest.getMasterNodeFlushMap();
+ if (masterNodeId != null) {
+ next.masterNodeId = masterNodeId;
+ next.masterValidSeq = validComponentSequence;
+ } else {
+ next.masterNodeId = latest.getMasterNodeId();
+ next.masterValidSeq = latest.getMasterValidSeq();
+ }
// remove any lsn from the map that wont be used anymore
- next.masterNodeFlushMap.values().removeIf(lsn -> lsn <= lowWatermark && lsn != HAS_NULL_MISSING_VALUES_FIX);
+ next.masterNodeFlushMap.values().removeIf(lsn -> lsn < lowWatermark && lsn != HAS_NULL_MISSING_VALUES_FIX);
return next;
}
@@ -111,6 +123,14 @@
}
}
+ public String getMasterNodeId() {
+ return masterNodeId;
+ }
+
+ public long getMasterValidSeq() {
+ return masterValidSeq;
+ }
+
public static IndexCheckpoint fromJson(String json) throws HyracksDataException {
try {
return OBJECT_MAPPER.readValue(json, IndexCheckpoint.class);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ReplicaIdentifier.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ReplicaIdentifier.java
index c4bb74c..1d02fee 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ReplicaIdentifier.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ReplicaIdentifier.java
@@ -26,16 +26,18 @@
private final int partition;
private final String id;
+ private final String nodeId;
private volatile InetSocketAddress location;
- private ReplicaIdentifier(int partition, InetSocketAddress location) {
+ private ReplicaIdentifier(int partition, String nodeId, InetSocketAddress location) {
this.partition = partition;
+ this.nodeId = nodeId;
this.location = location;
id = partition + "@" + location.getHostString() + ":" + location.getPort();
}
- public static ReplicaIdentifier of(int partition, InetSocketAddress location) {
- return new ReplicaIdentifier(partition, location);
+ public static ReplicaIdentifier of(int partition, String nodeId, InetSocketAddress location) {
+ return new ReplicaIdentifier(partition, nodeId, location);
}
public int getPartition() {
@@ -52,6 +54,10 @@
return location;
}
+ public String getNodeId() {
+ return nodeId;
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) {
@@ -69,6 +75,10 @@
return id.hashCode();
}
+ public String getId() {
+ return id;
+ }
+
@Override
public String toString() {
return id;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
index ebf212b..7065767 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
@@ -147,6 +147,10 @@
return ResourceReference.ofIndex(relativePath.getParent().resolve(dataset).toFile().getPath());
}
+ public boolean isMetadataResource() {
+ return getName().equals(StorageConstants.METADATA_FILE_NAME);
+ }
+
public Path getFileRelativePath() {
return relativePath.resolve(name);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/StorageIOStats.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/StorageIOStats.java
index 0b44f76..da4578f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/StorageIOStats.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/StorageIOStats.java
@@ -22,6 +22,7 @@
private int pendingFlushes;
private int pendingMerges;
+ private int pendingReplications;
public void addPendingFlushes(int pending) {
pendingFlushes += pending;
@@ -31,6 +32,10 @@
pendingMerges += pending;
}
+ public void addPendingReplications(int pending) {
+ pendingReplications += pending;
+ }
+
public int getPendingFlushes() {
return pendingFlushes;
}
@@ -38,4 +43,8 @@
public int getPendingMerges() {
return pendingMerges;
}
+
+ public int getPendingReplications() {
+ return pendingReplications;
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
index 6a080bd..2d231d3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
@@ -37,6 +37,7 @@
* begin with ".". Otherwise {@link AbstractLSMIndexFileManager} will try to
* use them as index files.
*/
+ public static final String INDEX_NON_DATA_FILES_PREFIX = ".";
public static final String INDEX_CHECKPOINT_FILE_PREFIX = ".idx_checkpoint_";
public static final String METADATA_FILE_NAME = ".metadata";
public static final String MASK_FILE_PREFIX = ".mask_";
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 32a226e..f84472e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -90,9 +90,11 @@
}
public static int getPartitionNumFromRelativePath(String relativePath) {
- int startIdx = relativePath.indexOf(StorageConstants.PARTITION_DIR_PREFIX)
+ int startIdx = relativePath.lastIndexOf(StorageConstants.PARTITION_DIR_PREFIX)
+ StorageConstants.PARTITION_DIR_PREFIX.length();
- String partition = relativePath.substring(startIdx, relativePath.indexOf(File.separatorChar, startIdx));
+ int partitionEndIdx = relativePath.indexOf(File.separatorChar, startIdx);
+ int idxEnd = partitionEndIdx != -1 ? partitionEndIdx : relativePath.length();
+ String partition = relativePath.substring(startIdx, idxEnd);
return Integer.parseInt(partition);
}
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index f7c6a77..5120908f 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -92,6 +92,7 @@
200 = External UDF cannot produce expected result. Please check the UDF configuration
201 = External UDF returned exception. Returned exception was: %1$s
+202 = External UDF protocol encountered an unexpected return result.
# Compile-time check errors
1001 = Syntax error: %1$s
@@ -106,10 +107,10 @@
1016 = Index of type %1$s is not supported for dataset of type %2$s
1017 = The filter field \"%1$s\" cannot be an optional field
1018 = Field of type %1$s cannot be used as a filter field
-1019 = Cannot autogenerate a composite primary key
-1020 = Cannot autogenerate a primary key for primary key of type %1$s. Autogenerated primary keys must be of type %2$s
-1021 = The primary key field \"%1$s\" cannot be nullable
-1022 = Field of type %1$s cannot be used as a primary key field
+1019 = Cannot autogenerate a composite %1$s key
+1020 = Cannot autogenerate a %1$s key for %1$s key of type %2$s. Autogenerated %1$s keys must be of type %3$s
+1021 = The %1$s key field \"%2$s\" cannot be nullable
+1022 = Field of type %1$s cannot be used as a %2$s key field
1023 = Cannot drop dataset %1$s since it is connected to active entity: %2$s
#1024 is no longer used
#1025 is no longer used
@@ -235,14 +236,29 @@
1146 = Cannot find feed policy with name %1$s
1147 = Cannot drop dataverse: %1$s %2$s being used by %3$s %4$s
1148 = Cannot drop %1$s %2$s being used by %3$s %4$s
-1149 = Illegal function recursion
+1149 = Illegal function or view recursion
1150 = Illegal use of function %1$s
-1151 = Full-text config %1$s not found
-1152 = Full-text filter %1$s not found
-1153 = Default full-text config with a name of null cannot be deleted or created
-1154 = Incompatible index type %1$s
-1155 = Full-text config %1$s already exists
-1156 = Full-text filter %1$s already exists
+1151 = No authentication credentials provided, '%1$s' field is required for anonymous access
+1152 = Full-text config %1$s not found
+1153 = Full-text filter %1$s not found
+1154 = Default full-text config with a name of null cannot be deleted or created
+1155 = Incompatible index type %1$s
+1156 = Full-text config %1$s already exists
+1157 = Full-text filter %1$s already exists
+1158 = Error compiling view %1$s. %2$s
+1159 = Cannot find view with name %1$s
+1160 = A view with this name %1$s already exists
+1161 = Type '%1$s' contains declared fields, which is not supported for 'parquet' format
+1162 = Invalid primary key definition
+1163 = Authenticating with '%1$s' is not supported for '%2$s' format
+1164 = Invalid foreign key definition
+1165 = Invalid foreign key definition: %1$s %2$s does not have a primary key
+1166 = Invalid foreign key definition: foreign key does not match primary key of %1$s %2$s
+1167 = Cannot change primary key of %1$s %2$s
+1168 = Ambiguous projection in SELECT clause
+1169 = Unable to do subquery coercion. %1$s
+1170 = Provided S3 region is not supported: '%1$s'
+1171 = Unable to process %1$s clause. %2$s
# Feed Errors
3001 = Illegal state.
@@ -361,6 +377,8 @@
3118 = Active Entity %1$s is not running (it is %2$s)
3119 = Parameter '%1$s' is required if '%2$s' is provided
3120 = Unexpected token %s: was expecting %s
+3121 = Parameter '%1$s' or '%2$s' is required if '%3$s' is provided
+3122 = Parameter '%1$s' is not allowed if '%2$s' is provided
# Lifecycle management errors
4000 = Partition id %1$s for node %2$s already in use by node %3$s
diff --git a/asterixdb/asterix-dashboard/pom.xml b/asterixdb/asterix-dashboard/pom.xml
index 2249a69..9d962f8 100644
--- a/asterixdb/asterix-dashboard/pom.xml
+++ b/asterixdb/asterix-dashboard/pom.xml
@@ -46,13 +46,13 @@
<artifactId>log4j-api</artifactId>
</dependency>
<dependency>
- <groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
- </dependency>
- <dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
+ </dependency>
</dependencies>
<profiles>
@@ -66,7 +66,7 @@
<plugin>
<groupId>com.github.eirslett</groupId>
<artifactId>frontend-maven-plugin</artifactId>
- <version>1.6</version>
+ <version>1.11.0</version>
<configuration>
<nodeVersion>v14.15.4</nodeVersion>
<npmVersion>6.14.11</npmVersion>
@@ -182,4 +182,26 @@
<id>skip-dashboard</id>
</profile>
</profiles>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>touch-3rdparty</id>
+ <phase>${skip-npm-touch.stage}</phase>
+ <configuration>
+ <target>
+ <touch file="${basedir}/target/dashboard/static/3rdpartylicenses.txt" mkdirs="true"/>
+ </target>
+ </configuration>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
</project>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
index 582b2f2..f885edb 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
@@ -93,7 +93,7 @@
//sql++ keywords
sqlppKeywords = "alter and as asc between by count create delete desc distinct drop from group having in insert into " +
"is join like not on or order select set union update values where limit use let dataverse dataset exists with index type" +
- "inner outer offset value type if exists declare function";
+ "inner outer offset value type if exists declare function explain";
//sql++ builtin types
sqlppTypes = "boolean tinyint smallint integer int bigint string float double binary point line rectangle circle polygon" +
@@ -294,12 +294,18 @@
}
onClickExplain() {
- let use_regex = /use .*?;/i
+ //for future use...currently we do not support explaining for INSERT, UPDATE, or DELETE
+ /*
+ let insert_regex = /insert/i;
+ let update_regex = /update/i;
+ let delete_regex = /delete/i;
+ */
+ let select_regex = /select/i;
let explainString = "";
- if (use_regex.test(this.queryString))
- explainString = this.queryString.replace(use_regex, "$& explain ");
+ if (select_regex.test(this.queryString))
+ explainString = this.queryString.replace(select_regex, "explain $& ");
else
explainString = "explain " + this.queryString;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
index f6b0533..0fbf6f6 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
@@ -20,8 +20,9 @@
<mat-panel-description class='content'>
<div class='panel'>
<span class='options'>
- <button mat-button [ngClass]="this.tableVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showTable()' matTooltip="Show Table View">TABLE</button>
<button mat-button *ngIf="isCSV == false;" [ngClass]="this.jsonVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showJSON()' matTooltip="Show JSON View">JSON</button>
+ <button mat-button *ngIf="isCSV == false;" [ngClass]="this.jsonlVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showJSONL()' matTooltip="Show JSONL View">JSONL</button>
+ <button mat-button [ngClass]="this.tableVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showTable()' matTooltip="Show Table View">TABLE</button>
<button mat-button *ngIf="isCSV == false;" [ngClass]="this.treeVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showTree()' matTooltip="Show Tree View">TREE</button>
<button mat-button [ngClass]="this.planVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showPlan()' matTooltip="Show Plan Viewer">PLAN</button>
@@ -56,6 +57,9 @@
<div *ngIf="planVisible" class="plan">
<plan-viewer [planFormat]="planFormat" [jsonPlan]="jsonPlan" [plan]="plan" [planName]="planName"></plan-viewer>
</div>
+ <div *ngIf="jsonlVisible" class="json">
+ <pre>{{jsonlData}}</pre>
+ </div>
<div id='bottom'></div>
</div>
<button *ngIf='showGoTop' mat-fab color='primary' class='button back-button' (click)='gotoTop()'>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
index 9818973..5309991 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
@@ -45,10 +45,12 @@
exportFileName: string = 'asterixdb-query-results';
jsonVisible: any = true;
+ jsonlVisible: any = true;
tableVisible: any = false;
treeVisible: any = false;
planVisible: any = false;
jsonData: any;
+ jsonlData: any;
jsonPath_: any = ': < JSON PATH >';
rawData: any;
treeData: any;
@@ -84,6 +86,7 @@
ngOnChanges(changes: SimpleChange) {
if (this.inputToOutput) {
this.jsonVisible = true;
+ this.jsonlVisible = false;
this.planVisible = false;
this.tableVisible = false;
this.treeVisible = false;
@@ -119,6 +122,7 @@
if (this.isExplain) {
this.jsonVisible = false;
+ this.jsonlVisible = false;
this.planVisible = true;
this.tableVisible = false;
this.treeVisible = false;
@@ -158,6 +162,7 @@
this.treeData_ = myData_;
/* Prepare the JSON view */
this.jsonData = JSON.stringify(this.treeData, null, 8)
+ this.jsonlData = this.jsonlinesTransform(this.treeData);
} else {
console.log('no data')
this.treeData = [];
@@ -165,6 +170,7 @@
} else {
this.treeData = [];
this.jsonData = JSON.stringify([ ], null, 8);
+ this.jsonlData = "";
this.metrics = {"resultCount": 0};
//clear tree data
@@ -188,6 +194,7 @@
*/
showJSON() {
this.jsonVisible = true;
+ this.jsonlVisible = false;
this.treeVisible = false;
this.tableVisible = false;
this.planVisible = false;
@@ -195,10 +202,23 @@
}
/*
+ * Shows JSONL View
+ */
+ showJSONL() {
+ this.jsonVisible = false;
+ this.jsonlVisible = true;
+ this.treeVisible = false;
+ this.tableVisible = false;
+ this.planVisible = false;
+ this.viewMode = 'JSONL'
+ }
+
+ /*
* Shows Table View
*/
showTable() {
this.jsonVisible = false;
+ this.jsonlVisible = false;
this.treeVisible = false;
this.tableVisible = true;
this.planVisible = false;
@@ -210,6 +230,7 @@
*/
showTree() {
this.jsonVisible = false;
+ this.jsonlVisible = false;
this.treeVisible = true;
this.tableVisible = false;
this.planVisible = false;
@@ -221,6 +242,7 @@
*/
showPlan() {
this.jsonVisible = false;
+ this.jsonlVisible = false;
this.treeVisible = false;
this.tableVisible = false;
this.planVisible = true;
diff --git a/asterixdb/asterix-doc/pom.xml b/asterixdb/asterix-doc/pom.xml
index 22bd84d..d0825d4 100644
--- a/asterixdb/asterix-doc/pom.xml
+++ b/asterixdb/asterix-doc/pom.xml
@@ -191,10 +191,10 @@
<id>extract-rr</id>
<phase>generate-sources</phase>
<configuration>
- <tasks>
+ <target>
<echo message="Extracting rr" />
<unzip src="${project.build.directory}/rr-1.62-java8.zip" dest="${project.build.directory}" />
- </tasks>
+ </target>
</configuration>
<goals>
<goal>run</goal>
@@ -204,13 +204,13 @@
<id>extract-diagrams</id>
<phase>process-resources</phase>
<configuration>
- <tasks>
+ <target>
<echo message="Extracting diagrams" />
<unzip src="${project.build.directory}/railroads.zip" dest="${project.build.directory}" />
<copy todir="${project.build.directory}/site/images/diagrams/">
<fileset dir="${project.build.directory}/diagram/"/>
</copy>
- </tasks>
+ </target>
</configuration>
<goals>
<goal>run</goal>
diff --git a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
index db89fe0..b237f30 100644
--- a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
+++ b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
@@ -5,7 +5,7 @@
| OperatorExpr Operator OperatorExpr?
| OperatorExpr "BETWEEN" OperatorExpr "AND" OperatorExpr
-QuantifiedExpr::= ( "SOME" | "EVERY" ) Variable "IN" Expr ( "," Variable "IN" Expr )* "SATISFIES" Expr ("END")?
+QuantifiedExpr::= ( "SOME" | "EVERY" | ( "SOME" "AND" "EVERY" ) ) Variable "IN" Expr ( "," Variable "IN" Expr )* "SATISFIES" Expr ("END")?
PathExpr ::= PrimaryExpr ("." Identifier | "[" Expr (":" (Expr)? )? "]")*
@@ -28,8 +28,6 @@
ParenthesizedExpr ::= ("(" Expr ")") | Subquery
-Subquery ::= ("(" Selection ")")
-
FunctionCall ::= OrdinaryFunctionCall | AggregateFunctionCall | WindowFunctionCall
OrdinaryFunctionCall ::= (DataverseName ".")? Identifier "(" Expr ("," Expr)* ")"
@@ -42,19 +40,21 @@
SearchedCaseExpr ::= "CASE" ("WHEN" Expr "THEN" Expr)+ ("ELSE" Expr)? "END"
-Constructor ::= ObjectConstructor | ArrayConstructor | MultisetConstructor
+Constructor ::= ObjectConstructor | ArrayConstructor | ParenthesizedArrayConstructor | MultisetConstructor
ObjectConstructor ::= "{" ( Expr ( ":" Expr )? ( "," Expr ( ":" Expr )? )* )? "}"
ArrayConstructor ::= "[" Expr ("," Expr)* "]"
+ParenthesizedArrayConstructor ::= "(" Expr ("," Expr)+ ")"
+
MultisetConstructor ::= "{{" Expr ("," Expr)* "}}"
Query ::= (Expr | Selection)
-Selection ::= WithClause? QueryBlock UnionOption* OrderByClause? ( LimitClause | OffsetClause )?
+Selection ::= (WithClause | LetClause)? QueryBlock UnionOption* OrderByClause? ( LimitClause | OffsetClause )?
-QueryBlock ::= SelectClause StreamGenerator?
+QueryBlock ::= SelectClause ( ( LetClause WhereClause? ) | StreamGenerator )?
| StreamGenerator SelectClause
StreamGenerator::= FromClause LetClause? WhereClause? (GroupByClause LetClause? HavingClause?)?
@@ -69,7 +69,8 @@
NamedExpr ::= Expr ("AS"? Variable)?
-JoinStep ::= ("INNER" | ( ( "LEFT" | "RIGHT" ) "OUTER"?))? "JOIN" NamedExpr "ON" Expr
+JoinStep ::= ( ("INNER" | ( ( "LEFT" | "RIGHT" ) "OUTER"?))? "JOIN" NamedExpr "ON" Expr ) |
+ ( "CROSS" "JOIN" NamedExpr )
UnnestStep ::= ("INNER" | ( "LEFT" "OUTER"?))? "UNNEST" NamedExpr
@@ -96,7 +97,7 @@
WithClause ::= "WITH" Variable "AS" Expr ("," Variable "AS" Expr)*
-OrderbyClause ::= "ORDER BY" Expr ( "ASC" | "DESC" )? ( "," Expr ( "ASC" | "DESC" )? )*
+OrderbyClause ::= "ORDER BY" Expr ( "ASC" | "DESC" )? ( "NULLS" ( "FIRST" | "LAST" ) )? ( "," Expr ( "ASC" | "DESC" )? ( "NULLS" ( "FIRST" | "LAST" ) )? )*
LimitClause ::= "LIMIT" Expr OffsetClause?
@@ -116,7 +117,7 @@
WindowPartitionClause ::= "PARTITION" "BY" Expr ("," Expr)*
-WindowOrderClause ::= "ORDER" "BY" Expr ("ASC"|"DESC")? ("," Expr ("ASC" | "DESC")?)*
+WindowOrderClause ::= "ORDER" "BY" Expr ("ASC" | "DESC")? ( "NULLS" ( "FIRST" | "LAST" ) )? ("," Expr ("ASC" | "DESC")? ( "NULLS" ( "FIRST" | "LAST" ) )? )*
WindowFrameClause ::= ("ROWS" | "RANGE" | "GROUPS") WindowFrameExtent
@@ -149,6 +150,7 @@
| CreateIndex
| CreateSynonym
| CreateFunction
+ | CreateView
DataverseName ::= Identifier ("." Identifier)*
@@ -211,6 +213,7 @@
CreateSecondaryIndex ::= "CREATE" "INDEX" Identifier ("IF" "NOT" "EXISTS")? "ON" QualifiedName
"(" IndexedElement ( "," IndexedElement )* ")" ("TYPE" IndexType)? ("ENFORCED")?
+ (( "EXCLUDE" | "INCLUDE" ) "UNKNOWN" "KEY")?
CreatePrimaryKeyIndex ::= "CREATE" "PRIMARY" "INDEX" Identifier? ("IF" "NOT" "EXISTS")? "ON" QualifiedName ("TYPE" "BTREE")?
@@ -240,8 +243,10 @@
ExternalFunctionDef ::= ("RETURNS" TypeExpr)? "AS" StringLiteral ("," StringLiteral )* "AT" QualifiedName ("WITH" ObjectConstructor)?
+CreateView ::= "CREATE" ("OR" "REPLACE")? "VIEW" QualifiedName ("IF" "NOT" "EXISTS")? "AS" Selection
+
DropStmnt ::= "DROP" ("DATAVERSE" DataverseName
- | ("TYPE" |"DATASET" | "SYNONYM") QualifiedName
+ | ("TYPE" |"DATASET" |"SYNONYM" |"VIEW") QualifiedName
| "INDEX" DoubleQualifiedName
| "FUNCTION" FunctionSignature ) ("IF" "EXISTS")?
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/7_temporal.md b/asterixdb/asterix-doc/src/main/markdown/builtins/7_temporal.md
index ab8b753..a1cdda2 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/7_temporal.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/7_temporal.md
@@ -39,7 +39,7 @@
"year": get_year(date("2010-10-30")),
"month": get_month(datetime("1987-11-19T23:49:23.938")),
"day": get_day(date("2010-10-30")),
- "hour": get_hour(time("12:23:34.930+07:00")),
+ "hour": get_hour(time("12:23:34.930")),
"min": get_minute(duration("P3Y73M632DT49H743M3948.94S")),
"second": get_second(datetime("1987-11-19T23:49:23.938")),
"ms": get_millisecond(duration("P3Y73M632DT49H743M3948.94S"))
@@ -48,7 +48,7 @@
* The expected result is:
- { "year": 2010, "month": 11, "day": 30, "hour": 5, "min": 28, "second": 23, "ms": 94 }
+ { "year": 2010, "month": 11, "day": 30, "hour": 12, "min": 28, "second": 23, "ms": 94 }
### adjust_datetime_for_timezone ###
@@ -282,6 +282,15 @@
* a `date` value from the datetime,
* any other non-datetime input value will cause a type error.
+* Example:
+
+ get_date_from_datetime(datetime("2016-03-26T10:10:00"));
+
+* The expected result is:
+
+ date("2016-03-26")
+
+
### get_time_from_datetime ###
* Syntax:
@@ -302,74 +311,110 @@
* The expected result is:
- time("10:10:00.000Z")
+ time("10:10:00.000")
### day_of_week ###
* Syntax:
- day_of_week(date)
+ day_of_week(date[, week_start_day])
* Finds the day of the week for a given date (1_7)
* Arguments:
- * `date`: a `date` value (Can also be a `datetime`)
+ * `date`: a `date` or a `datetime` value
+ * `week_start_day`: (Optional) an integer or a string value (case-insensitive) specifying the day of the week
+ to start counting from: 1=Sun[day], 2=Mon[day], ..., 7=Sat[urday]. If omitted, the default is 1 (Sunday).
* Return Value:
- * an `tinyint` representing the day of the week (1_7),
+ * an `bigint` representing the day of the week (1_7),
* `missing` if the argument is a `missing` value,
* `null` if the argument is a `null` value,
* any other non-date input value will cause a type error.
* Example:
- day_of_week(datetime("2012-12-30T12:12:12.039Z"));
-
+ {
+ "day_1": day_of_week(datetime("2012-12-30T12:12:12.039")),
+ "day_2": day_of_week(datetime("2012-12-30T12:12:12.039"), 2),
+ "day_3": day_of_week(datetime("2012-12-30T12:12:12.039"), "Monday"),
+ "day_4": day_of_week(datetime("2012-12-30T12:12:12.039"), "MON")
+ };
* The expected result is:
- 7
+ { "day_1": 1, "day_2": 7, "day_3": 7, "day_4": 7 }
+### day_of_year ###
+* Syntax:
-### date_from_unix_time_in_days ###
- * Syntax:
+ day_of_year(date)
- date_from_unix_time_in_days(numeric_value)
-
- * Gets a date representing the time after `numeric_value` days since 1970_01_01.
- * Arguments:
- * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint` value representing the number of days.
- * Return Value:
- * a `date` value as the time after `numeric_value` days since 1970-01-01,
+* Finds the day of the year for a given date
+* Arguments:
+ * `date`: a `date` or a `datetime` value
+* Return Value:
+ * an `bigint` representing the day of the year,
* `missing` if the argument is a `missing` value,
* `null` if the argument is a `null` value,
- * any other non-numeric input value will cause a type error.
+ * any other non-date input value will cause a type error.
-### datetime_from_unix_time_in_ms ###
- * Syntax:
+* Example:
- datetime_from_unix_time_in_ms(numeric_value)
+ day_of_year(date("2011-12-31"));
- * Gets a datetime representing the time after `numeric_value` milliseconds since 1970_01_01T00:00:00Z.
- * Arguments:
- * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint` value representing the number of milliseconds.
- * Return Value:
- * a `datetime` value as the time after `numeric_value` milliseconds since 1970-01-01T00:00:00Z,
+* The expected result is:
+
+ 365
+
+### week_of_year ###
+* Syntax:
+
+ week_of_year(date[, week_start_day])
+
+* Finds the week of the year for a given date
+* Arguments:
+ * `date`: a `date` or a `datetime` value
+ * `week_start_day`: (Optional) an integer or a string value (case-insensitive) specifying the day of the week
+ to start counting from: 1=Sun[day], 2=Mon[day], ..., 7=Sat[urday]. If omitted, the default is 1 (Sunday).
+* Return Value:
+ * an `bigint` representing the week of the year,
* `missing` if the argument is a `missing` value,
* `null` if the argument is a `null` value,
- * any other non-numeric input value will cause a type error.
+ * any other non-date input value will cause a type error.
-### datetime_from_unix_time_in_secs ###
- * Syntax:
+* Example:
- datetime_from_unix_time_in_secs(numeric_value)
+ {
+ "week_1": week_of_year(date("2012-12-01")),
+ "week_2": week_of_year(date("2012-12-01"), 2),
+ "week_3": week_of_year(date("2012-12-01"), "Monday"),
+ "week_4": week_of_year(date("2012-12-01"), "MON")
+ };
- * Gets a datetime representing the time after `numeric_value` seconds since 1970_01_01T00:00:00Z.
- * Arguments:
- * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint` value representing the number of seconds.
- * Return Value:
- * a `datetime` value as the time after `numeric_value` seconds since 1970_01_01T00:00:00Z,
+* The expected result is:
+
+ { "week_1": 48, "week_2": 49, "week_3": 49, "week_4": 49 }
+
+### quarter_of_year ###
+* Syntax:
+
+ quarter_of_year(date)
+
+* Finds the quarter of the year for a given date
+* Arguments:
+ * `date`: a `date` or a `datetime` value
+* Return Value:
+ * an `bigint` representing the quarter of the year (1_4),
* `missing` if the argument is a `missing` value,
* `null` if the argument is a `null` value,
- * any other non-numeric input value will cause a type error.
+ * any other non-date input value will cause a type error.
+
+* Example:
+
+ quarter_of_year(date("2011-12-31"));
+
+* The expected result is:
+
+ 4
### datetime_from_date_time ###
* Syntax:
@@ -388,40 +433,111 @@
* the first argument is any other non-date value,
* or, the second argument is any other non-time value.
-### time_from_unix_time_in_ms ###
+### date_from_unix_time_in_days ###
* Syntax:
- time_from_unix_time_in_ms(numeric_value)
+ date_from_unix_time_in_days(numeric_value)
- * Gets a time representing the time after `numeric_value` milliseconds since 00:00:00.000Z.
+ * Gets a date representing the time after `numeric_value` days since 1970-01-01.
+ * Arguments:
+ * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint` value representing the number of days.
+ * Return Value:
+ * a `date` value as the time after `numeric_value` days since 1970-01-01,
+ * `missing` if the argument is a `missing` value,
+ * `null` if the argument is a `null` value,
+ * any other non-numeric input value will cause a type error.
+
+* Example:
+
+ date_from_unix_time_in_days(15800);
+
+* The expected result is:
+
+ date("2013-04-05")
+
+
+### datetime_from_unix_time_in_ms ###
+ * Syntax:
+
+ datetime_from_unix_time_in_ms(numeric_value[, string])
+
+ * Gets a datetime representing the time after `numeric_value` milliseconds since 1970-01-01T00:00:00Z.
* Arguments:
* `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint` value representing the number of milliseconds.
+ * `string` : (Optional) a string representing the target timezone as defined by IANA Time Zone Database.
+ If omitted, the default is UTC.
* Return Value:
- * a `time` value as the time after `numeric_value` milliseconds since 00:00:00.000Z,
+ * a `datetime` value as the time in the target time zone after `numeric_value` milliseconds since 1970-01-01T00:00:00Z,
* `missing` if the argument is a `missing` value,
* `null` if the argument is a `null` value,
* any other non-numeric input value will cause a type error.
* Example:
+ {
+ "datetime_1": datetime_from_unix_time_in_ms(1365139700000),
+ "datetime_2": datetime_from_unix_time_in_ms(1365139700000, "America/Los_Angeles")
+ };
+
+* The expected result is:
+
+ { "datetime_1": datetime("2013-04-05T05:28:20.000"), "datetime_2": datetime("2013-04-04T22:28:20.000") }
+
+### datetime_from_unix_time_in_secs ###
+ * Syntax:
+
+ datetime_from_unix_time_in_secs(numeric_value[, string])
+
+ * Gets a datetime representing the time after `numeric_value` seconds since 1970-01-01T00:00:00Z.
+ * Arguments:
+ * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint` value representing the number of seconds.
+ * `string` : (Optional) a string representing the target timezone as defined by IANA Time Zone Database.
+ If omitted, the default is UTC.
+ * Return Value:
+ * a `datetime` value as the time in the target time zone after `numeric_value` seconds since 1970-01-01T00:00:00Z,
+ * `missing` if the argument is a `missing` value,
+ * `null` if the argument is a `null` value,
+ * any other non-numeric input value will cause a type error.
+
+* Example:
+
{
- "date": date_from_unix_time_in_days(15800),
- "datetime": datetime_from_unix_time_in_ms(1365139700000),
- "time": time_from_unix_time_in_ms(3748)
+ "datetime_1": datetime_from_unix_time_in_secs(1365139700),
+ "datetime_2": datetime_from_unix_time_in_secs(1365139700, "America/Los_Angeles")
};
+* The expected result is:
+
+ { "datetime_1": datetime("2013-04-05T05:28:20.000"), "datetime_2": datetime("2013-04-04T22:28:20.000") }
+
+### time_from_unix_time_in_ms ###
+ * Syntax:
+
+ time_from_unix_time_in_ms(numeric_value)
+
+ * Gets a time representing the time after `numeric_value` milliseconds since 00:00:00.000.
+ * Arguments:
+ * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint` value representing the number of milliseconds.
+ * Return Value:
+ * a `time` value as the time after `numeric_value` milliseconds since 00:00:00.000,
+ * `missing` if the argument is a `missing` value,
+ * `null` if the argument is a `null` value,
+ * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+ time_from_unix_time_in_ms(3748);
* The expected result is:
- { "date": date("2013-04-05"), "datetime": datetime("2013-04-05T05:28:20.000Z"), "time": time("00:00:03.748Z") }
-
+ time("00:00:03.748")
### unix_time_from_date_in_days ###
* Syntax:
unix_time_from_date_in_days(date_value)
- * Gets an integer value representing the number of days since 1970_01_01 for `date_value`.
+ * Gets an integer value representing the number of days since 1970-01-01 for `date_value`.
* Arguments:
* `date_value`: a `date` value.
* Return Value:
@@ -430,43 +546,76 @@
* `null` if the argument is a `null` value,
* any other non-date input value will cause a type error.
+* Example:
+
+ unix_time_from_date_in_days(date("2013-04-05"));
+
+* The expected result is:
+
+ 15800
### unix_time_from_datetime_in_ms ###
* Syntax:
- unix_time_from_datetime_in_ms(datetime_value)
+ unix_time_from_datetime_in_ms(datetime_value[, string])
- * Gets an integer value representing the time in milliseconds since 1970_01_01T00:00:00Z for `datetime_value`.
+ * Gets an integer value representing the time in milliseconds since 1970-01-01T00:00:00Z for `datetime_value`.
* Arguments:
* `datetime_value` : a `datetime` value.
+ * `string` : (Optional) a string representing the source timezone as defined by IANA Time Zone Database.
+ If omitted, the default is UTC.
+
* Return Value:
* a `bigint` value representing the number of milliseconds,
* `missing` if the argument is a `missing` value,
* `null` if the argument is a `null` value,
* any other non-datetime input value will cause a type error.
+* Example:
+
+ {
+ "unix_time_1": unix_time_from_datetime_in_ms(datetime("2013-04-05T05:28:20.000")),
+ "unix_time_2": unix_time_from_datetime_in_ms(datetime("2013-04-04T22:28:20.000"), "America/Los_Angeles")
+ };
+
+* The expected result is:
+
+ { "unix_time_1": 1365139700000, "unix_time_2": 1365139700000 }
+
### unix_time_from_datetime_in_secs ###
* Syntax:
- unix_time_from_datetime_in_secs(datetime_value)
+ unix_time_from_datetime_in_secs(datetime_value[, string])
- * Gets an integer value representing the time in seconds since 1970_01_01T00:00:00Z for `datetime_value`.
+ * Gets an integer value representing the time in seconds since 1970-01-01T00:00:00Z for `datetime_value`.
* Arguments:
* `datetime_value` : a `datetime` value.
+ * `string` : (Optional) a string representing the source timezone as defined by IANA Time Zone Database.
+ If omitted, the default is UTC.
* Return Value:
* a `bigint` value representing the number of seconds,
* `missing` if the argument is a `missing` value,
* `null` if the argument is a `null` value,
* any other non-datetime input value will cause a type error.
+* Example:
+
+ {
+ "unix_time_1": unix_time_from_datetime_in_secs(datetime("2013-04-05T05:28:20.000")),
+ "unix_time_2": unix_time_from_datetime_in_secs(datetime("2013-04-04T22:28:20.000"), "America/Los_Angeles")
+ };
+
+* The expected result is:
+
+ { "unix_time_1": 1365139700, "unix_time_2": 1365139700 }
### unix_time_from_time_in_ms ###
* Syntax:
unix_time_from_time_in_ms(time_value)
- * Gets an integer value representing the time the milliseconds since 00:00:00.000Z for `time_value`.
+ * Gets an integer value representing the time the milliseconds since 00:00:00.000 for `time_value`.
* Arguments:
* `time_value` : a `time` value.
* Return Value:
@@ -477,17 +626,11 @@
* Example:
- {
- "date": date_from_unix_time_in_days(15800),
- "datetime": datetime_from_unix_time_in_ms(1365139700000),
- "time": time_from_unix_time_in_ms(3748)
- }
-
+ unix_time_from_time_in_ms(time("00:00:03.748"));
* The expected result is:
- { "date": date("2013-04-05"), "datetime": datetime("2013-04-05T05:28:20.000Z"), "time": time("00:00:03.748Z") }
-
+ 3748
### parse_date/parse_time/parse_datetime ###
* Syntax:
@@ -501,13 +644,16 @@
* `h` hours
* `m` minutes
* `s` seconds
- * `n` milliseconds
+ * `n` (or `S`) milliseconds
* `a` am/pm
- * `z` timezone
+ * `z` timezone (parsed and ignored)
* `Y` year
+ * `Q` quarter of year (1-4)
+ * `QQ` quarter of year (01-04)
* `M` month
* `D` day
- * `W` weekday
+ * `EEE` weekday (abbreviated name, parsed and ignored)
+ * `EEEE` weekday (full name, parsed and ignored)
* `_`, `'`, `/`, `.`, `,`, `T` seperators for both time and date
* Return Value:
* a `date/time/date_time` value corresponding to `date`,
@@ -523,7 +669,7 @@
* The expected result is:
- time("00:30:30.000Z")
+ time("00:30:30.000")
### print_date/print_time/print_datetime ###
@@ -538,13 +684,18 @@
* `h` hours
* `m` minutes
* `s` seconds
- * `n` milliseconds
+ * `n` (or `S`) milliseconds
* `a` am/pm
- * `z` timezone
* `Y` year
+ * `Q` quarter of year (1-4)
+ * `QQ` quarter of year (01-04)
* `M` month
+ * `MMM` month (abbreviated name)
+ * `MMMM` month (full name)
* `D` day
- * `W` weekday
+ * `DDD` day of year
+ * `EEE` weekday (abbreviated name)
+ * `EEEE` weekday (full name)
* `_`, `'`, `/`, `.`, `,`, `T` seperators for both time and date
* Return Value:
* a `string` value corresponding to `date`,
@@ -556,7 +707,7 @@
* Example:
- print_time(time("00:30:30.000Z"),"m:s");
+ print_time(time("00:30:30.000"),"m:s");
* The expected result is:
@@ -588,7 +739,7 @@
* The expected result is:
- { "start": date("1984_01_01"), "end": date("1985_01_01") }
+ { "start": date("1984-01-01"), "end": date("1985-01-01") }
### get_interval_start_date/get_interval_start_datetimeget_interval_start_time, get_interval_end_date/get_interval_end_datetime/get_interval_end_time ###
@@ -622,10 +773,10 @@
{
"start1": date("1984-01-01"),
"end1": date("1985-01-01"),
- "start2": datetime("1984-01-01T08:30:00.000Z"),
- "end2": datetime("1985-01-01T09:30:00.000Z"),
- "start3": time("08:30:00.000Z"),
- "end3": time("09:30:00.000Z")
+ "start2": datetime("1984-01-01T08:30:00.000"),
+ "end2": datetime("1985-01-01T09:30:00.000"),
+ "start3": time("08:30:00.000"),
+ "end3": time("09:30:00.000")
}
@@ -647,7 +798,8 @@
* Example:
- { "overlap1": get_overlapping_interval(interval(time("11:23:39"), time("18:27:19")), interval(time("12:23:39"), time("23:18:00"))),
+ {
+ "overlap1": get_overlapping_interval(interval(time("11:23:39"), time("18:27:19")), interval(time("12:23:39"), time("23:18:00"))),
"overlap2": get_overlapping_interval(interval(time("12:23:39"), time("18:27:19")), interval(time("07:19:39"), time("09:18:00"))),
"overlap3": get_overlapping_interval(interval(date("1980-11-30"), date("1999-09-09")), interval(date("2013-01-01"), date("2014-01-01"))),
"overlap4": get_overlapping_interval(interval(date("1980-11-30"), date("2099-09-09")), interval(date("2013-01-01"), date("2014-01-01"))),
@@ -657,11 +809,12 @@
* The expected result is:
- { "overlap1": interval(time("12:23:39.000Z"), time("18:27:19.000Z")),
+ {
+ "overlap1": interval(time("12:23:39.000"), time("18:27:19.000")),
"overlap2": null,
"overlap3": null,
- "overlap4": interval(date("2013-01-01"), date("2014_01_01")),
- "overlap5": interval(datetime("1989-03-04T12:23:39.000Z"), datetime("2000-10-30T18:27:19.000Z")),
+ "overlap4": interval(date("2013-01-01"), date("2014-01-01")),
+ "overlap5": interval(datetime("1989-03-04T12:23:39.000"), datetime("2000-10-30T18:27:19.000")),
"overlap6": null
}
@@ -680,32 +833,32 @@
* date +|_ year_month_duration
* date +|_ day_time_duration
* time +|_ day_time_duration
- * Return Value:
- * a `interval` value representing the bin containing the `time_to_bin` value. Note that the internal type of
- this interval value should be the same as the `time_to_bin` type,
- * `missing` if any argument is a `missing` value,
- * `null` if any argument is a `null` value but no argument is a `missing` value,
- * a type error will be raised if:
- * the first argument or the second argument is any other non-date/non-time/non-datetime value,
- * or, the second argument is any other non-year_month_duration/non-day_time_duration value.
+ * Return Value:
+ * a `interval` value representing the bin containing the `time_to_bin` value. Note that the internal type of
+ this interval value should be the same as the `time_to_bin` type,
+ * `missing` if any argument is a `missing` value,
+ * `null` if any argument is a `null` value but no argument is a `missing` value,
+ * a type error will be raised if:
+ * the first argument or the second argument is any other non-date/non-time/non-datetime value,
+ * or, the second argument is any other non-year_month_duration/non-day_time_duration value.
- * Example:
+ * Example:
- {
- "bin1": interval_bin(date("2010-10-30"), date("1990-01-01"), year_month_duration("P1Y")),
- "bin2": interval_bin(datetime("1987-11-19T23:49:23.938"), datetime("1990-01-01T00:00:00.000Z"), year_month_duration("P6M")),
- "bin3": interval_bin(time("12:23:34.930+07:00"), time("00:00:00"), day_time_duration("PT1M")),
- "bin4": interval_bin(datetime("1987-11-19T23:49:23.938"), datetime("2013-01-01T00:00:00.000"), day_time_duration("PT24H"))
- };
+ {
+ "bin1": interval_bin(date("2010-10-30"), date("1990-01-01"), year_month_duration("P1Y")),
+ "bin2": interval_bin(datetime("1987-11-19T23:49:23.938"), datetime("1990-01-01T00:00:00.000"), year_month_duration("P6M")),
+ "bin3": interval_bin(time("12:23:34.930+07:00"), time("00:00:00"), day_time_duration("PT1M")),
+ "bin4": interval_bin(datetime("1987-11-19T23:49:23.938"), datetime("2013-01-01T00:00:00.000"), day_time_duration("PT24H"))
+ };
- * The expected result is:
+ * The expected result is:
- {
- "bin1": interval(date("2010-01-01"),date("2011-01-01")),
- "bin2": interval(datetime("1987-07-01T00:00:00.000Z"), datetime("1988-01-01T00:00:00.000Z")),
- "bin3": interval(time("05:23:00.000Z"), time("05:24:00.000Z")),
- "bin4": interval(datetime("1987-11-19T00:00:00.000Z"), datetime("1987-11-20T00:00:00.000Z"))
- }
+ {
+ "bin1": interval(date("2010-01-01"), date("2011-01-01")),
+ "bin2": interval(datetime("1987-07-01T00:00:00.000"), datetime("1988-01-01T00:00:00.000")),
+ "bin3": interval(time("12:23:00.000"), time("12:24:00.000")),
+ "bin4": interval(datetime("1987-11-19T00:00:00.000"), datetime("1987-11-20T00:00:00.000"))
+ }
### interval_start_from_date/time/datetime ###
@@ -733,12 +886,12 @@
"interval3": interval_start_from_datetime("1999-09-09T09:09:09.999", duration("P2M30D"))
};
- * The expectecd result is:
+ * The expected result is:
{
"interval1": interval(date("1984-01-01"), date("1985-01-01")),
- "interval2": interval(time("02:23:28.394Z"), time("05:47:28.394Z")),
- "interval3": interval(datetime("1999-09-09T09:09:09.999Z"), datetime("1999-12-09T09:09:09.999Z"))
+ "interval2": interval(time("02:23:28.394"), time("05:47:28.394")),
+ "interval3": interval(datetime("1999-09-09T09:09:09.999"), datetime("1999-12-09T09:09:09.999"))
}
@@ -746,20 +899,20 @@
* Return Value:
* a `interval` value representing the bin containing the `time_to_bin` value. Note that the internal type of this interval value should be the same as the `time_to_bin` type.
- * Syntax:
+ * Syntax:
- overlap_bins(interval, time_bin_anchor, duration_bin_size)
+ overlap_bins(interval, time_bin_anchor, duration_bin_size)
- * Returns an ordered list of `interval` values representing each bin that is overlapping the `interval`.
- * Arguments:
- * `interval`: an `interval` value
- * `time_bin_anchor`: a date/time/datetime value representing an anchor of a bin starts. The type of this argument should be the same as the first `time_to_bin` argument.
- * `duration_bin_size`: the duration value representing the size of the bin, in the type of year_month_duration or day_time_duration. The type of this duration should be compatible with the type of `time_to_bin`, so that the arithmetic operation between `time_to_bin` and `duration_bin_size` is well_defined. Currently AsterixDB supports the following arithmetic operations:
- * datetime +|_ year_month_duration
- * datetime +|_ day_time_duration
- * date +|_ year_month_duration
- * date +|_ day_time_duration
- * time +|_ day_time_duration
+ * Returns an ordered list of `interval` values representing each bin that is overlapping the `interval`.
+ * Arguments:
+ * `interval`: an `interval` value
+ * `time_bin_anchor`: a date/time/datetime value representing an anchor of a bin starts. The type of this argument should be the same as the first `time_to_bin` argument.
+ * `duration_bin_size`: the duration value representing the size of the bin, in the type of year_month_duration or day_time_duration. The type of this duration should be compatible with the type of `time_to_bin`, so that the arithmetic operation between `time_to_bin` and `duration_bin_size` is well_defined. Currently AsterixDB supports the following arithmetic operations:
+ * datetime +|_ year_month_duration
+ * datetime +|_ day_time_duration
+ * date +|_ year_month_duration
+ * date +|_ day_time_duration
+ * time +|_ day_time_duration
* Return Value:
* a ordered list of `interval` values representing each bin that is overlapping the `interval`.
Note that the internal type as `time_to_bin` and `duration_bin_size`.
@@ -779,25 +932,25 @@
datetime("1900-01-01T00:00:00.000"), year_month_duration("P100Y"))
};
- * The expected result is:
+ * The expected result is:
- {
- "timebins": [
- interval(time("17:00:00.000Z"), time("17:30:00.000Z")),
- interval(time("17:30:00.000Z"), time("18:00:00.000Z")),
- interval(time("18:00:00.000Z"), time("18:30:00.000Z")),
- interval(time("18:30:00.000Z"), time("19:00:00.000Z"))
- ],
- "datebins": [
- interval(date("1980-01-01"), date("1990-01-01")),
- interval(date("1990-01-01"), date("2000-01-01")),
- interval(date("2000-01-01"), date("2010-01-01")),
- interval(date("2010-01-01"), date("2020-01-01"))
- ],
- "datetimebins": [
- interval(datetime("1800-01-01T00:00:00.000Z"), datetime("1900-01-01T00:00:00.000Z")),
- interval(datetime("1900-01-01T00:00:00.000Z"), datetime("2000-01-01T00:00:00.000Z")),
- interval(datetime("2000-01-01T00:00:00.000Z"), datetime("2100-01-01T00:00:00.000Z"))
- ]
- };
-
+ {
+ "timebins": [
+ interval(time("17:00:00.000"), time("17:30:00.000")),
+ interval(time("17:30:00.000"), time("18:00:00.000")),
+ interval(time("18:00:00.000"), time("18:30:00.000")),
+ interval(time("18:30:00.000"), time("19:00:00.000"))
+ ],
+ "datebins": [
+ interval(date("1980-01-01"), date("1990-01-01")),
+ interval(date("1990-01-01"), date("2000-01-01")),
+ interval(date("2000-01-01"), date("2010-01-01")),
+ interval(date("2010-01-01"), date("2020-01-01"))
+ ],
+ "datetimebins":
+ [
+ interval(datetime("1800-01-01T00:00:00.000"), datetime("1900-01-01T00:00:00.000")),
+ interval(datetime("1900-01-01T00:00:00.000"), datetime("2000-01-01T00:00:00.000")),
+ interval(datetime("2000-01-01T00:00:00.000"), datetime("2100-01-01T00:00:00.000"))
+ ]
+ };
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
index 47ed4e7..01ea63c 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
@@ -81,6 +81,7 @@
* [Create Index](#Indices)
* [Create Synonym](#Synonyms)
* [Create Function](#Create_function)
+ * [Create View](#Create_view)
* [Drop Statement](#Removal)
* [Load Statement](#Load_statement)
* [Modification Statements](#Modification_statements)
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
index de7f442..46ff6e7 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
@@ -53,7 +53,7 @@
| || | String concatenation |
| IS NULL, IS NOT NULL, IS MISSING, IS NOT MISSING, <br/>IS UNKNOWN, IS NOT UNKNOWN, IS VALUED, IS NOT VALUED | Unknown value comparison |
| BETWEEN, NOT BETWEEN | Range comparison (inclusive on both sides) |
-| =, !=, <>, <, >, <=, >=, LIKE, NOT LIKE, IN, NOT IN | Comparison |
+| =, !=, <>, <, >, <=, >=, LIKE, NOT LIKE, IN, NOT IN, IS DISTINCT FROM, IS NOT DISTINCT FROM | Comparison |
| NOT | Logical negation |
| AND | Conjunction |
| OR | Disjunction |
@@ -130,6 +130,8 @@
| >= | Greater than or equal to | FROM customers AS c <br/> WHERE c.rating >= 640 <br/> SELECT *; |
| LIKE | Test if the left side matches a pattern defined on the right side; in the pattern, "%" matches any string while "_" matches any character. | FROM customers AS c WHERE c.name LIKE "%Dodge%" SELECT *;|
| NOT LIKE | Test if the left side does not match a pattern defined on the right side; in the pattern, "%" matches any string while "_" matches any character. | FROM customers AS c WHERE c.name NOT LIKE "%Dodge%" SELECT *;|
+| IS DISTINCT FROM | Inequality test that that treats NULL values as equal to each other and MISSING values as equal to each other | FROM orders AS o <br/> WHERE o.order_date IS DISTINCT FROM o.ship_date <br/> SELECT *; | |
+| IS NOT DISTINCT FROM | Equality test that treats NULL values as equal to each other and MISSING values as equal to each other | FROM orders AS o <br/> WHERE o.order_date IS NOT DISTINCT FROM o.ship_date <br/> SELECT *; |
The following table summarizes how the missing value comparison operators work.
@@ -191,7 +193,7 @@
The following pair of examples illustrate the use of a quantified expression to test that every (or some) element in the
set [1, 2, 3] of integers is less than three. The first example yields `FALSE` and second example yields `TRUE`.
-It is useful to note that if the set were instead the empty set, the first expression would yield `TRUE` ("every" value in an empty set satisfies the condition) while the second expression would yield `FALSE` (since there isn't "some" value, as there are no values in the set, that satisfies the condition).
+It is useful to note that if the set were instead the empty set, the first expression would yield `TRUE` ("every" value in an empty set satisfies the condition) while the second expression would yield `FALSE` (since there isn't "some" value, as there are no values in the set, that satisfies the condition). To express a universal predicate that yields `FALSE` with the empty set, we would use the quantifier `SOME AND EVERY` in lieu of `EVERY`.
A quantified expression will return a `NULL` (or `MISSING`) if the first expression in it evaluates to `NULL` (or `MISSING`).
Otherwise, a type error will be raised if the first expression in a quantified expression does not return a collection.
@@ -406,6 +408,9 @@
##### ArrayConstructor
data:image/s3,"s3://crabby-images/525a9/525a96212a27bc76d191d45a16e8b96300cc6d1a" alt=""
+##### ParenthesizedArrayConstructor
+data:image/s3,"s3://crabby-images/33f4c/33f4c98610de4aee5dd1ab7cd6626f56a158f490" alt=""
+
##### MultisetConstructor
data:image/s3,"s3://crabby-images/ea916/ea9167b39d5b8b8430b6797a34e57bf67c6f7bb9" alt=""
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
index 3e70922..04a65d1 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/3_query.md
@@ -742,7 +742,7 @@
WHERE get_year(date(o.order_date)) = 2020
GROUP BY get_month(date(o.order_date)) AS month
SELECT month, COUNT(*) AS order_count
- ORDER BY order_count desc
+ ORDER BY order_count DESC, month DESC
LIMIT 3;
Result:
@@ -794,6 +794,327 @@
}
]
+#### <a id="Rollup">ROLLUP</a>
+
+The `ROLLUP` subclause is an aggregation feature that extends the functionality of the `GROUP BY` clause.
+It returns extra _super-aggregate_ items in the query results, giving subtotals and a grand total for the aggregate
+functions in the query.
+To illustrate, first consider the following query.
+
+##### Example
+
+(Q3.R1) List the number of orders, grouped by customer region and city.
+
+ SELECT customer_region AS Region,
+ customer_city AS City,
+ COUNT(o.orderno) AS `Order Count`
+ FROM customers AS c LEFT OUTER JOIN orders AS o ON c.custid = o.custid
+ LET address_line = SPLIT(c.address.city, ","),
+ customer_city = TRIM(address_line[0]),
+ customer_region = TRIM(address_line[1])
+ GROUP BY customer_region, customer_city
+ ORDER BY customer_region ASC, customer_city ASC, `Order Count` DESC;
+
+Result:
+
+ [
+ {
+ "Region": "Italy",
+ "City": "Rome",
+ "Order Count": 0
+ },
+ {
+ "Region": "MA",
+ "City": "Boston",
+ "Order Count": 2
+ },
+ {
+ "Region": "MA",
+ "City": "Hanover",
+ "Order Count": 0
+ },
+ {
+ "Region": "MO",
+ "City": "St. Louis",
+ "Order Count": 7
+ }
+ ]
+
+This query uses string functions to split each customer's address into city and region.
+The query then counts the total number of orders placed by each customer, and groups the results first by customer
+region, then by customer city.
+The aggregate results (labeled `Order Count`) are only shown by city, and there are no subtotals or grand total.
+We can add these using the `ROLLUP` subclause, as in the following example.
+
+##### Example
+
+(Q3.R2) List the number of orders by customer region and city, including subtotals and a grand total.
+
+ SELECT customer_region AS Region,
+ customer_city AS City,
+ COUNT(o.orderno) AS `Order Count`
+ FROM customers AS c LEFT OUTER JOIN orders AS o ON c.custid = o.custid
+ LET address_line = SPLIT(c.address.city, ","),
+ customer_city = TRIM(address_line[0]),
+ customer_region = TRIM(address_line[1])
+ GROUP BY ROLLUP(customer_region, customer_city)
+ ORDER BY customer_region ASC, customer_city ASC, `Order Count` DESC;
+
+Result:
+
+ [
+ {
+ "Region": null,
+ "City": null,
+ "Order Count": 9
+ },
+ {
+ "Region": "Italy",
+ "City": null,
+ "Order Count": 0
+ },
+ {
+ "Region": "Italy",
+ "City": "Rome",
+ "Order Count": 0
+ },
+ {
+ "Region": "MA",
+ "City": null,
+ "Order Count": 2
+ },
+ {
+ "Region": "MA",
+ "City": "Boston",
+ "Order Count": 2
+ },
+ {
+ "Region": "MA",
+ "City": "Hanover",
+ "Order Count": 0
+ },
+ {
+ "Region": "MO",
+ "City": null,
+ "Order Count": 7
+ },
+ {
+ "Region": "MO",
+ "City": "St. Louis",
+ "Order Count": 7
+ }
+ ]
+
+With the addition of the `ROLLUP` subclause, the results now include an extra item at the start of each region,
+giving the subtotal for that region.
+There is also another extra item at the very start of the results, giving the grand total for all regions.
+
+The order of the fields specified by the `ROLLUP` subclause determines the hierarchy of the super-aggregate items.
+The customer region is specified first, followed by the customer city; so the results are aggregated by region first,
+and then by city within each region.
+
+The grand total returns `null` as a value for the city and the region, and the subtotals return `null` as the
+value for the city, which may make the results hard to understand at first glance.
+A workaround for this is given in the next example.
+
+##### Example
+
+(Q3.R3) List the number of orders by customer region and city, with meaningful subtotals and grand total.
+
+ SELECT IFNULL(customer_region, "All regions") AS Region,
+ IFNULL(customer_city, "All cities") AS City,
+ COUNT(o.orderno) AS `Order Count`
+ FROM customers AS c LEFT OUTER JOIN orders AS o ON c.custid = o.custid
+ LET address_line = SPLIT(c.address.city, ","),
+ customer_city = TRIM(address_line[0]),
+ customer_region = TRIM(address_line[1])
+ GROUP BY ROLLUP(customer_region, customer_city)
+ ORDER BY customer_region ASC, customer_city ASC, `Order Count` DESC;
+
+Result:
+
+ [
+ {
+ "Region": "All regions",
+ "City": "All cities",
+ "Order Count": 9
+ },
+ {
+ "Region": "Italy",
+ "City": "All cities",
+ "Order Count": 0
+ },
+ {
+ "Region": "Italy",
+ "City": "Rome",
+ "Order Count": 0
+ },
+ {
+ "Region": "MA",
+ "City": "All cities",
+ "Order Count": 2
+ },
+ {
+ "Region": "MA",
+ "City": "Boston",
+ "Order Count": 2
+ },
+ {
+ "Region": "MA",
+ "City": "Hanover",
+ "Order Count": 0
+ },
+ {
+ "Region": "MO",
+ "City": "All cities",
+ "Order Count": 7
+ },
+ {
+ "Region": "MO",
+ "City": "St. Louis",
+ "Order Count": 7
+ }
+ ]
+
+This query uses the `IFNULL` function to populate the region and city fields with meaningful values for the
+super-aggregate items.
+This makes the results clearer and more readable.
+
+#### <a id="Cube">CUBE</a>
+
+The `CUBE` subclause is similar to the `ROLLUP` subclause, in that it returns extra super-aggregate items in the query
+results, giving subtotals and a grand total for the aggregate functions.
+Whereas `ROLLUP` returns a grand total and a hierarchy of subtotals based on the specified fields,
+the `CUBE` subclause returns a grand total and subtotals for every possible combination of the specified fields.
+
+The following example is a modification of Q3.R3 which illustrates the `CUBE` subclause.
+
+##### Example
+
+(Q3.C) List the number of orders by customer region and order date, with all possible subtotals and a grand total.
+
+ SELECT IFNULL(customer_region, "All regions") AS Region,
+ IFNULL(order_month, "All months") AS Month,
+ COUNT(o.orderno) AS `Order Count`
+ FROM customers AS c INNER JOIN orders AS o ON c.custid = o.custid
+ LET address_line = SPLIT(c.address.city, ","),
+ customer_region = TRIM(address_line[1]),
+ order_month = get_month(date(o.order_date))
+ GROUP BY CUBE(customer_region, order_month)
+ ORDER BY customer_region ASC, order_month ASC;
+
+Result:
+
+ [
+ {
+ "Region": "All regions",
+ "Order Count": 9,
+ "Month": "All months"
+ },
+ {
+ "Region": "All regions",
+ "Order Count": 1,
+ "Month": 4
+ },
+ {
+ "Region": "All regions",
+ "Order Count": 1,
+ "Month": 5
+ },
+ {
+ "Region": "All regions",
+ "Order Count": 1,
+ "Month": 6
+ },
+ {
+ "Region": "All regions",
+ "Order Count": 1,
+ "Month": 7
+ },
+ {
+ "Region": "All regions",
+ "Order Count": 1,
+ "Month": 8
+ },
+ {
+ "Region": "All regions",
+ "Order Count": 2,
+ "Month": 9
+ },
+ {
+ "Region": "All regions",
+ "Order Count": 2,
+ "Month": 10
+ },
+ {
+ "Region": "MA",
+ "Order Count": 2,
+ "Month": "All months"
+ },
+ {
+ "Region": "MA",
+ "Order Count": 1,
+ "Month": 7
+ },
+ {
+ "Region": "MA",
+ "Order Count": 1,
+ "Month": 8
+ },
+ {
+ "Region": "MO",
+ "Order Count": 7,
+ "Month": "All months"
+ },
+ {
+ "Region": "MO",
+ "Order Count": 1,
+ "Month": 4
+ },
+ {
+ "Region": "MO",
+ "Order Count": 1,
+ "Month": 5
+ },
+ {
+ "Region": "MO",
+ "Order Count": 1,
+ "Month": 6
+ },
+ {
+ "Region": "MO",
+ "Order Count": 2,
+ "Month": 9
+ },
+ {
+ "Region": "MO",
+ "Order Count": 2,
+ "Month": 10
+ }
+ ]
+
+To simplify the results, this query uses an inner join, so that customers who have not placed an order are not included
+in the totals.
+The query uses string functions to extract the region from each customer's address,
+and a temporal function to extract the year from the order date.
+
+The query uses the `CUBE` subclause with customer region and order month.
+This means that there are four possible aggregates to calculate:
+
+* All regions, all months
+* All regions, each month
+* Each region, all months
+* Each region, each month
+
+The results start with the grand total, showing the total number of orders across all regions for all months.
+This is followed by date subtotals, showing the number of orders across all regions for each month.
+Following that are the regional subtotals, showing the total number of orders for all months in each region;
+and the result items, giving the number of orders for each month in each region.
+
+The query also uses the `IFNULL` function to populate the region and date fields with meaningful values for the
+super-aggregate items.
+This makes the results clearer and more readable.
+
### <a id="Having_clauses">HAVING Clause</a>
##### HavingClause
@@ -1220,9 +1541,12 @@
The last three (optional) clauses to be processed in a query are `ORDER BY`, `LIMIT`, and `OFFSET`.
The `ORDER BY` clause is used to globally sort data in either ascending order (i.e., `ASC`) or descending order (i.e., `DESC`).
-During ordering, `MISSING` and `NULL` are treated as being smaller than any other value if they are encountered
+During ordering (if the `NULLS` modifier is not specified), `MISSING` and `NULL` are treated as being smaller than any other value if they are encountered
in the ordering key(s). `MISSING` is treated as smaller than `NULL` if both occur in the data being sorted.
-The ordering of values of a given type is consistent with its type's `<=` ordering; the ordering of values across types is implementation-defined but stable.
+The `NULLS` modifier determines how `MISSING` and `NULL` are ordered relative to all other values:
+first (`NULLS` `FIRST`) or last (`NULLS` `LAST`). The relative order between `MISSING` and `NULL` is not affected by the `NULLS` modifier
+(i.e. `MISSING` is still treated as smaller than `NULL`). The ordering of values of a given type is consistent with its type's `<=` ordering;
+the ordering of values across types is implementation-defined but stable.
The `LIMIT` clause is used to limit the result set to a specified maximum size.
The optional `OFFSET` clause is used to specify a number of items in the output stream to be discarded before the query result begins.
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
index 1569d69..4ec5033 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
@@ -339,6 +339,13 @@
CREATE INDEX cCustIdx ON orders(custid) TYPE BTREE;
+The following example creates a btree index called `oCNameIdx` on the `cname` field of the orders dataset, but does not insert `NULL` and `MISSING` values into the index.
+By default, if `INCLUDE/EXCLUDE UNKNOWN KEY` is not specified, unknown values will be inserted into btree indexes.
+
+##### Example
+
+ CREATE INDEX oCNametIdx ON orders(cname) EXCLUDE UNKNOWN KEY;
+
The following example creates an open btree index called `oCreatedTimeIdx` on the (non-declared) `createdTime` field of the `orders` dataset having `datetime` type.
This index can be useful for accelerating exact-match queries, range search queries, and joins involving the `createdTime` field.
The index is enforced so that records that do not have the `createdTime` field or have a mismatched type on the field
@@ -370,11 +377,11 @@
The following example creates an array index called `oItemsPriceIdx` on the `price` field inside the `items` array of the `orders` dataset.
This index can be useful for accelerating membership queries, existential or universal quantification queries, or joins involving the `price` field inside this array.
-(To enable array index query optimization, be sure to set the [`arrayindex` compiler option](manual.html#ArrayIndexFlag).)
+Unknown values cannot currently be stored inside array indexes, so `EXCLUDE UNKNOWN KEY` must be specified.
#### Example
- CREATE INDEX oItemsPriceIdx ON orders(UNNEST items SELECT price);
+ CREATE INDEX oItemsPriceIdx ON orders(UNNEST items SELECT price) EXCLUDE UNKNOWN KEY;
The following example creates an open rtree index called `oOrderLocIdx` on the order-location field of the `orders` dataset. This index can be useful for accelerating queries that use the [`spatial-intersect` function](builtins.html#spatial_intersect) in a predicate involving the sender-location field.
@@ -451,9 +458,10 @@
##### CreateSynonym
data:image/s3,"s3://crabby-images/b79f9/b79f937bf5fab5077d654773d732b308116c1bfa" alt=""
-The `CREATE SYNONYM` statement creates a synonym for a given dataset.
-This synonym may be used instead of the dataset name in `SELECT`, `INSERT`, `UPSERT`, `DELETE`, and `LOAD` statements.
-The target dataset does not need to exist when the synonym is created.
+The `CREATE SYNONYM` statement creates a synonym for a given dataset or a view.
+This synonym may be used instead of the dataset name in `SELECT`, `INSERT`, `UPSERT`, `DELETE`, and `LOAD` statements,
+or instead of the view name in `SELECT` statements.
+The target dataset or view does not need to exist when the synonym is created.
A synonym may be created for another synonym.
##### Example
@@ -519,6 +527,25 @@
CREATE FUNCTION sentiment(a) AS "sentiment_mod", "sent_model.sentiment" AT pylib;
+#### <a id="Create_view">Create View</a>
+
+The `CREATE VIEW` statement creates a **named** view that can then be used in queries.
+The body of a view can be any `SELECT` statement.
+
+##### CreateView
+data:image/s3,"s3://crabby-images/f6c80/f6c80e17c5408418299c8344445ebb68b3ea67ca" alt=""
+
+##### Example
+
+ CREATE DATASET customers(customersType) PRIMARY KEY custid;
+
+ CREATE VIEW customersView AS
+ SELECT c.custid, c.name
+ FROM customers AS c
+ WHERE c.address.city = "Boston, MA";
+
+ SELECT * FROM customersView;
+
### <a id="Removal">Drop Statement</a>
##### DropStmnt
@@ -555,6 +582,8 @@
DROP SYNONYM customersSynonym;
+ DROP VIEW customersView;
+
DROP DATAVERSE CommerceData;
When an artifact is dropped, it will be droppped from the current dataverse if none is specified
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md
index ee90efb..c3c13fc 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md
@@ -18,15 +18,15 @@
!-->
## <a id="ArrayIndexFlag">Controlling Array-Index Access Method Plan Parameter</a>
-By default, the system does not attempt to utilize array indexes as an access method (even if an array index is present and is applicable).
-If you believe that your query will benefit from an array index, toggle the parameter below.
+By default, the system attempts to utilize array indexes as an access method if an array index is present and is applicable.
+If you believe that your query will not benefit from an array index, toggle the parameter below.
-* **compiler.arrayindex**: if this is set to true, array indexes will be considered as an access method for applicable queries; the default value is false.
+* **compiler.arrayindex**: if this is set to true, array indexes will be considered as an access method for applicable queries; the default value is true.
#### Example
- set `compiler.arrayindex` "true";
+ set `compiler.arrayindex` "false";
SELECT o.orderno
FROM orders o
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md
index a4acbb0..4393f5e 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_3_resolution.md
@@ -215,7 +215,7 @@
The rules for resolving the leftmost identifier are:
1. _In a `FROM` clause_: Names in a `FROM` clause identify the collections over which the query block will iterate.
- These collections may be stored datasets or may be the results of nested query blocks.
+ These collections may be stored datasets, views, synonyms, or may be the results of nested query blocks.
A stored dataset may be in a named dataverse or in the default dataverse.
Thus, if the two-part name `a.b` is in a `FROM` clause, a might represent a dataverse and `b` might represent a dataset in that dataverse.
Another example of a two-part name in a `FROM` clause is `FROM orders AS o, o.items AS i`.
@@ -229,12 +229,13 @@
- (1B): Otherwise, if the identifier is the first part of a two-part name like `a.b`, the name is treated as `dataverse.dataset`.
If the identifier stands alone as a one-part name, it is treated as the name of a dataset in the default dataverse.
If the designated dataset exists then the identifier is resolved to that dataset,
- otherwise if a synonym with given name exists then the identifier is resolved to the target dataset of that
+ othwerise if a view with given name exists then the identifier is resolved to that view,
+ otherwise if a synonym with given name exists then the identifier is resolved to the target dataset or the target view of that
synonym (potentially recursively if this synonym points to another synonym). An error will result if the designated
- dataset or a synonym with this name does not exist.
+ dataset, view, or a synonym with this name does not exist.
- Datasets take precedence over synonyms, so if both a dataset and a synonym have the same name then the
- resolution is to the dataset.
+ Datasets and views take precedence over synonyms, so if both a dataset (or a view) and a synonym have the same name then the
+ resolution is to the dataset. Note that there cannot be a dataset and a view with the same name.
2. _Elsewhere in a query block_: In clauses other than `FROM`, a name typically identifies a field of some object.
For example, if the expression `a.b` is in a `SELECT` or `WHERE` clause, it's likely that `a` represents an object and `b` represents a field in that object.
diff --git a/asterixdb/asterix-doc/src/main/spatial_join/spatial_join.md b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join.md
new file mode 100644
index 0000000..ea89ef2
--- /dev/null
+++ b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join.md
@@ -0,0 +1,65 @@
+<!--
+ ! 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.
+ !-->
+
+## <a id="spatial_joins">Spatial Joins</a>
+AsterixDB supports efficient spatial join query performance.
+In particular, it will execute the [Partition Based Spatial-Merge Join](http://pages.cs.wisc.edu/~dewitt/includes/paradise/spjoin.pdf).
+(PBSM) algorithm for the join queries with a spatial function in join condition.
+
+Supported spatial functions:
+- `spatial_intersect(ARectangle, ARectangle)`.
+- ESRI's spatial functions: `st_intersects()`, `st_overlaps()`, `st_touches()`, `st_contains()`, `st_crosses()`, `st_within()`, `st_distance()`.
+
+Once the join condition contains a supported spatial function, users do not need to do any further action to trigger this efficient query plan.
+
+### <a id="spatial_partitioning_hint">Using a spatial partitioning hint</a>
+PBSM algorithm requires the following information to partition data into a grid:
+- The MBR of two input datasets.
+- Number of rows and columns of the grid.
+
+By default, the MBR will be computed at running time and the grid size is 100x100.
+However, users can also set other values for these parameters using spatial partitioning hint.
+
+##### Spatial partitioning hint example
+In this example, assume that MBR of two input datasets is (-180.0, -83.0, 180.0, 90.0) and grid size is 10x10.
+
+
+ /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */
+
+
+##### Spatial partitioning hint in a query
+
+ DROP DATAVERSE test IF EXISTS;
+ CREATE DATAVERSE test;
+ USE test;
+
+ -- Make GeomType
+ CREATE TYPE GeomType as closed {
+ id: int32,
+ geom: rectangle
+ };
+
+ -- Make Park dataset
+ CREATE DATASET ParkSet (GeomType) primary key id;
+
+ -- Make Lake dataset
+ CREATE DATASET LakeSet (GeomType) primary key id;
+
+ SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+ WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
diff --git a/asterixdb/asterix-doc/src/main/spatial_join/spatial_join_title.md b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join_title.md
new file mode 100644
index 0000000..2edac18
--- /dev/null
+++ b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join_title.md
@@ -0,0 +1,24 @@
+<!--
+ ! 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.
+ !-->
+
+# Spatial Joins
+
+## <a id="#toc">Table of Contents</a> ##
+* [Introduction](#spatial_joins)
+* [Spatial partitioning hint](#spatial_partitioning_hint)
diff --git a/asterixdb/asterix-doc/src/site/markdown/api.md b/asterixdb/asterix-doc/src/site/markdown/api.md
index d39919f..81f0e3b 100644
--- a/asterixdb/asterix-doc/src/site/markdown/api.md
+++ b/asterixdb/asterix-doc/src/site/markdown/api.md
@@ -39,6 +39,7 @@
used e.g. to match individual requests, jobs, and responses. Another option could be to use it for groups of requests
if an application decides to put e.g. an group identifier into that field to route groups of responses to a
particular response processor.
+* `dataverse` - Default dataverse for this statement (Optional). If the specified dataverse does not exist then this setting is ignored.
* `mode` - Result delivery mode. Possible values are `immediate`, `deferred`, `async` (default: `immediate`).
If the delivery mode is `immediate` the query result is returned with the response.
If the delivery mode is `deferred` the response contains a handle to the <a href="#queryresult">result</a>.
diff --git a/asterixdb/asterix-doc/src/site/markdown/dashboard.md b/asterixdb/asterix-doc/src/site/markdown/dashboard.md
index 2d6a519..66c2632 100644
--- a/asterixdb/asterix-doc/src/site/markdown/dashboard.md
+++ b/asterixdb/asterix-doc/src/site/markdown/dashboard.md
@@ -30,60 +30,71 @@
## <a id="basics">Basic Usage</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
-Executing a query on this console is easy. First, select from the input options and then select your execution mode.
+Executing a query is easy. Type your query into the query input box, select your run configurations, and then click the
+green play button to run the query. The output will appear below in the “Output” section.
__Input Options__
-* `Dataverse` - the dataverse that the query will use. The default is the `Default` dataverse. This is not required to
-run a query and the console will try and autodetect the dataverse used in the query.
-* `Plan Format` - specifies what format of the resulting query plan.
- * `JSON` - results in the showing the interactive query plan viewer.
- * `STRING` - results in the text/string format of the query plan. Equivalent to the text format from the old 19001
- console.
-* `Output Format` - the format of the result of the query.
- * `JSON` - the default and will return the results in JSON. Can also view in `Tree` and `Table` views in the output
- section.
- * `CSV (no header)` - will return CSV format but without the header. Can only view this in `Table` view in the output
- section.
- * `CSV (header)` - will return CSV format with the header. Can only view this in `Table` view in the output
- section. See the [Exporting Data](#exporting) section for more information and examples.
+* `Dataverse` - the dataverse that the query will use. The default is the `Default` dataverse.
+This input is not required and can be autodetected by running the query (see Figure 1).
+* `Output Format` - specifies the format of the result of the query.
+ * `JSON` (default) - query results are returned in JSON. Viewable in “Tree” and “Table” mode.
+ * `CSV (no header)` - query results are returned in CSV format without the header. Viewable only in “Table” mode.
+ * `CSV (header)` - query results are returned in CSV format with the header. Viewable only in “Table” mode.
+ See the [Exporting Data](#exporting) section for more information and examples.
+* `Plan Format` - specifies the format of the query plan (if requested).
+ * `JSON` (default) - results in showing the interactive query plan viewer.
+ * `STRING` - results in the text / string format of the query plan. Equivalent to the text format from the legacy 19001 console.
-To execute the query click the green triangle in the bottom right. Users may also choose to click the `Explain` button.
-This will not actually run the query (no results returned) and will only return the query plan. The console will default
-the view in the output section to `Plan`.
+To execute the query, click the green triangle in the bottom right of the query input section.
+Users may also choose to click the “Explain” button. This option will not actually run the query.
+It will return only the query plan instead. The console will default the view in the output section to “Plan” as well.
-To cancel the query click the red stop button in the bottom right. This will send a `DELETE` request to the server and cancel the previous
-request.
+To cancel the query, click the red stop button in the bottom right of the query input section.
+This will send a “DELETE” request to the server and cancel the previous request.
-The dashboard now supports autocomplete for SQL++ keywords. Use `CTRL+Space` to activate the autocomplete.
+data:image/s3,"s3://crabby-images/2995e/2995e6f5fbca2bb20944f04703999f18ef2e0bb6" alt="Figure 1: Input Component"\
+Figure 1: The input component of the console
+
+The dashboard also now supports autocomplete of SQL++ keywords. Use `CTRL+Space` to activate the autocomplete feature (see Figure 2).
+
+data:image/s3,"s3://crabby-images/d1ce2/d1ce2672cf77ed2f1b34c7fc55242f2093e01c25" alt="Figure 2: Autocomplete"\
+Figure 2: Example of autocomplete
## <a id="qnav">Query Navigation</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
-This console supports query history and has two different ways of navigating the query history. On the input bar there is a section
-for `QUERY HISTORY` and there are also two arrows `<` and `>`.
+This console supports query history and has two different ways of navigating the query history. On the input bar there
+is a section for `QUERY HISTORY` and there are also two arrows (`<` and `>`).
-Utilizing the arrows will let you traverse the queries one by one. However, if the console is already at the most recent query
-in the history and the user clicks the `>` or forward arrow, it will create a new empty query.
+Utilizing the arrows will let you traverse all previously run queries one by one. However, if the console is already at
+the most recent query in the history and the user clicks the `>` or forward arrow, it will create a new empty query.
-The `QUERY HISTORY` dropdown allows users to jump through the history without having to step through it with the arrows.
+The `QUERY HISTORY` drop down allows users to jump to a specific query in the history without having to step through it with the arrows.
-When executing a query, this query will be counted as a new query if it is different (purely the text of the query, not
-the results) from the most recent query. It will subsequently be added to the query history.
+When executing a query, this query will be counted as a new query if it is different
+(comparison is purely based on the query text, not the results) from the most recent query.
+It will subsequently be added to the front of the query history.
## <a id="metadatainspector">Metadata Inspector</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
-The metadata inspector is the column on the rightside of the console. The `Refresh` button is used to update the current metadata.
-When a user creates or drops a Dataverse, Dataset, Datatype, or Index the changes will not be automatically reflected. User must
-click the `Refresh` button to get the most up to date data.
+The metadata inspector is the column on the right side of the console (see Figure 3). The `Refresh` button is used to
+update the current metadata. When a user creates or drops a dataverse, dataset, datatype, user-defined function, or index,
+the changes will not be automatically reflected. The user must click the `Refresh` button to get the most up to date metadata.
+
+data:image/s3,"s3://crabby-images/9c4db/9c4db405db33ac94f92f8307d60899da21e94281" alt="Figure 3: Metadata Inspector"\
+Figure 3: The metadata inspector
The console supports multiple dialogs/windows open at once. All of these are resizable and draggable as well.
-Users can also click the `JSON` / `SUMMARY` button to toggle from the raw and parsed views. `SUMMARY` is the default.
+Users can also click the `JSON` / `SUMMARY` button in the dialog windows to toggle between the raw and parsed views of
+the metadata objects. `SUMMARY` is the default.
#### Dataverse
-Clicking a dataverse will add it to the shown metadata in this inspector. Users can select as many dataverses as desired.
-The corresponding datasets, datatypes, and indices will appear.
+Clicking a dataverse will add it to the shown metadata in the inspector. Users can easily see which dataverses are
+currently being shown by the check mark to the left of the dataverse name. Users can select as many dataverses as desired.
+The datasets, datatypes, indices, and user-defined functions that are contained in the selected dataverses will appear
+in their corresponding sections.
#### Datasets
@@ -94,20 +105,21 @@
* `Datatype Name` - the name of the datatype of the dataset.
* `Primary Keys` - the primary keys of the dataset.
* `Sample` - if there is data inserted into the dataset, this is a section where viewers can see a sample of the dataset.
-It is a `SELECT * FROM {dataset} LIMIT 1` query.
+It is equivalent to the user querying: `SELECT * FROM {dataset} LIMIT 1`.
#### Datatypes
-Clicking on a datatypes will open a draggable and expandable window that contains information about the datatype. This console
-does support nested datatypes.
+Clicking on a datatype will open a draggable and expandable window that contains information about the datatype.
+The console includes support for nested datatypes.
* `Dataverse` - which dataverse the datatype belongs to.
* `Datatype Name` - the name of the datatype.
-* `Fields` - a list of the fields in the dataset. Each field has information on whether it is nullable or required. If the
-field is nested / not a primitive type, click on it to see the information of that type. If the field is wrapped in `[ ]` or `{{ }}`,
-then it is an ordered list or unordered of that type respectively. If a field is italicized, it is an anonymous type.
+* `Fields` - a list of the fields in the dataset. Each field has information on whether it is nullable or required. If
+the field is nested (not a primitive type), click on it to see the information about that type. If the field is wrapped
+in `[...]` or `{{...}}`, then it is an ordered list or unordered list respectively. If a field is italicized, it means
+it is an anonymous type.
-NOTE: the `JSON` view does not support nested like `SUMMARY` does.
+NOTE: the `JSON` view does not support nested like the `SUMMARY` view does.
#### Index
@@ -115,85 +127,106 @@
* `Dataverse` - which dataverse the index belongs to.
* `Index Name` - the name of the index.
-* `Index Type` - the type of the index.
+* `Index Type` - the type of the index (primary or not primary).
* `Search Key(s)` - the key(s) of the index.
+#### User-Defined Functions
+Clicking on an user-defined function will open a draggable and expandable window that contains information about the
+user-defined function.
+
+* `Dataverse` - which dataverse the user defined function (UDF) belongs to
+* `Function Name` - the name of the UDF
+* `Arity` - the number of parameters of the UDF
+* `Parameters` - the name of the parameters
+* `Definition` - the definition of the UDF
## <a id="planviewer">Interactive Plan Viewer</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
-To view the interactive plan viewer, execute a query and switch to the `PLAN` tab in the output section. Alternatively,
-users can click explain the query by clicking `EXPLAIN` instead of execute and the output section will default to the
-`PLAN` tab.
+To view the interactive plan viewer, either execute a query and switch to the `PLAN` tab in the output section or `EXPLAIN`
+the query and it will default to the `PLAN` view.
-To interact with the query plan, drag to move the view of the graph. Use the scroll wheel or scroll movement to zoom in and out
-of the plan.
+To interact with the query plan, drag to move around the graph. Users can also choose to utilize the scroll function to
+zoom in and out of the plan. To the left of the plan will be the `Plan Explorer` (see Figure 5). This is the main way that users will
+be able to interact with the plan.
-The default plan orientation is `Bottom to Top` and can be swapped for `Top to Bottom` if so desired.
+The plan orientation can be changed with the `Plan Explorer` under the `View` title. The default is
+`bottom to top`, but it can be swapped to `top to bottom` if desired.
-The default view of the plan is not detailed (just operator IDs and operator names). To look at a more detailed plan, check
-the `Detailed` checkbox and the plan will reload with more detail per node.
+The default view of the plan is not detailed (just showing operator IDs and operator names). To look at a more detailed
+view of the plan, check the `Detailed` checkbox in the plan explorer and the plan will reload with more detail per node.
+
+data:image/s3,"s3://crabby-images/5f34b/5f34b144e8afd081ac3b657e7d6b044e4d86d602" alt="Figure 4: Plan Explorer"\
+Figure 5: The plan explorer
#### Traversing
-There are multiple ways to traverse the query plan. the `Go to Node` dropdown will keep track of the currently selected
-node. Using the arrows next to the `Go to Node` dropdown will traverse the plan node by node in a Depth First Search (DFS)
-fashion. Selecting nodes on the `Go to Node` dropdown will jump the plan to the selected node.
+There are two main ways to traverse the query plan. The dropdown under `Node` will display the node the user is currently at.
+This dropdown will keep track of the node the user is at throughout.
-Utilizing both the arrows and the `Go to Node` dropdown, it is easy to trace through a plan.
+The arrows next to the dropdown can be used to step through the plan node by node in a Depth First Search (DFS) fashion.
-#### Search (Detailed mode only)
+Selecting a node from the dropdown will jump the viewer to the node in the plan that was selected.
-The `Search` function appears when the plan is in `Detailed` mode. Search for specific string occurrences in the plan. When
-the search icon is clicked, the first mathc will be selected (if there is a match). Use the arrows that appear next to it
-to iterate through every match.
-
-Must click `Clear Selections` after done with the search.
-
-Unfortunately, at this time regular expression search is not supported.
+Utilizing both the dropdown and arrows, it is easy to trace through an entire plan.
#### Variables (Detailed mode only)
-The `See Variable Occurences` dropdown will appear when the plan is in `Detailed` mode. Users can select from any variable
-that appears in the plan. Selecting a variable will jump to the node of last occurrence. The user can see how many occurence there
-are by the `See Variable Occurences` dropdown title (it will now include a fraction).
+Under the `Variable` section of the `Plan Explorer`, there is a dropdown that will contain all the variables that occur
+in the plan. Selecting a variable there will jump the plan viewer to the node that contains the last (top-most in the plan)
+occurrence of that variable. The user can see how many occurrences there are via the `Variable Occurrences` title above the dropdown.
+The arrows to the right can be used to step through the occurrences.
-The arrows that appear can iterate through the occurences.
+To skip to the variable’s declaration, click the `DECLARATION` button. This will jump the plan viewer to the node of the
+very first occurrence of that variable. To get back to the previous node, click `BACK`.
-Often, it is useful to be able to skip right to the declaration of a variable. By clicking on the skip button, the plan
-will select the node where that variable was declared. To jump back to whatever node before, click the undo button.
+#### Search (Detailed mode only)
-#### Clear Selections
+Use the `Search` bar to type in a string of interest. The plan explorer will search the plan for that specific string in
+each node. The number of matches will be displayed in the title above the search bar. Users can use the arrows to the
+right to step through the nodes that matched.
-Clicking `Clear Selections` will reset the graph and focus on the first node in the plan.
+Users must click `Clear` after finishing with a search.
+
+Unfortunately, at this time, regular expression search is not supported.
+
+#### Clear
+
+Clicking `Clear` will reset the query plan graph and focus the viewer on the first node in the plan.
## <a id="exporting">Exporting Data</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
+Exporting data is a key part of the console’s functionality. Users can select from JSON, JSONL, CSV (no header),
+and CSV (header) as data types for the output.
+
### JSON/JSONL:
-1. Select `JSON` in the input `Output Format` option and run the query that you want to export.
-2. Click `Export` in the output section.
-3. Select between `JSON` and `JSONL` (JSON Lines). Adjust the filename to the desired name.
-4. Click `Export` to start the download.
+1. Select `JSON` on the input `Output Format` option and run the query that you want to export the results of
+2. Click `Export` in the output section
+3. Select between `JSON` and `JSONL` (JSON Lines) and adjust the filename to the desired name
+4. Click `Export` to start the download
### CSV (no header):
-1. Select `CSV (no header)` in the input `Output Format` option and run the query that you want to export.
-2. Click `Export` in the output section.
-3. Adjust the filename to the desired name.
-4. Click `Export` to start the download.
+1. Select `CSV (no header)` on the input `Output Format` option and run the query that you want to export the results of
+2. Click `Export` in the output section
+3. Adjust the filename to the desired name
+4. Click `Export` to start the download
### CSV (header):
-1. Create a type that supports the query you want to run.
-2. Set the `output-record-type` before your query
-3. Select `CSV (no header)` in the input `Output Format` option and run the query that you want to export.
-4. Click `Export` in the output section.
-5. Adjust the filename to the desired name.
-6. Click `Export` to start the download.
+1. Create a datatype that supports the query you want to run
+2. Set the `output-record-type` to this type before your query
+3. Select `CSV (no header)` on the input `Output Format` option and run the query that you want to export the results of
+4. Click `Export` in the output section
+5. Adjust the filename to the desired name
+6. Click `Export` to start the download
-This one is trickier. In order to get the header in the CSV format, we need to set the `output-record-type` in the query
-in order to get the headers. To explain further, here is an example using the TinySocial dataset from the Using SQL++ Primer.
+This one is clearly a little more involved. In order to get the desired header in the CSV format, it is necessary to
+set an `output-record-type` for the query. To better illustrate how to control this format, here is an example using the
+TinySocial dataset from the “Using SQL++” AsterixDB primer.
+
+For context, here’s the GleambookMessages DDL statement.
CREATE TYPE GleambookMessageType AS {
messageId: int,
@@ -206,8 +239,8 @@
CREATE DATASET GleambookMessages(GleambookMessageType)
PRIMARY KEY messageId;
-If we wanted to export `messageId`, `authorId`, and `senderLocation` in CSV format with headers, we would have to create
-an additional type to support this export.
+First, create the type of the expected output. If the goal is to export `messageId`, `authorId`, and `senderLocation` in
+CSV format with headers, create an additional type to support this export.
CREATE TYPE GleambookMessages_exportCSV AS {
messageId: int,
@@ -215,7 +248,7 @@
senderLocation: point
};
-The query would then look something like this:
+The query should then look something like this:
USE TinySocial;
@@ -224,8 +257,11 @@
SELECT messageId, authorId, senderLocation
FROM GleambookMessages;
-Now run the query with the `CSV (header)` input option and the result will contain the hedaer `messageId`, `authorId`,
-and `senderLocation`.
+Now run the query with the `CSV (header)` input option and the result will contain the header `messageId`, `authorId`,
+and `senderLocation` (see Figure 5).
+
+data:image/s3,"s3://crabby-images/030e7/030e71268770351d78985f334175aacb23255f92" alt="Figure 5: Sample CSV Header Output"\
+Figure 5: CSV (header) sample output
## <a id="development">Development</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
diff --git a/asterixdb/asterix-doc/src/site/markdown/geo/functions.md b/asterixdb/asterix-doc/src/site/markdown/geo/functions.md
new file mode 100644
index 0000000..2bd2250
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/markdown/geo/functions.md
@@ -0,0 +1,643 @@
+<!--
+ ! 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.
+ !-->
+
+# Support the standard GIS objects (DRAFT) #
+## <a id="toc">Table of Contents</a> ##
+
+* [Introduction](#Introduction)
+* [Construction functions](#construction)
+* [Primitive functions](#primitive)
+* [Spatial Predicate](#predicate)
+* [Spatial Analysis](#analysis)
+* [Spatial Aggregates](#aggregate)
+
+## <a id="Introduction">Introduction</a>
+
+To support standard GIS objects in AsterixDB, you need to use the `geometry` data type as follows.
+
+```
+DROP dataverse GeoJSON if exists;
+CREATE dataverse GeoJSON;
+
+USE GeoJSON;
+
+CREATE TYPE GeometryType AS{
+ id : int,
+ myGeometry : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
+```
+
+Please note that even though the [SRID](http://desktop.arcgis.com/en/arcmap/10.3/manage-data/using-sql-with-gdbs/what-is-an-srid.htm)
+input is supported for certain functions and is represented internally in the correct manner the serialized result (printed in the output) displays the SRID as 4326 always because of the limitations in Esri API.
+
+## <a id="construction">Construction functions</a>
+The Geometry datatype can be created by the constructor functions.
+
+### st_make_point ###
+* Creates a 2D,3DZ or 4D point geometry.
+
+* Example:
+ * Create a 2D point at coordinates (x,y) = (-71, 42)
+ * Command:
+
+ st_make_point(-71, 42);
+ * Result:
+
+ {"type":"Point","coordinates":[-71,42],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+* Example:
+ * Create a 3D point at coordinates (x,y,z) = (1,2,1.59)
+ * Command:
+
+ st_make_point(1,2,1.59);
+ * Result:
+
+ {"type":"Point","coordinates":[1,2,1.59],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_geom_from_text ###
+* Return a specified ST_Geometry value from Well-Known Text representation (WKT).
+
+* Example:
+ * Create a LineString geometry from the WKT format.
+ * Command:
+
+ st_geom_from_text("LINESTRING(1 2,3 4,5 6)");
+ * Result:
+
+ {"type":"LineString","coordinates":[[1,2],[3,4],[5,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+* Example:
+ * Create a MultiLineString geometry from the WKT format.
+ * Command:
+
+ st_geom_from_text('MULTILINESTRING((1 2,3 4,5 6),(7 8,9 10))');
+ * Result:
+
+ {"type":"MultiLineString","coordinates":[[[1,2],[3,4],[5,6]],[[7,8],[9,10]]],"crs":null}
+
+### st_geom_from_wkb ###
+* Creates a geometry instance from a Well-Known Binary geometry representation (WKB) and optional SRID.
+
+* Example:
+ * Command:
+
+ st_geom_from_wkb(hex("010100000000000000000000400000000000001440"));
+ * Result:
+
+ {"type":"Point","coordinates":[2,5],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+
+
+### st_geom_from_geojson
+* Creates a geometry instance from its GeoJSON representation
+
+* Example:
+ * Command:
+
+ st_geom_from_geojson({"type":"Point","coordinates":[2,5],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}});
+
+ * Result:
+
+ {"type":"Point","coordinates":[2,5],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_make_envelope ###
+* Creates a rectangular Polygon formed from the given minimums and maximums. Input values must be in SRS specified by the SRID.
+
+* Example:
+ * Command:
+
+ st_make_envelope(10, 10, 11, 11, 4326);
+ * Result:
+
+ {"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+## <a id="primitive">Primitive functions</a>
+There are primitive functions that take as input geometry/es and return a primitive type.
+
+### st_area ###
+* Return the area of the surface if it is a Polygon or MultiPolygon. For geometry, a 2D Cartesian area is determined with units specified by the SRID. For geography, area is determined on a curved surface with units in square meters.
+
+* Example:
+ * Command:
+
+ st_area(st_geom_from_text('POLYGON((7 2,4 9,3 6,2.6 7,8 16))'));
+ * Result:
+
+ 26.500000000000007
+
+### st_coord_dim ###
+* Return the coordinate dimension of the Geometry value.
+
+* Example:
+ * Command:
+
+ st_coord_dim(st_make_point(1,2));
+ * Result:
+
+ 2
+
+### st_dimension ###
+* Return the inherent dimension of this Geometry object, which must be less than or equal to the coordinate dimension.
+
+* Example:
+ * Command:
+
+ st_dimension(st_geom_from_text('GEOMETRYCOLLECTION(LINESTRING(1 1,0 0),POINT(0 0))'));
+ * Result:
+
+ 1
+
+### geometry_type ###
+* Return the type of the geometry as a string. Eg: 'LINESTRING', 'POLYGON', 'MULTIPOINT', etc.
+
+* Example:
+ * Command:
+
+ geometry_type(st_geom_from_text('LINESTRING(77.29 29.07,77.42 29.26,77.27 29.31,77.29 29.07)'));
+ * Result:
+
+ "LineString"
+
+### st_m ###
+* Return the M coordinate of the point, or NULL if not available. Input must be a point.
+
+* Example:
+ * Command:
+
+ st_m(st_make_point(1, 2, 3, 4));
+ * Result:
+
+ 4.0
+
+### st_n_points ###
+* Return the number of points (vertexes) in a geometry.
+
+* Example:
+ * Command:
+
+ st_n_points(st_geom_from_text('LINESTRING(77.29 29.07,77.42 29.26,77.27 29.31,77.29 29.07)'));
+ * Result:
+
+ 4
+
+### st_n_rings ###
+* If the geometry is a polygon or multi-polygon return the number of rings.
+
+* Example:
+ * Command:
+
+ st_n_rings(st_geom_from_text('POLYGON((10.689 -25.092, 34.595 -20.170, 38.814 -35.639, 13.502 -39.155, 10.689 -25.092))'));
+ * Result:
+
+ 1
+
+### st_num_geometries ###
+* If geometry is a GEOMETRYCOLLECTION (or MULTI*) return the number of geometries, for single geometries will return 1, otherwise return NULL.
+
+* Example:
+ * Command:
+
+ st_num_geometries(st_geom_from_text('LINESTRING(77.29 29.07,77.42 29.26,77.27 29.31,77.29 29.07)'));
+ * Result:
+
+ 1
+* Example:
+ * Command:
+
+ st_num_geometries(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2), LINESTRING(5 5 ,10 10), POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'));
+ * Result:
+
+ 3
+
+### st_num_interiorRings ###
+* Return the number of interior rings of a polygon geometry.
+
+* Example:
+
+ data:image/s3,"s3://crabby-images/501d6/501d693b4b30c72e605bfb46c8c0dba573027361" alt="Image of interiorRings"
+ * Command:
+
+ st_num_interior_rings(st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))"));
+ * Result:
+
+ 1
+
+### st_x ###
+* Return the X coordinate of the point, or NULL if not available. Input must be a point.
+
+* Example:
+ * Command:
+
+ st_x(st_make_point(1, 2, 3, 4));
+ * Result:
+
+ 1.0
+
+### st_y ###
+* Return the Y coordinate of the point, or NULL if not available. Input must be a point.
+
+* Example:
+ * Command:
+
+ st_y(st_make_point(1, 2, 3, 4));
+ * Result:
+
+ 1.0
+
+### st_x_max ###
+* Return X maximum of a bounding box 2d or 3d or a geometry.
+
+* Example:
+ * Command:
+
+ st_x_max(st_geom_from_text('POLYGON((10.689 -25.092, 34.595 -20.170, 38.814 -35.639, 13.502 -39.155, 10.689 -25.092))'));
+ * Result:
+
+ 38.814
+
+### st_x_min ###
+* Return X minimum of a bounding box 2d or 3d or a geometry.
+
+### st_y_max ###
+* Return Y maximum of a bounding box 2d or 3d or a geometry.
+
+### st_y_min ###
+* Return Y minimum of a bounding box 2d or 3d or a geometry.
+
+### st_z ###
+* Return the Z coordinate of the point, or NULL if not available. Input must be a point.
+
+### st_z_max ###
+* Return Z maximum of a bounding box 2d or 3d or a geometry.
+
+### st_z_min ###
+* Return Z minimum of a bounding box 2d or 3d or a geometry.
+
+### st_as_binary ###
+* Return the Well-Known Binary (WKB) representation of the geometry/geography without SRID meta data.
+
+* Example:
+ * Command:
+
+ st_as_binary(st_geom_from_geojson({"type":"Point","coordinates":[2,5],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}));
+ * Result:
+
+ "010100000000000000000000400000000000001440"
+
+### st_as_geojson ###
+* Return the geometry as a GeoJSON element.
+
+* Example:
+ * Command:
+
+ st_as_geojson(st_geom_from_text('POLYGON((10.689 -25.092, 34.595 -20.170, 38.814 -35.639, 13.502 -39.155, 10.689 -25.092))'));
+ * Result:
+
+ "{\"type\":\"Polygon\",\"coordinates\":[[[10.689,-25.092],[13.502,-39.155],[38.814,-35.639],[34.595,-20.17],[10.689,-25.092]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}"
+
+### st_distance ###
+* For geometry type Return the 2D Cartesian distance between two geometries in projected units (based on spatial ref). For geography type defaults to return minimum geodesic distance between two geographies in meters.
+
+* Example:
+ * Command:
+
+ st_distance(st_geom_from_text('POINT(-72.1235 42.3521)'),st_geom_from_text('LINESTRING(-72.1260 42.45, -72.123 42.1546)'));
+ * Result:
+
+ 0.0015056772638282166
+
+### st_length ###
+* Return the 2D length of the geometry if it is a LineString or MultiLineString. geometry are in units of spatial reference and geography are in meters (default spheroid).
+
+* Example:
+ * Command:
+
+ st_length(st_geom_from_text('LINESTRING(-72.1260 42.45, -72.1240 42.45666, -72.123 42.1546)'));
+ * Result:
+
+ 0.30901547439030225
+
+## <a id="predicate">Spatial Predicate</a>
+Spatial predicate functions test for a relationship between two geometries and return a Boolean value (true/false).
+
+### st_intersects ###
+* Return TRUE if the Geometries/Geography "spatially intersect in 2D".
+
+* Example:
+ * Command:
+
+ st_intersects(st_geom_from_text('POINT(0 0)'), st_geom_from_text('LINESTRING ( 0 0, 0 2 )'));
+ * Result:
+
+ true
+
+### st_isclosed ###
+* Return TRUE if the LINESTRING's start and end points are coincident.
+
+* Example:
+ * Command:
+
+ st_is_closed(st_geom_from_text('LINESTRING(0 0, 0 1, 1 1, 0 0)'));
+ * Result:
+
+ true
+
+### st_iscollection ###
+* Return TRUE if the argument is a collection (MULTI*, GEOMETRYCOLLECTION, ...)
+
+* Example:
+ * Command:
+
+ st_is_collection(st_geom_from_text('MULTIPOINT EMPTY'));
+ * Result:
+
+ true
+
+### st_is_empty ###
+* Return true if this Geometry is an empty geometrycollection, polygon, point etc.
+
+* Example:
+ * Command:
+
+ st_is_empty(st_geom_from_text('POLYGON EMPTY'));
+ * Result:
+
+ true
+
+### st_is_ring ###
+* Return TRUE if this LINESTRING is both closed and simple.
+
+* Example:
+ * Command:
+
+ st_is_ring(st_geom_from_text('LINESTRING(0 0, 0 1, 1 1, 1 0, 0 0)'));
+ * Result:
+
+ true
+
+### st_is_simple ###
+* Return (TRUE) if this Geometry has no anomalous geometric points, such as self intersection or self tangency.
+
+* Example:
+ * Command:
+
+ st_is_simple(st_geom_from_text('LINESTRING(1 1,2 2,2 3.5,1 3,1 2,2 1)'));
+ * Result:
+
+ false
+
+### st_contains ###
+* Return true if and only if no points of B lie in the exterior of A, and at least one point of the interior of B lies in the interior of A.
+
+* Example:
+ * Command:
+
+ st_contains(st_geom_from_text('LINESTRING(1 1,-1 -1,2 3.5,1 3,1 2,2 1)'), st_make_point(-1, -1));
+ * Result:
+
+ true
+
+### st_crosses ###
+* Return TRUE if the supplied geometries have some, but not all, interior points in common.
+
+* Example:
+ * Command:
+
+ st_crosses(st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), st_geom_from_text('LINESTRING(0 2,1 2,2 2,3 2,4 2,5 2)'));
+ * Result:
+
+ true
+
+### st_disjoint ###
+* Return TRUE if the Geometries do not "spatially intersect" - if they do not share any space together.
+
+* Example:
+ * Command:
+
+ st_disjoint(st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), st_geom_from_text('POINT(0 0)'));
+ * Result:
+
+ true
+
+### st_equals ###
+* Return true if the given geometries represent the same geometry. Directionality is ignored.
+
+* Example:
+ * Command:
+
+ st_equals(st_geom_from_text('LINESTRING(0 0, 10 10)'), st_geom_from_text('LINESTRING(0 0, 5 5, 10 10)'));
+ * Result:
+
+ true
+
+### st_overlaps ###
+* Return TRUE if the Geometries share space, are of the same dimension, but are not completely contained by each other.
+
+* Example:
+ * Command:
+
+ st_overlaps(st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), st_geom_from_text('LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)'));
+ * Result:
+
+ true
+
+### st_relate ###
+* Return true if this Geometry is spatially related to anotherGeometry, by testing for intersections between the Interior, Boundary and Exterior of the two geometries as specified by the values in the intersectionMatrixPattern.
+
+* Example:
+ * Command:
+
+ st_relate(st_geom_from_text('LINESTRING(1 2, 3 4)'), st_geom_from_text('LINESTRING(5 6, 7 8)'), "FF1FF0102");
+ * Result:
+
+ true
+
+### st_touches ###
+* Return TRUE if the geometries have at least one point in common, but their interiors do not intersect.
+
+* Example:
+ * Command:
+
+ st_touches(st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), st_geom_from_text('POINT(0 2)'));
+ * Result:
+
+ true
+
+### st_within ###
+* Return true if the geometry A is completely inside geometry B.
+
+## <a id="analysis">Spatial Analysis</a>
+Spatial analysis functions take as input one or more geometries and return a geometry as output.
+
+### st_union ###
+* Return a geometry that represents the point set union of the Geometries.
+
+* Example:
+ * Command:
+
+ st_union(st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), st_geom_from_text('POINT(0 2)'));
+ * Result:
+
+ {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_boundary ###
+* Return the closure of the combinatorial boundary of this Geometry.
+
+* Example:
+ * Command:
+
+ st_boundary(st_geom_from_text('POLYGON((1 1,0 0, -1 1, 1 1))'));
+ * Result:
+
+ {"type":"MultiLineString","coordinates":[[[1,1],[-1,1],[0,0],[1,1]]],"crs":null}
+
+### st_end_point ###
+* Return the last point of a LINESTRING or CIRCULARLINESTRING geometry as a POINT.
+
+* Example:
+ * Command:
+
+ st_end_point(st_geom_from_text('LINESTRING(1 1, 2 2, 3 3)'));
+ * Result:
+
+ {"type":"Point","coordinates":[3,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_envelope ###
+* Return a geometry representing the double precision (float8) bounding box of the supplied geometry.
+
+* Example:
+ * Command:
+
+ st_envelope(st_geom_from_text('LINESTRING(1 1, 2 2, 3 3)'));
+ * Result:
+
+ {"type":"Polygon","coordinates":[[[1,1],[3,1],[3,3],[1,3],[1,1]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_exterior_ring ###
+* Return a line string representing the exterior ring of the POLYGON geometry. Return NULL if the geometry is not a polygon. Will not work with MULTIPOLYGON.
+
+* Example:
+ * Command:
+
+ st_exterior_ring(st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))"));
+ * Result:
+
+ {"type":"LineString","coordinates":[[35,10],[45,45],[15,40],[10,20],[35,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_geometry_n ###
+* Return the 1-based Nth geometry if the geometry is a GEOMETRYCOLLECTION, (MULTI)POINT, (MULTI)LINESTRING, MULTICURVE or (MULTI)POLYGON, POLYHEDRALSURFACE. Otherwise, return NULL.
+
+* Example:
+ * Command:
+
+ st_geometry_n(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2),LINESTRING(5 5 ,10 10),POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'),2);
+ * Result:
+
+ {"type":"Polygon","coordinates":[[[-7,4.2],[-7.1,5],[-7.1,4.3],[-7,4.2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_interior_ring_n ###
+* Return the Nth interior linestring ring of the polygon geometry. Return NULL if the geometry is not a polygon or the given N is out of range.
+
+* Example:
+ * Command:
+
+ st_interior_ring_n(st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))"), 0);
+ * Result:
+
+ {"type":"LineString","coordinates":[[20,30],[35,35],[30,20],[20,30]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_point_n ###
+* Return the Nth point in the first LineString or circular LineString in the geometry. Negative values are counted backwards from the end of the LineString. Return NULL if there is no linestring in the geometry.
+
+* Example:
+ * Command:
+
+ st_point_n(st_geom_from_text("LINESTRING(1 1, 2 2, 3 3)"), 1);
+ * Result:
+
+ {"type":"Point","coordinates":[2,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_start_point ###
+* Return the first point of a LINESTRING geometry as a POINT.
+
+* Example:
+ * Command:
+
+ st_start_point(st_geom_from_text("LINESTRING(1 1, 2 2, 3 3)"));
+ * Result:
+
+ {"type":"Point","coordinates":[1,1],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_difference ###
+* Return a geometry that represents that part of geometry A that does not intersect with geometry B.
+
+* Example:
+ * Command:
+
+ st_difference(st_geom_from_text("LINESTRING(1 1, 2 2, 3 3)"), st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))"));
+ * Result:
+
+ {"type":"LineString","coordinates":[[1,1],[2,2],[3,3]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_intersection ###
+* Return a geometry that represents the shared portion of geomA and geomB.
+
+* Example:
+ * Command:
+
+ st_intersection(st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)"));
+ * Result:
+
+ {"type":"LineString","coordinates":[[2,2],[3,3]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+### st_sym_difference ###
+* Return a geometry that represents the portions of A and B that do not intersect. It is called a symmetric difference because ST_SymDifference(A,B) = ST_SymDifference(B,A).
+
+* Example:
+ * Command:
+
+ st_sym_difference(st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)"));
+ * Result:
+
+ {"type":"MultiLineString","coordinates":[[[0,2],[1,2],[2,2],[1,1]],[[5,2],[4,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null}
+
+### st_polygonize ###
+* Aggregate. Creates a GeometryCollection containing possible polygons formed from the constituent linework of a set of geometries.
+
+* Example:
+ * Command:
+
+ st_polygonize([st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)")]);
+ * Result:
+
+ {"type":"GeometryCollection","geometries":[{"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]]},{"type":"LineString","coordinates":[[0,2],[1,2],[2,2],[3,3],[4,2],[5,2]]}],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+
+## <a id="aggregate">Spatial Aggregates</a>
+spatial aggregate function which takes as input a set of geometries and return one geometry as the result.
+
+### st_union ###
+* Returns a geometry that represents the point set union of the Geometries.
+
+* Example:
+ * Command:
+
+ st_union((SELECT VALUE gbu FROM [st_make_point(1.0,1.0),st_make_point(1.0,2.0)] as gbu));
+ * Result:
+
+ {"type":"MultiPoint","coordinates":[[1,1],[1,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
\ No newline at end of file
diff --git a/asterixdb/asterix-doc/src/site/markdown/geo/quickstart.md b/asterixdb/asterix-doc/src/site/markdown/geo/quickstart.md
new file mode 100644
index 0000000..29f16bd
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/markdown/geo/quickstart.md
@@ -0,0 +1,256 @@
+<!--
+ ! 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.
+ !-->
+
+# Getting Started with GIS in AsterixDB (DRAFT) #
+## <a id="toc">Table of Contents</a> ##
+
+* [Introduction](#Introduction)
+* [Create a GIS data type](#create)
+* [Insert geometry data](#update)
+* [Query geometries](#query)
+* [Spatial analysis function](#query2)
+* [Spatial aggregate example](#aggre)
+* [Range query](#range)
+* [K Nearest Neighbor (KNN) query](#knn)
+* [Spatial join query](#joint)
+
+## <a id="Introduction">Introduction</a>
+ This page provides a simple guide to the OGC-compliant geometry functionality in AsterixDB. Internally, AsterixDB relies on the open source library [Esri/geometry-api-java](https://github.com/Esri/geometry-api-java) that provides OGC-geometry feature processing. Currently, the AsterixDB geometry library supports
+ [GeoJSON](https://tools.ietf.org/html/rfc7946), [Well known Text](http://docs.opengeospatial.org/is/12-063r5/12-063r5.html)
+ and [Well known Binary formats](http://portal.opengeospatial.org/files/?artifact_id=25354).
+ For a complete list of all the functions, please check the [AsterixDB GIS functions page](functions.md).
+ Here are some detailed examples.
+
+## <a id="create">Create a GIS data type</a>
+
+```
+DROP DATAVERSE GISTest IF EXISTS;
+CREATE DATAVERSE GISTest;
+USE GISTest;
+
+CREATE TYPE GeometryType AS{
+ id : int,
+ myGeometry : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
+```
+
+## <a id="update">Insert geometry data</a>
+
+ ```
+USE GISTest;
+
+INSERT INTO Geometries ([
+{"id": 123, "myGeometry": st_geom_from_geojson({"type":"Point","coordinates":[-118.4,33.93]})},
+{"id": 124, "myGeometry": st_geom_from_geojson({"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]]})},
+{"id": 126, "myGeometry": st_geom_from_geojson({"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]]})},
+{"id": 127, "myGeometry": st_geom_from_geojson({"type": "MultiPoint","coordinates": [[10, 40], [40, 30], [20, 20], [30, 10]]})},
+{"id": 128, "myGeometry": st_geom_from_geojson({"type": "MultiLineString","coordinates": [[[10, 10], [20, 20], [10, 40]],[[40, 40], [30, 30], [40, 20], [30, 10]]]})},
+{"id": 129, "myGeometry": st_geom_from_geojson({"type": "MultiPolygon","coordinates": [[[[40, 40], [20, 45], [45, 30], [40, 40]]],[[[20, 35], [10, 30], [10, 10], [30, 5], [45, 20], [20, 35]],[[30, 20], [20, 15], [20, 25], [30, 20]]]]})},
+{"id": 130, "myGeometry": st_make_point(-71.1043443253471, 42.3150676015829)},
+{"id": 131, "myGeometry": st_make_point(1.0,2.0,3.0)},
+{"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)},
+{"id": 133, "myGeometry": st_geom_from_text('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))')},
+{"id": 134, "myGeometry": st_geom_from_wkb(hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"))},
+{"id": 135, "myGeometry": st_line_from_multipoint(st_geom_from_text('MULTIPOINT(1 2 , 4 5 , 7 8 )'))},
+{"id": 136, "myGeometry": st_make_envelope(10, 10, 11, 11, 4326)},
+{"id": 137, "myGeometry": st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))")}
+]);
+```
+
+## <a id="query">Query geometries</a>
+
+ ```
+USE GISTest;
+
+FROM Geometries SELECT *;
+```
+
+result:
+
+```
+{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],"crs":null} } }
+{ "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],"crs":null} } }
+{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+```
+
+## <a id="query2">Spatial analysis functions</a>
+
+ The following query filters out only the geometries of type “Polygon” and displays the geometry
+ in the Well known text format along with the area of the relevant geometry.
+
+ ```
+USE GISTest;
+
+FROM Geometries as geo
+WHERE geometry_type(geo.myGeometry)='Polygon'
+SELECT VALUE {"Polygon":st_as_text(geo.myGeometry), "Area":st_area(geo.myGeometry)};
+```
+
+result:
+
+```
+{ "Polygon": "POLYGON ((8.7599721 49.7103028, 8.759997 49.7102752, 8.7600145 49.7102818, 8.7600762 49.7102133, 8.760178 49.7102516, 8.7600914 49.7103478, 8.7599721 49.7103028))", "Area": 1.3755215000294761E-8 }
+{ "Polygon": "POLYGON ((743238 2967416, 743265.625 2967416, 743265 2967450, 743238 2967450, 743238 2967416))", "Area": 928.625 }
+{ "Polygon": "POLYGON ((10 10, 11 10, 11 11, 10 11, 10 10))", "Area": 1.0 }
+{ "Polygon": "POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10), (20 30, 35 35, 30 20, 20 30))", "Area": 675.0 }
+```
+
+## <a id="query2">Spatial aggregate example</a>
+
+`st_union` function has been implemented both as a normal and an aggregate function. The following query shows how to query the aggregate version of this function:
+
+```
+USE GISTest;
+
+st_union((SELECT VALUE gbu.myGeometry FROM Geometries as gbu));
+```
+
+result:
+
+```
+{"type":"MultiPolygon","coordinates":[[[[10,10],[30,5],[35,10],[45,20],[38.90243902439025,23.65853658536585],[41.34146341463415,32.19512195121951],[45,30],[42.27272727272727,35.45454545454545],[45,45],[30,42.5],[20,45],[25.434782608695656,41.73913043478261],[15,40],[12.857142857142858,31.428571428571427],[10,30],[10,20],[10,11],[10,10]],[[32.5,27.5],[25.357142857142858,31.785714285714285],[35,35],[32.5,27.5]],[[20,15],[20,16],[21.11111111111111,15.555555555555555],[20,15]]],[[[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133]]],[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]]],"crs":null}
+```
+
+## <a id="range">Range query</a>
+
+ ```
+USE GISTest;
+
+FROM Geometries geo
+WHERE st_intersects(geo.myGeometry, st_geom_from_text("POLYGON((1 1,5 1,5 5,1 5,1 1),(2 2, 3 2, 3 3, 2 3,2 2))"))
+SELECT VALUE geo.myGeometry;
+```
+
+result:
+
+```
+{"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+{"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+{"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+```
+
+## <a id="knn">K Nearest Neighbor (KNN) query</a>
+
+ ```
+USE GISTest;
+
+FROM Geometries geo
+SELECT VALUE geo.myGeometry
+ORDER BY st_distance(geo.myGeometry, st_make_point(1,2))
+LIMIT 5;
+```
+
+result:
+
+```
+{"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+{"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+{"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+{"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
+{"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],"crs":null}
+```
+
+## <a id="joint">Spatial join query</a>
+
+For the spatial join query let us create a new dataverse and two new data types:
+
+ ```
+DROP DATAVERSE SJTest IF EXISTS;
+CREATE DATAVERSE SJTest;
+USE SJTest;
+
+CREATE TYPE StateType AS{
+id : int,
+name: string,
+boundary : geometry
+};
+
+CREATE DATASET States (StateType) PRIMARY KEY id;
+
+CREATE TYPE POIType AS {
+id : int,
+longitude : double,
+latitude : double
+};
+
+CREATE DATASET POIS (POIType) PRIMARY KEY id;
+```
+
+Insert data into states:
+
+```
+USE SJTest;
+
+INSERT INTO States ([
+ {"id": 1, "name": "Nebraska", "boundary": st_geom_from_text("POLYGON ((-104.05341854507101 41.1705389679833, -104.053028 43.000586999999996, -98.49855 42.99856, -98.01304599999999 42.762299, -97.306677 42.867604, -96.38600699999999 42.474495, -96.06487899999999 41.79623, -96.09200799999999 41.53391, -95.87468899999999 41.307097, -95.88534899999999 40.721092999999996, -95.30829 39.999998, -102.051744 40.003077999999995, -102.051614 41.002376999999996, -104.053249 41.001405999999996, -104.05341854507101 41.1705389679833))") },
+ {"id": 2, "name": "Washington", "boundary": st_geom_from_text("MULTIPOLYGON (((-124.732755385025 48.165328947686795, -124.676262 48.391371, -123.981032 48.164761, -123.10189199999999 48.184951999999996, -122.871992 47.993493, -122.75413 48.1447, -122.610341 47.887343, -122.784553 47.686561, -122.864651 47.804669, -123.157948 47.356235999999996, -122.874586 47.413874, -123.119681 47.385532, -122.525329 47.912335999999996, -122.54636949132416 47.317877648507704, -122.324833 47.348521, -122.43694099999999 47.661719, -122.218982 48.020275999999996, -122.383911 48.227486, -122.47892813788141 48.175746487177165, -122.388048 48.30083, -122.57760827271139 48.38291646865838, -122.505828 48.297677, -122.732358 48.226144, -122.3773 47.905941, -122.769939 48.227548, -122.60660653630984 48.395473767832804, -122.674158 48.424726, -122.425271 48.599522, -122.535803 48.776128, -122.673472 48.733081999999996, -122.75802 49.002356999999996, -117.032351 48.999188, -117.062748 46.353623999999996, -116.915989 45.995413, -118.987129 45.999855, -121.145534 45.607886, -121.533106 45.726541, -122.266701 45.543841, -122.67500799999999 45.618038999999996, -123.004233 46.133823, -124.07776799999999 46.272324, -124.06905 46.647258, -123.953699 46.378845, -123.829356 46.713356, -124.092176 46.741623999999995, -124.138225 46.905533999999996, -123.83890000000001 46.953950999999996, -124.122057 47.04165, -124.173877 46.927234999999996, -124.425195 47.738434, -124.732755385025 48.165328947686795), (-122.56199279209496 47.29381043649037, -122.683943 47.365154999999994, -122.76539771783851 47.18116187703539, -122.678476 47.102742, -122.56199279209496 47.29381043649037), (-122.77734484602688 47.19194045282469, -122.82666 47.405806999999996, -122.871472 47.276861, -122.77734484602688 47.19194045282469)), ((-122.4789801236288 48.17567493623048, -122.358963 48.054851, -122.510562 48.132207, -122.4789801236288 48.17567493623048)), ((-122.526031 47.358906, -122.457246 47.505848, -122.373627 47.388718, -122.526031 47.358906)))") },
+ {"id": 3, "name": "New Mexico", "boundary": st_geom_from_text("POLYGON ((-109.050173 31.480003999999997, -109.045223 36.999083999999996, -103.002199 37.000104, -103.064423 32.000518, -106.618486 32.000495, -106.528242 31.783147999999997, -108.208394 31.783599, -108.208573 31.333395, -109.050044 31.332501999999998, -109.050173 31.480003999999997))") },
+ {"id": 4, "name": "South Dakota", "boundary": st_geom_from_text("POLYGON ((-104.057698 44.997431, -104.045443 45.94531, -96.563672 45.935238999999996, -96.857751 45.605962, -96.45306699999999 45.298114999999996, -96.45326 43.500389999999996, -96.60285999999999 43.450907, -96.436589 43.120841999999996, -96.639704 42.737071, -96.44550799999999 42.490629999999996, -97.23786799999999 42.853139, -98.035034 42.764205, -98.49855 42.99856, -104.053028 43.000586999999996, -104.057698 44.997431))") }
+ ]);
+```
+
+Insert data into POIS:
+
+```
+USE SJTest;
+
+INSERT INTO POIS ([{"id": 477884092592037888, "latitude": 41.1029498, "longitude": -96.2632202 },
+ {"id": 477689754977181696, "latitude": 47.23433434, "longitude": -122.15083003 },
+ {"id": 477697263058157569, "latitude": 35.27988499, "longitude": -106.6787443 },
+ {"id": 477833117374611456, "latitude": 44.11614436, "longitude": -103.06577797 },
+ {"id": 477957785909735424, "latitude": 39.81871193, "longitude": -75.53023171 },
+ {"id": 477890178640384001, "latitude": 37.5688636, "longitude": -77.4540628 },
+ {"id": 478004308827717632, "latitude": 39.14933024, "longitude": -84.43623134 },
+ {"id": 478029048799846401, "latitude": 40.3030824, "longitude": -121.228368 }
+ ]);
+```
+
+Now let us perform the spatial join query:
+
+```
+USE SJTest;
+
+FROM States, POIS
+WHERE st_contains(States.boundary, st_make_point(POIS.longitude, POIS.latitude))
+SELECT States.name, POIS.id;
+```
+
+result:
+
+```
+{ "name": "Nebraska", "id": 477884092592037888 }
+{ "name": "Washington", "id": 477689754977181696 }
+{ "name": "South Dakota", "id": 477833117374611456 }
+{ "name": "New Mexico", "id": 477697263058157569 }
+```
+
+You can find a more comprehensive spatial join example [here](../../resources/data/SJ.sqlpp).
\ No newline at end of file
diff --git a/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md b/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md
index 559e2d2..cafa381 100644
--- a/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md
+++ b/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md
@@ -70,40 +70,33 @@
Let us now create an index on the `categories` multiset of the `products` dataset.
- CREATE INDEX pCategoriesIdx ON products (UNNEST categories);
+ CREATE INDEX pCategoriesIdx ON products (UNNEST categories) EXCLUDE UNKNOWN KEY;
Suppose we now want to find all products that have the category "Food".
The following membership query will utilize the index we just created.
- SET `compiler.arrayindex` "true";
-
SELECT p
FROM products p
WHERE "Food" IN p.categories;
We can also rewrite the query above as an explicit existential quantification query with an equality predicate and the index will be utilized.
- SET `compiler.arrayindex` "true";
-
SELECT p
FROM products p
WHERE SOME c IN p.categories SATISFIES c = "Food";
Let us now create an index on the `qty` and `price` fields in the `items` array of the `orders` dataset.
- CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty, price);
+ CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty, price) EXCLUDE UNKNOWN KEY;
Now suppose we want to find all orders that only have items with large quantities and low prices, not counting orders without any items.
The following universal quantification query will utilize the index we just created.
- SET `compiler.arrayindex` "true";
-
SELECT o
FROM orders o
- WHERE LEN(o.items) > 0 AND
- (EVERY i IN o.items SATISFIES i.qty > 100 AND i.price < 5.00);
+ WHERE SOME AND EVERY i IN o.items SATISFIES i.qty > 100 AND i.price < 5.00;
-Take note of the `LEN(o.items) > 0` conjunct.
+Take note of the `SOME AND EVERY` quantifier instead of the `EVERY` quantifier.
Array indexes cannot be used for queries with potentially empty arrays.
@@ -113,8 +106,6 @@
We can express the same membership / existential example above using an explicit `UNNEST` query.
(To keep the same cardinality as the query above (i.e. to undo the `UNNEST`), we add a `DISTINCT` clause, though the index would be utilized either way.)
- SET `compiler.arrayindex` "true";
-
SELECT DISTINCT p
FROM products p, p.categories c
WHERE c = "Food";
@@ -122,8 +113,6 @@
As another example, suppose that we want to find all orders that have *some* item with a large quantity.
The following query will utilize the `oItemsQtyPriceIdx` we created, using only the first field in the index `qty`.
- SET `compiler.arrayindex` "true";
-
SELECT DISTINCT o
FROM orders o, o.items i
WHERE i.qty > 100;
@@ -134,18 +123,16 @@
Finally, array indexes can also be used for index nested-loop joins if the field being joined is located within an array.
Let us create another index on the `items` array of the `orders` dataset, this time on the `productno` field.
- CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno);
+ CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno) EXCLUDE UNKNOWN KEY;
Now suppose we want to find all products located in a specific order.
We can accomplish this with the join query below.
Note that we must specify the `indexnl` join hint to tell AsterixDB that we want to optimize this specific join, as hash joins are the default join method otherwise.
- SET `compiler.arrayindex` "true";
-
SELECT DISTINCT p
- FROM products p, orders o, o.items i
- WHERE i.productno /*+ indexnl */ = p.productno
- AND o.custid = "C41";
+ FROM products p, orders o
+ WHERE o.custid = "C41" AND
+ SOME i IN o.items SATISFIES i.productno /*+ indexnl */ = p.productno;
## <a id="ComplexIndexingExamples">Complex Indexing Examples</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
@@ -155,9 +142,17 @@
Similar to atomic indexes, array indexes are not limited to closed fields.
The following DDLs illustrate how we could express `CREATE INDEX` statements comparable to those above if the to-be-indexed fields were not included in the their dataset's type definitions.
- CREATE INDEX pCategoriesIdx ON products (UNNEST categories : string);
- CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty : int, price : int);
- CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno : int);
+ CREATE INDEX pCategoriesIdx ON products (UNNEST categories : string) EXCLUDE UNKNOWN KEY;
+ CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty : int, price : int) EXCLUDE UNKNOWN KEY;
+ CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno : int) EXCLUDE UNKNOWN KEY;
+
+### Composite Atomic-Array Indexes
+
+Indexed elements within array indexes are also not limited to fields within arrays.
+The following DDLs demonstrate indexing fields that are within an array and fields that are outside any array.
+
+ CREATE INDEX oOrderNoItemPriceIdx ON orders (orderno, ( UNNEST items SELECT price )) EXCLUDE UNKNOWN KEY;
+ CREATE INDEX oOrderItemPriceNoIdx ON orders (( UNNEST items SELECT price ), orderno) EXCLUDE UNKNOWN KEY;
### Arrays in Arrays
@@ -166,17 +161,15 @@
The following DDLs describe indexing the `qty` field in an `items` array at various depths.
// { orderno: ..., items0: [ { items1: [ { qty: int, ... } ] } ] }
- CREATE INDEX oItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 SELECT qty);
+ CREATE INDEX oItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 SELECT qty) EXCLUDE UNKNOWN KEY;
// { orderno: ..., items0: [ { items1: [ { items2: [ { qty: int, ... } ] } ] } ] }
- CREATE INDEX oItemItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 UNNEST items2 SELECT qty);
+ CREATE INDEX oItemItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 UNNEST items2 SELECT qty) EXCLUDE UNKNOWN KEY;
The queries below will utilize the indexes above.
The first query utilizes the `oItemItemQtyIdx` index through nested existential quantification.
The second query utilizes the `oItemItemItemQtyIdx` index with three unnesting clauses.
- SET `compiler.arrayindex` "true";
-
SELECT o
FROM orders o
WHERE SOME o0 IN o.items0 SATISFIES (
diff --git a/asterixdb/asterix-doc/src/site/resources/data/SJ.sqlpp b/asterixdb/asterix-doc/src/site/resources/data/SJ.sqlpp
new file mode 100644
index 0000000..5ac1852
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/data/SJ.sqlpp
@@ -0,0 +1,1107 @@
+/*
+ * Copyright 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.
+ */
+
+drop dataverse SJTest if exists;
+create dataverse SJTest;
+
+use SJTest;
+
+CREATE TYPE StateType AS {
+ id : int,
+ name: string,
+ boundary : geometry
+};
+
+CREATE DATASET States (StateType) PRIMARY KEY id;
+
+CREATE TYPE POIType AS {
+ id : int,
+ longitude : double,
+ latitude : double
+};
+
+CREATE DATASET POIS (POIType) PRIMARY KEY id;
+
+
+// ----------- Insert data into states
+
+use SJTest;
+
+INSERT INTO States ([
+{"id": 1, "name": "Nebraska", "boundary": st_geom_from_text("POLYGON ((-104.05341854507101 41.1705389679833, -104.053028 43.000586999999996, -98.49855 42.99856, -98.01304599999999 42.762299, -97.306677 42.867604, -96.38600699999999 42.474495, -96.06487899999999 41.79623, -96.09200799999999 41.53391, -95.87468899999999 41.307097, -95.88534899999999 40.721092999999996, -95.30829 39.999998, -102.051744 40.003077999999995, -102.051614 41.002376999999996, -104.053249 41.001405999999996, -104.05341854507101 41.1705389679833))") },
+{"id": 2, "name": "Washington", "boundary": st_geom_from_text("MULTIPOLYGON (((-124.732755385025 48.165328947686795, -124.676262 48.391371, -123.981032 48.164761, -123.10189199999999 48.184951999999996, -122.871992 47.993493, -122.75413 48.1447, -122.610341 47.887343, -122.784553 47.686561, -122.864651 47.804669, -123.157948 47.356235999999996, -122.874586 47.413874, -123.119681 47.385532, -122.525329 47.912335999999996, -122.54636949132416 47.317877648507704, -122.324833 47.348521, -122.43694099999999 47.661719, -122.218982 48.020275999999996, -122.383911 48.227486, -122.47892813788141 48.175746487177165, -122.388048 48.30083, -122.57760827271139 48.38291646865838, -122.505828 48.297677, -122.732358 48.226144, -122.3773 47.905941, -122.769939 48.227548, -122.60660653630984 48.395473767832804, -122.674158 48.424726, -122.425271 48.599522, -122.535803 48.776128, -122.673472 48.733081999999996, -122.75802 49.002356999999996, -117.032351 48.999188, -117.062748 46.353623999999996, -116.915989 45.995413, -118.987129 45.999855, -121.145534 45.607886, -121.533106 45.726541, -122.266701 45.543841, -122.67500799999999 45.618038999999996, -123.004233 46.133823, -124.07776799999999 46.272324, -124.06905 46.647258, -123.953699 46.378845, -123.829356 46.713356, -124.092176 46.741623999999995, -124.138225 46.905533999999996, -123.83890000000001 46.953950999999996, -124.122057 47.04165, -124.173877 46.927234999999996, -124.425195 47.738434, -124.732755385025 48.165328947686795), (-122.56199279209496 47.29381043649037, -122.683943 47.365154999999994, -122.76539771783851 47.18116187703539, -122.678476 47.102742, -122.56199279209496 47.29381043649037), (-122.77734484602688 47.19194045282469, -122.82666 47.405806999999996, -122.871472 47.276861, -122.77734484602688 47.19194045282469)), ((-122.4789801236288 48.17567493623048, -122.358963 48.054851, -122.510562 48.132207, -122.4789801236288 48.17567493623048)), ((-122.526031 47.358906, -122.457246 47.505848, -122.373627 47.388718, -122.526031 47.358906)))") },
+{"id": 3, "name": "New Mexico", "boundary": st_geom_from_text("POLYGON ((-109.050173 31.480003999999997, -109.045223 36.999083999999996, -103.002199 37.000104, -103.064423 32.000518, -106.618486 32.000495, -106.528242 31.783147999999997, -108.208394 31.783599, -108.208573 31.333395, -109.050044 31.332501999999998, -109.050173 31.480003999999997))") },
+{"id": 4, "name": "South Dakota", "boundary": st_geom_from_text("POLYGON ((-104.057698 44.997431, -104.045443 45.94531, -96.563672 45.935238999999996, -96.857751 45.605962, -96.45306699999999 45.298114999999996, -96.45326 43.500389999999996, -96.60285999999999 43.450907, -96.436589 43.120841999999996, -96.639704 42.737071, -96.44550799999999 42.490629999999996, -97.23786799999999 42.853139, -98.035034 42.764205, -98.49855 42.99856, -104.053028 43.000586999999996, -104.057698 44.997431))") },
+{"id": 5, "name": "Kentucky", "boundary": st_geom_from_text("POLYGON ((-89.405654 36.528165, -89.15908 36.666351999999996, -89.08652599999999 37.165602, -88.45894799999999 37.073796, -88.476592 37.386874999999996, -88.067844 37.485867, -88.15937199999999 37.661847, -87.904789 37.924892, -87.635806 37.827014999999996, -87.585916 37.975442, -87.111133 37.782512, -86.820071 37.999392, -86.63826499999999 37.842718, -86.360377 38.198796, -86.03818799999999 37.95935, -85.42307699999999 38.531580999999996, -85.43406499999999 38.729455, -84.812877 38.786086999999995, -84.897364 39.057378, -84.750749 39.147358, -84.449793 39.117754, -84.212904 38.805707, -83.679484 38.630036, -82.879492 38.751476, -82.604089 38.459841, -82.501862 37.9332, -81.96497099999999 37.543026, -83.136395 36.743088, -83.690714 36.582581, -88.070532 36.678118, -88.05320499999999 36.497129, -89.405654 36.528165))") },
+{"id": 6, "name": "Georgia", "boundary": st_geom_from_text("POLYGON ((-85.605165 34.984677999999995, -83.10286699999999 34.997257, -83.34261699999999 34.682991, -82.858837 34.45522, -82.55683499999999 33.945353, -81.49225299999999 33.009342, -81.117234 32.117605, -80.841127 32.023522, -81.203572 31.719448, -81.26955199999999 31.258740999999997, -81.493651 30.977528, -81.444124 30.709713999999998, -81.949787 30.827492999999997, -82.036825 30.377883999999998, -82.161757 30.357851, -82.214839 30.568590999999998, -84.913522 30.752291, -85.10751599999999 31.186450999999998, -85.141831 31.839260999999997, -84.88907999999999 32.260911, -85.18440000000001 32.861317, -85.605165 34.984677999999995))") },
+{"id": 7, "name": "Arkansas", "boundary": st_geom_from_text("POLYGON ((-94.6178329666013 36.4994141203285, -90.152481 36.497952, -90.06398 36.303038, -90.37789 35.995683, -89.644395 35.894782, -89.956254 35.733385999999996, -89.909022 35.520548, -90.179265 35.385194, -90.065392 35.137691, -90.29559599999999 35.040093, -90.307384 34.846195, -90.479872 34.883264, -90.571145 34.420319, -90.92901499999999 34.244541, -90.810884 34.155902999999995, -91.087921 33.975335, -90.988466 33.78453, -91.14661799999999 33.732456, -91.03146 33.678142, -91.22901499999999 33.677543, -91.208113 33.402007, -91.057621 33.445341, -91.166073 33.004106, -94.042964 33.019219, -94.04344999999999 33.552253, -94.485875 33.637867, -94.430662 35.392478, -94.6178329666013 36.4994141203285))") },
+{"id": 8, "name": "Pennsylvania", "boundary": st_geom_from_text("POLYGON ((-80.519891 40.906661, -80.519425 41.977523, -79.761951 42.26986, -79.76137399999999 41.999067, -75.359579 41.999445, -75.072172 41.813731999999995, -74.983341 41.480894, -74.689516 41.363842999999996, -75.130575 40.991093, -75.051029 40.865662, -75.20392 40.691497999999996, -74.724304 40.14701, -75.773786 39.7222, -80.519342 39.721402999999995, -80.519891 40.906661))") },
+{"id": 9, "name": "Mississippi", "boundary": st_geom_from_text("POLYGON ((-91.65500899999999 31.251814999999997, -91.472465 31.371326, -91.403915 31.589765999999997, -91.51233599999999 31.634722, -91.005006 32.142852, -91.164171 32.196888, -90.875631 32.372434, -91.116008 32.48314, -90.987202 32.495663, -91.153556 32.626180999999995, -91.08680199999999 32.976265999999995, -91.213972 32.927198, -91.057621 33.445341, -91.208113 33.402007, -91.22901499999999 33.677543, -91.034565 33.673018, -91.14661799999999 33.732456, -90.988466 33.78453, -91.087921 33.975335, -90.810884 34.155902999999995, -90.92901499999999 34.244541, -90.571145 34.420319, -90.479872 34.883264, -90.307384 34.846195, -90.309297 34.995694, -88.200064 34.995633999999995, -88.097888 34.892202, -88.473227 31.893856, -88.390746 30.352075, -89.335942 30.374015999999997, -89.570154 30.180297, -89.852263 30.662934, -89.729616 31.003926999999997, -91.63694199999999 30.999416, -91.65500899999999 31.251814999999997))") },
+{"id": 10, "name": "Colorado", "boundary": st_geom_from_text("POLYGON ((-109.060253 38.599328, -109.050076 41.000659, -102.051614 41.002376999999996, -102.042089 36.993016, -109.045223 36.999083999999996, -109.060253 38.599328))") },
+{"id": 11, "name": "Utah", "boundary": st_geom_from_text("POLYGON ((-114.052962 37.592783, -114.041723 41.993719999999996, -111.046689 42.001567, -111.046723 40.997959, -109.050076 41.000659, -109.045223 36.999083999999996, -114.0506 37.000395999999995, -114.052962 37.592783))") },
+{"id": 12, "name": "Tennessee", "boundary": st_geom_from_text("POLYGON ((-90.31029799999999 35.004295, -90.065392 35.137691, -90.179265 35.385194, -89.909022 35.520548, -89.956254 35.733385999999996, -89.64534599999999 35.891419, -89.705328 36.239898, -89.534745 36.252576, -89.539232 36.497934, -88.05320499999999 36.497129, -88.070532 36.678118, -81.6469 36.611917999999996, -81.707438 36.335170999999995, -82.033141 36.120422, -82.557874 35.953901, -82.637165 36.065805, -82.992053 35.773948, -83.880074 35.518744999999996, -84.052612 35.269982, -84.29024 35.225572, -84.32186899999999 34.988408, -90.31029799999999 35.004295))") },
+{"id": 13, "name": "Wyoming", "boundary": st_geom_from_text("POLYGON ((-111.056888 44.866658, -111.055199 45.001321, -104.057698 44.997431, -104.053249 41.001405999999996, -111.046723 40.997959, -111.056888 44.866658))") },
+{"id": 14, "name": "Indiana", "boundary": st_geom_from_text("POLYGON ((-88.09776 37.904026, -87.911428 38.16174, -87.987618 38.257380999999995, -87.496494 38.742728, -87.658745 39.135996999999996, -87.531204 39.355104, -87.52404399999999 41.708335, -84.805883 41.760216, -84.812877 38.786086999999995, -85.43406499999999 38.729455, -85.42307699999999 38.531580999999996, -86.042354 37.958017999999996, -86.360377 38.198796, -86.63826499999999 37.842718, -86.820071 37.999392, -87.111133 37.782512, -87.589816 37.976042, -87.615399 37.831973999999995, -87.898062 37.927513999999995, -87.95259 37.771741999999996, -88.09776 37.904026))") },
+{"id": 15, "name": "Nevada", "boundary": st_geom_from_text("POLYGON ((-120.005743 39.228663999999995, -119.999168 41.99454, -114.041723 41.993719999999996, -114.043944 36.193349999999995, -114.152795 36.023694, -114.753196 36.089513, -114.633487 35.001857, -120.001014 38.999573999999996, -120.005743 39.228663999999995))") },
+{"id": 16, "name": "Illinois", "boundary": st_geom_from_text("POLYGON ((-91.512974 40.181062, -91.40524099999999 40.554641, -90.96291599999999 40.924957, -91.04589 41.414085, -90.461432 41.523533, -90.180954 41.809354, -90.16289499999999 42.116718, -90.642843 42.508480999999996, -87.80209099999999 42.492576, -87.828569 42.269922, -87.524141 41.72399, -87.531204 39.355104, -87.658745 39.135996999999996, -87.496494 38.742728, -87.987618 38.257380999999995, -88.160187 37.657592, -88.067844 37.485867, -88.470224 37.396255, -88.45894799999999 37.073796, -88.974711 37.229707999999995, -89.132685 36.9822, -89.378277 37.039605, -89.51425499999999 37.689923, -90.35955899999999 38.224525, -90.10940699999999 38.843548, -90.628485 38.891617, -90.72995999999999 39.255894, -91.367753 39.729029, -91.512974 40.181062))") },
+{"id": 17, "name": "Vermont", "boundary": st_geom_from_text("POLYGON ((-73.43773999999999 44.045006, -73.293855 44.437556, -73.343124 45.010839999999995, -71.46455499999999 45.013636999999996, -71.631883 44.752463, -71.577643 44.502691999999996, -72.033136 44.320364999999995, -72.030182 44.079682999999996, -72.37944 43.574069, -72.556214 42.866949999999996, -72.458519 42.726853, -73.276421 42.746019, -73.24158899999999 43.534973, -73.430947 43.587036, -73.43773999999999 44.045006))") },
+{"id": 18, "name": "Montana", "boundary": st_geom_from_text("POLYGON ((-116.04966720193102 48.4403221865285, -116.049193 49.000912, -104.048736 48.999877, -104.039138 44.99852, -111.055199 45.001321, -111.048974 44.474072, -111.381665 44.754405, -111.467804 44.53902, -112.286187 44.568472, -112.855395 44.359975, -113.131453 44.772836999999996, -113.454858 44.865572, -113.935505 45.694646999999996, -114.333218 45.459316, -114.564691 45.558026999999996, -114.566172 45.773863999999996, -114.388243 45.88234, -114.527096 46.146218, -114.320665 46.646963, -114.76689 46.696901, -115.321032 47.256105, -115.759233 47.423471, -115.62910000000001 47.47672, -116.04885 47.977185999999996, -116.04966720193102 48.4403221865285))") },
+{"id": 19, "name": "Iowa", "boundary": st_geom_from_text("POLYGON ((-96.639704 42.737071, -96.436589 43.120841999999996, -96.598928 43.500457, -91.21770599999999 43.50055, -91.057918 43.255365999999995, -91.179457 43.067426999999995, -91.06468 42.750913999999995, -90.70630299999999 42.634169, -90.140613 41.995999, -90.343228 41.587832999999996, -91.04589 41.414085, -91.113648 41.241400999999996, -90.95223299999999 40.954046999999996, -91.12392799999999 40.669152, -91.40524099999999 40.554641, -91.38835999999999 40.384929, -91.729115 40.61364, -95.76564499999999 40.585208, -96.129505 41.971672999999996, -96.639704 42.737071))") },
+{"id": 20, "name": "South Carolina", "boundary": st_geom_from_text("POLYGON ((-83.35391 34.699219, -83.108606 35.000659, -82.39292999999999 35.215402, -81.04362499999999 35.149877, -80.79754299999999 34.819786, -79.675299 34.804744, -78.54203 33.851924, -78.938076 33.639826, -79.362292 33.009087, -80.472068 32.496964, -80.452956 32.322494, -80.733637 32.319469, -80.66916599999999 32.216783, -80.917845 32.037575, -81.117234 32.117605, -81.49225299999999 33.009342, -82.55683499999999 33.945353, -82.858837 34.45522, -83.35391 34.699219))") },
+{"id": 21, "name": "Arizona", "boundary": st_geom_from_text("POLYGON ((-114.816294 32.508038, -114.468971 32.845155, -114.731223 33.302434, -114.524599 33.552231, -114.535478 33.934650999999995, -114.131211 34.26273, -114.635176 34.875003, -114.754798 36.084703999999995, -114.14762999999999 36.025186, -114.0506 37.000395999999995, -109.045223 36.999083999999996, -109.050044 31.332501999999998, -111.074825 31.332238999999998, -114.816294 32.508038))") },
+{"id": 22, "name": "Ohio", "boundary": st_geom_from_text("POLYGON ((-84.82015899999999 39.227225, -84.805972 41.696118, -83.45383199999999 41.732647, -82.71093499999999 41.536648, -82.9422 41.419754, -82.481214 41.381342, -81.738755 41.48855, -80.519425 41.977523, -80.518991 40.638801, -80.667957 40.582496, -80.59989499999999 40.317668999999995, -80.83187099999999 39.705655, -81.217315 39.387589999999996, -81.689483 39.266042999999996, -81.762659 38.924121, -82.035963 39.025478, -82.33033499999999 38.444500000000005, -82.578999 38.407782999999995, -82.88919299999999 38.756076, -83.29419299999999 38.596588, -83.76509 38.652881, -84.212904 38.805707, -84.455342 39.12036, -84.820157 39.10548, -84.82015899999999 39.227225))") },
+{"id": 23, "name": "Alabama", "boundary": st_geom_from_text("POLYGON ((-88.473227 31.893856, -88.097888 34.892202, -88.20295899999999 35.008027999999996, -85.605165 34.984677999999995, -85.18440000000001 32.861317, -84.88907999999999 32.260911, -85.140731 31.857460999999997, -85.002499 31.000681999999998, -87.598928 30.997456999999997, -87.368191 30.433407, -87.452378 30.300200999999998, -88.027678 30.223644999999998, -87.755314 30.280071999999997, -87.985559 30.677508999999997, -88.138576 30.311988, -88.395023 30.369425, -88.473227 31.893856))") },
+{"id": 24, "name": "Wisconsin", "boundary": st_geom_from_text("POLYGON ((-92.888114 45.628377, -92.712503 45.891704999999995, -92.294033 46.074377, -92.291292 46.668141999999996, -90.855874 46.962232, -90.751031 46.887963, -90.94506299999999 46.588974, -90.784425 46.729375, -90.12048899999999 46.336852, -88.18978899999999 45.952208, -87.80586699999999 45.706841, -87.888052 45.354696999999994, -87.657349 45.368752, -87.736509 45.173389, -87.575283 45.070454, -87.839028 44.931718, -88.008702 44.541714, -87.578305 44.853383, -87.385547 44.831137, -87.236997 45.169449, -86.983355 45.295367999999996, -87.736178 43.880421, -87.91178699999999 43.250406, -87.80209099999999 42.492576, -90.642843 42.508480999999996, -91.053733 42.738237999999996, -91.179457 43.067426999999995, -91.057918 43.255365999999995, -91.21477 43.365874, -91.284138 43.847065, -92.807317 44.750364, -92.646602 45.441635, -92.888114 45.628377))") },
+{"id": 25, "name": "Oregon", "boundary": st_geom_from_text("POLYGON ((-124.564595 42.840876, -124.150267 43.910849999999996, -123.962887 45.280218, -124.01263399999999 46.23685, -123.115904 46.185268, -122.813998 45.960983999999996, -122.76381 45.657137999999996, -122.294901 45.543541, -121.811304 45.706761, -120.40396 45.699248999999995, -118.941242 46.000574, -116.915989 45.995413, -116.463504 45.615784999999995, -117.24267499999999 44.396547999999996, -116.894083 44.160191, -117.032881 43.830025, -117.026197 41.99989, -124.21160499999999 41.99846, -124.564595 42.840876))") },
+{"id": 26, "name": "Missouri", "boundary": st_geom_from_text("POLYGON ((-95.773549 40.578205, -91.729115 40.61364, -91.419422 40.378264, -91.510322 40.127994, -91.370009 39.732524, -90.72995999999999 39.255894, -90.657254 38.920269999999995, -90.10940699999999 38.843548, -90.35955899999999 38.224525, -89.51425499999999 37.689923, -89.38393699999999 37.046441, -89.099594 36.964543, -89.15908 36.666351999999996, -89.464153 36.457189, -89.56318499999999 36.568749, -89.534745 36.252576, -89.70351099999999 36.243412, -89.591605 36.144096, -89.706932 36.000980999999996, -90.37789 35.995683, -90.06398 36.303038, -90.152481 36.497952, -94.617919 36.499414, -94.588413 39.149868999999995, -95.109304 39.542285, -94.862943 39.742993999999996, -94.929574 39.888754, -95.40726 40.033111999999996, -95.773549 40.578205))") },
+{"id": 27, "name": "North Carolina", "boundary": st_geom_from_text("MULTIPOLYGON (((-75.52756300011173 35.26974874385652, -75.52344599999999 35.773568, -75.521782 35.271777, -75.52756300011173 35.26974874385652)), ((-84.31749939310801 35.0217784890312, -84.29024 35.225572, -84.052612 35.269982, -83.880074 35.518744999999996, -82.992053 35.773948, -82.637165 36.065805, -82.557874 35.953901, -82.033141 36.120422, -81.707438 36.335170999999995, -81.67753499999999 36.588117, -75.86704399999999 36.550754, -75.533012 35.787377, -76.00262599999999 36.537113999999995, -75.79397399999999 36.071709999999996, -76.185401 36.301099, -76.064224 36.143775, -76.45469299999999 36.192799, -76.30400399999999 36.094527, -76.58042499999999 36.010554, -76.699978 36.285391, -76.726598 35.943234, -76.05396499999999 35.987095, -76.03804 35.645765, -75.836283 35.970501, -75.73476099999999 35.625675, -76.157355 35.327019, -76.53217599999999 35.401254, -76.46538799999999 35.558414, -76.63794399999999 35.512962, -76.577558 35.387575999999996, -77.05317 35.535208, -76.468909 35.281483, -76.79060055335052 34.97612606557105, -76.246961 34.987347, -76.512686 34.720333, -77.125863 34.685241999999995, -77.582323 34.400506, -77.96279299999999 33.842316, -78.54203 33.851924, -79.675299 34.804744, -80.79754299999999 34.819786, -81.04362499999999 35.149877, -82.39292999999999 35.215402, -83.108606 35.000659, -84.31749939310801 35.0217784890312), (-76.82001743959837 34.97551888945656, -77.05958799999999 35.146865999999996, -76.935756 34.97313, -76.82001743959837 34.97551888945656)))") },
+{"id": 28, "name": "Oklahoma", "boundary": st_geom_from_text("POLYGON ((-103.002565 36.526588, -103.002199 37.000104, -94.617964 36.998905, -94.430662 35.392478, -94.48751399999999 33.628938999999995, -95.21763299999999 33.962646, -96.348306 33.686378999999995, -96.981337 33.956378, -97.12610199999999 33.716941, -97.210921 33.916064, -97.967777 33.88243, -98.109462 34.154111, -98.486328 34.062598, -99.189511 34.214312, -99.36961 34.458698999999996, -99.69497 34.378333, -100.000381 34.560508999999996, -100.000406 36.499702, -103.002565 36.526588))") },
+{"id": 29, "name": "Virginia", "boundary": st_geom_from_text("MULTIPOLYGON (((-76.02479943943091 37.2626855899573, -75.756897 37.901553, -75.242266 38.027209, -75.832004 37.174974999999996, -75.971876 37.085376, -76.02479943943091 37.2626855899573)), ((-83.675395 36.600784, -83.136395 36.743088, -81.968012 37.538035, -81.67820999999999 37.201482999999996, -80.309331 37.50288, -79.649075 38.591515, -79.282971 38.418095, -78.998171 38.847353, -78.869276 38.762991, -78.403697 39.167451, -78.347087 39.466012, -77.82815699999999 39.132329, -77.56659599999999 39.306121, -77.51992899999999 39.120925, -77.040599 38.871212, -77.31728799999999 38.383576, -77.041524 38.400186, -76.962311 38.214075, -76.23672499999999 37.889174, -76.361887 37.609017, -76.927185 37.984978, -76.29803199999999 37.559867, -76.276214 37.310829, -76.412893 37.418079999999996, -76.354554 37.272331, -76.509035 37.239478999999996, -76.304272 37.001377999999995, -76.63641318866935 37.11849077871624, -76.48336499999999 36.878153999999995, -75.996252 36.922047, -75.86704399999999 36.550754, -83.675395 36.600784)))") },
+{"id": 30, "name": "West Virginia", "boundary": st_geom_from_text("POLYGON ((-82.6431981036679 38.1690897960737, -82.59348 38.421821, -82.177267 38.603784, -82.221566 38.787186999999996, -82.035963 39.025478, -81.762659 38.924121, -81.683627 39.270939, -80.88036 39.620706, -80.59989499999999 40.317668999999995, -80.62717099999999 40.619935999999996, -80.518991 40.638801, -80.519342 39.721402999999995, -79.47666199999999 39.721078, -79.486873 39.205960999999995, -78.76584 39.648486999999996, -78.471166 39.516103, -78.107834 39.682137, -77.833568 39.602936, -77.736409 39.392683999999996, -77.82815699999999 39.132329, -78.347087 39.466012, -78.403697 39.167451, -78.869276 38.762991, -78.998171 38.847353, -79.282971 38.418095, -79.649075 38.591515, -80.29978899999999 37.508271, -80.947896 37.295871999999996, -81.849949 37.285227, -82.6431981036679 38.1690897960737))") },
+{"id": 31, "name": "Louisiana", "boundary": st_geom_from_text("MULTIPOLYGON (((-94.04314699999999 32.693031, -94.042964 33.019219, -91.09693 32.986412, -91.15131799999999 32.615919, -91.014286 32.640482, -90.987202 32.495663, -91.116008 32.48314, -90.875631 32.372434, -91.164171 32.196888, -91.004106 32.146152, -91.345714 31.842861, -91.26340599999999 31.754468, -91.515462 31.630371999999998, -91.40541499999999 31.576466, -91.654027 31.255753, -91.63694199999999 30.999416, -89.729616 31.003926999999997, -89.851889 30.661199, -89.524504 30.180753, -89.852312 29.977649999999997, -89.650027 29.862143999999997, -89.48442299999999 30.078833, -89.370063 29.892128, -89.24784799999999 29.997328, -89.385702 29.835486999999997, -89.285742 29.763391, -89.683455 29.549266, -89.187834 29.342029999999998, -89.066617 29.090714, -89.417901 28.928608999999998, -89.29502699999999 29.198867, -89.842641 29.318823, -89.59545399999999 29.356015, -89.85197 29.475734, -90.222869 29.087069, -90.440227 29.349375, -90.836107 29.066063, -91.288324 29.255743, -91.12629 29.333420999999998, -91.54137999999999 29.526327, -91.62789599999999 29.741079, -91.88074999999999 29.710839, -91.83080799999999 29.829269999999998, -92.203312 29.753401999999998, -92.008896 29.612944, -92.323465 29.531496999999998, -93.21344099999999 29.775622, -93.837971 29.690618999999998, -93.92799199999999 29.809639999999998, -93.699396 30.05925, -93.740253 30.539569, -93.508292 31.032352, -94.041833 31.992402, -94.04314699999999 32.693031)), ((-89.342163 30.059172, -89.185715 30.157000999999998, -89.201584 30.001428999999998, -89.342163 30.059172)))") },
+{"id": 32, "name": "Michigan", "boundary": st_geom_from_text("MULTIPOLYGON (((-86.824828 41.760239999999996, -86.247078 42.490969, -86.220582 42.955791, -86.53849699999999 43.617501, -86.514702 44.058119, -86.066745 44.905685, -85.540748 45.210906, -85.652355 44.849092, -85.474937 44.991721, -85.527192 44.7475, -85.36141099999999 45.287095, -84.914543 45.395623, -85.108848 45.518485, -84.94052599999999 45.721832, -85.049835 45.769605999999996, -84.72886 45.788145, -83.385104 45.274195, -83.260657 45.025894, -83.455535 45.025082, -83.270288 44.70878, -83.333708 44.336579, -83.952642 43.749567, -83.674404 43.587222999999994, -83.402535 43.916968999999995, -82.963861 44.068266, -82.738992 43.989506, -82.415937 43.005555, -82.523337 42.607486, -82.679059 42.52221, -82.63085099999999 42.673341, -82.80560899999999 42.648765, -83.45383199999999 41.732647, -86.824828 41.760239999999996)), ((-83.88286699999999 45.975452999999995, -83.70011099999999 46.103867, -83.473221 45.984421, -83.88286699999999 45.975452999999995)), ((-90.41813599999999 46.566094, -88.972802 47.002095999999995, -88.218153 47.449521, -87.80118399999999 47.473301, -87.71242099999999 47.4014, -88.349952 47.076377, -88.496479 46.756493999999996, -88.14295399999999 46.966902999999995, -88.283423 46.822987, -87.816794 46.891154, -87.35941 46.503451999999996, -86.644706 46.410801, -86.161681 46.669475, -84.956248 46.771893999999996, -85.015211 46.479712, -84.128925 46.530119, -84.097766 46.256512, -84.273018 46.201451, -83.89546399999999 45.986436, -84.656567 46.052654, -84.751987 45.839605, -85.50621 46.096393, -86.275768 45.943551, -86.614266 45.600121, -86.717512 45.680291, -86.535206 45.885805, -86.967855 45.668261, -86.977941 45.905623, -87.592041 45.094252, -87.736509 45.173389, -87.657349 45.368752, -87.888052 45.354696999999994, -87.782226 45.683053, -88.12946099999999 45.809287999999995, -88.102908 45.921869, -90.12048899999999 46.336852, -90.41813599999999 46.566094)), ((-85.628221 45.601745, -85.501267 45.754414999999995, -85.491727 45.607527, -85.628221 45.601745)), ((-89.2628 47.869864, -88.418244 48.180369999999996, -88.911665 47.891344, -89.2628 47.869864)))") },
+{"id": 33, "name": "Massachusetts", "boundary": st_geom_from_text("MULTIPOLYGON (((-73.50814199999999 42.086256999999996, -73.264957 42.74594, -71.29420499999999 42.69699, -70.902768 42.88653, -70.591469 42.639821, -71.041333 42.302926, -70.76633799999999 42.254742, -70.412476 41.744397, -70.008462 41.800785999999995, -70.245385 42.063733, -70.082624 42.054657, -69.928393 41.708449, -70.003861 41.540551, -70.014211 41.671971, -70.65662499999999 41.514779, -70.718739 41.73574, -71.03836199999999 41.481105, -71.381401 42.018798, -73.50814199999999 42.086256999999996)), ((-70.234054 41.285652999999996, -70.049053 41.391701999999995, -69.96018099999999 41.264545999999996, -70.234054 41.285652999999996)), ((-70.838777 41.347209, -70.603555 41.482383999999996, -70.44623299999999 41.39648, -70.838777 41.347209)))") },
+{"id": 34, "name": "Idaho", "boundary": st_geom_from_text("POLYGON ((-117.24267499999999 44.396547999999996, -116.463635 45.602785, -116.85979499999999 45.907264, -117.06263 46.352522, -117.032351 48.999188, -116.049193 49.000912, -116.04885 47.977185999999996, -115.722626 47.694894, -115.62910000000001 47.47672, -115.759233 47.423471, -115.321032 47.256105, -114.76689 46.696901, -114.320665 46.646963, -114.527096 46.146218, -114.388243 45.88234, -114.566172 45.773863999999996, -114.564691 45.558026999999996, -114.333218 45.459316, -113.935505 45.694646999999996, -113.454858 44.865572, -113.131453 44.772836999999996, -112.855395 44.359975, -112.286187 44.568472, -111.467804 44.53902, -111.381665 44.754405, -111.048974 44.474072, -111.046689 42.001567, -117.026197 41.99989, -117.032881 43.830025, -116.894083 44.160191, -117.24267499999999 44.396547999999996))") },
+{"id": 35, "name": "Florida", "boundary": st_geom_from_text("MULTIPOLYGON (((-87.6348442887866 30.866109347267003, -87.598928 30.997456999999997, -85.0019 31.000681, -84.864693 30.711541999999998, -82.214839 30.568590999999998, -82.161757 30.357851, -82.036825 30.377883999999998, -81.949787 30.827492999999997, -81.42584599999999 30.700452, -81.253877 29.776851999999998, -80.57424 28.585338999999998, -80.571995 28.111594, -80.031362 26.796339, -80.130697 25.764269, -80.306682 25.612786, -80.358183 25.153228, -80.65769499999999 24.897354, -80.465501 25.211755, -81.087927 25.116170999999998, -81.290328 25.687506, -81.729489 25.909408, -81.868983 26.378648, -82.183813 26.687796, -82.08208599999999 26.653601, -82.05380199999999 26.939798999999997, -82.261941 26.717084999999997, -82.74574799999999 27.538833999999998, -82.393383 27.837519, -82.68674299999999 28.030200999999998, -82.58694799999999 27.819616999999997, -82.73499199999999 27.610839, -82.849126 27.863200000000003, -82.696013 28.930820999999998, -82.813902 29.162502, -83.056542 29.129908999999998, -83.680791 29.921574, -84.20558899999999 30.114323, -84.437597 29.988129999999998, -84.349066 29.896811999999997, -85.351514 29.666669, -85.405052 29.938487, -86.188741 30.334187999999997, -87.51832399999999 30.280434999999997, -87.396177 30.650454, -87.6348442887866 30.866109347267003)), ((-81.815793 24.562739, -81.44351 24.813364, -81.29689499999999 24.655382, -81.815793 24.562739)))") },
+{"id": 36, "name": "Kansas", "boundary": st_geom_from_text("POLYGON ((-102.051744 40.003077999999995, -94.943867 39.898129999999995, -94.862943 39.742993999999996, -95.109304 39.542285, -94.588413 39.149868999999995, -94.617964 36.998905, -102.042089 36.993016, -102.051744 40.003077999999995))") },
+{"id": 37, "name": "New Hampshire", "boundary": st_geom_from_text("POLYGON ((-72.55724699999999 42.853018999999996, -72.065434 44.277235, -71.577643 44.502691999999996, -71.443365 45.237724, -71.083924 45.305451, -70.98764899999999 43.389520999999995, -70.712085 43.043724999999995, -71.29420499999999 42.69699, -72.458519 42.726853, -72.55724699999999 42.853018999999996))") },
+{"id": 38, "name": "Delaware", "boundary": st_geom_from_text("POLYGON ((-75.78865590755069 39.659333499406, -75.66282199999999 39.821149999999996, -75.405414 39.796396, -75.59232899999999 39.467577999999996, -75.30407799999999 38.91316, -75.09201399999999 38.803902, -75.04893899999999 38.451263, -75.69367 38.46008, -75.78865590755069 39.659333499406))") },
+{"id": 39, "name": "Texas", "boundary": st_geom_from_text("MULTIPOLYGON (((-106.645479 31.89867, -106.618486 32.000495, -103.064423 32.000518, -103.041924 36.500439, -100.000406 36.499702, -100.000381 34.560508999999996, -99.69497 34.378333, -99.36961 34.458698999999996, -99.189511 34.214312, -98.486328 34.062598, -98.109462 34.154111, -97.967777 33.88243, -97.210921 33.916064, -97.12610199999999 33.716941, -96.981337 33.956378, -96.348306 33.686378999999995, -95.21763299999999 33.962646, -94.38608599999999 33.544923, -94.056598 33.567825, -94.041833 31.992402, -93.50846 31.029441, -93.740253 30.539569, -93.699396 30.05925, -93.92799199999999 29.809639999999998, -93.83772499999999 29.679024, -94.778691 29.361483, -94.47070199999999 29.556582, -94.778992 29.530157, -94.754808 29.781122999999997, -94.99906399999999 29.709049, -94.86436599999999 29.370510999999997, -95.38368299999999 28.87, -96.342483 28.418716, -97.04607999999999 27.839539, -96.880061 28.131072, -96.416348 28.413871, -96.66519799999999 28.30961, -96.815055 28.475209, -96.785226 28.22954, -97.22283 28.077347999999997, -97.024962 28.113307, -97.075077 27.919354, -97.51854 27.86545, -97.243544 27.689227, -97.414436 27.321568, -97.70880199999999 27.385638, -97.74011399999999 27.267542, -97.42259299999999 27.262045, -97.445708 26.609361999999997, -97.29427199999999 26.105843, -97.14966199999999 26.064135999999998, -97.40525 25.837728, -97.662728 26.038078, -98.193985 26.053324, -99.085126 26.398781999999997, -99.446086 27.023338, -99.512219 27.568094, -100.291397 28.275396999999998, -100.674656 29.099777, -101.400884 29.770069, -102.315389 29.87992, -102.673997 29.744574, -102.86846800000001 29.222879, -103.283457 28.976944999999997, -104.509025 29.633281999999998, -104.922935 30.603848, -106.645479 31.89867)), ((-97.398388 26.867894, -97.36182099999999 27.35906, -97.05682999999999 27.841994, -97.365386 27.199887, -97.276173 26.565462999999998, -97.398388 26.867894)))") },
+{"id": 40, "name": "Minnesota", "boundary": st_geom_from_text("POLYGON ((-97.239209 48.968683999999996, -95.153711 48.998903, -95.153314 49.384358, -94.957465 49.370186, -94.816222 49.320986999999995, -94.645387 48.744079, -93.794454 48.516020999999995, -92.954876 48.631493, -92.634931 48.542873, -92.71256199999999 48.463013, -92.45632499999999 48.414204, -92.369174 48.220268, -92.055228 48.359213, -91.56725399999999 48.043718999999996, -90.88548 48.245784, -90.75160799999999 48.090968, -89.491739 48.005212, -90.777003 47.60623, -92.08534 46.795767, -92.01529 46.706469, -92.291292 46.668141999999996, -92.294033 46.074377, -92.869193 45.717568, -92.883749 45.575483, -92.646602 45.441635, -92.807317 44.750364, -91.43252199999999 43.996826999999996, -91.21770599999999 43.50055, -96.45326 43.500389999999996, -96.45306699999999 45.298114999999996, -96.857751 45.605962, -96.583085 45.820024, -96.554507 46.083977999999995, -96.797941 46.629113, -96.851051 47.59794, -97.146667 48.143203, -97.090328 48.684512, -97.239209 48.968683999999996))") },
+{"id": 41, "name": "Connecticut", "boundary": st_geom_from_text("POLYGON ((-73.727775 41.100696, -73.482709 41.212759999999996, -73.487314 42.049638, -71.80064999999999 42.023568999999995, -71.857458 41.320789, -72.91308699999999 41.296737, -73.657336 40.985171, -73.727775 41.100696))") },
+{"id": 42, "name": "New Jersey", "boundary": st_geom_from_text("POLYGON ((-75.559102 39.629056, -74.72260399999999 40.150009999999995, -75.20181199999999 40.617188, -75.051029 40.865662, -75.130575 40.991093, -74.79504 41.320406999999996, -73.893979 40.997197, -74.27269 40.488405, -73.982855 40.448569, -74.10334399999999 39.751646, -74.792723 38.991991, -74.967274 38.933413, -74.88716699999999 39.158825, -75.151405 39.189996, -75.559102 39.629056))") },
+{"id": 43, "name": "North Dakota", "boundary": st_geom_from_text("POLYGON ((-104.048652 48.865733999999996, -104.048736 48.999877, -97.22872199999999 49.000561999999995, -97.090328 48.684512, -97.146667 48.143203, -96.851051 47.59794, -96.797941 46.629113, -96.562525 45.937087, -104.045443 45.94531, -104.048652 48.865733999999996))") },
+{"id": 44, "name": "Maryland", "boundary": st_geom_from_text("POLYGON ((-79.48764992808078 39.2799539963283, -79.47666199999999 39.721078, -75.788596 39.722198999999996, -75.69367 38.46008, -75.054591 38.414829999999995, -75.242266 38.027209, -75.885032 37.911716999999996, -75.85027199999999 38.366394, -75.97026 38.233806, -76.011419 38.37683, -76.032044 38.216684, -76.226376 38.309988, -76.286102 38.625656, -76.026512 38.56682, -76.22461799999999 38.760387, -76.339709 38.670642, -76.255016 38.862268, -76.155334 38.77234, -76.21000000000001 38.946314, -76.368195 38.836124999999996, -76.361727 38.939175, -76.163988 38.999541, -76.274637 39.16549, -75.948909 39.593177, -76.281747 39.299659999999996, -76.357124 39.394079, -76.442482 39.195408, -76.586485 39.261337, -76.394498 39.012684, -76.560011 38.762626, -76.322158 38.037901999999995, -77.01637099999999 38.445572, -77.25017199999999 38.382781, -76.909395 38.892812, -77.46145 39.075151, -77.838008 39.606125, -78.176625 39.695966999999996, -78.565929 39.519444, -78.76584 39.648486999999996, -79.48764992808078 39.2799539963283))") },
+{"id": 45, "name": "Maine", "boundary": st_geom_from_text("POLYGON ((-71.08000299999999 45.306987, -70.85704199999999 45.22916, -70.798028 45.426705999999996, -70.634661 45.383607999999995, -70.72284499999999 45.512772, -70.259117 45.890755, -70.29273599999999 46.191599, -69.997086 46.695229999999995, -69.22442 47.459686, -68.90008499999999 47.177718999999996, -68.23460399999999 47.355035, -67.790515 47.067921, -67.817892 45.693705, -67.429716 45.583773, -67.489333 45.281282, -67.15791899999999 45.161004, -66.949895 44.817419, -67.564718 44.532067999999995, -67.847776 44.563, -68.049334 44.330729999999996, -68.211426 44.52014, -68.36568 44.435345, -68.17449599999999 44.345181, -68.333831 44.221286, -68.425338 44.498275, -68.522786 44.228099, -68.827197 44.31216, -68.805863 44.524291999999996, -68.998384 44.425585999999996, -69.125281 43.977914999999996, -69.43773399999999 43.975750999999995, -69.655611 43.781039, -69.677078 43.926984, -69.836799 43.699735, -70.045006 43.736667999999995, -69.950647 43.86327, -70.19426899999999 43.768601, -70.196911 43.565146, -70.703819 43.059825, -70.98764899999999 43.389520999999995, -71.08000299999999 45.306987))") },
+{"id": 46, "name": "Rhode Island", "boundary": st_geom_from_text("MULTIPOLYGON (((-71.19951625965957 41.60319584997731, -71.132888 41.660101999999995, -71.12057 41.497448, -71.19951625965957 41.60319584997731)), ((-71.8626677422346 41.310863612697, -71.79924199999999 42.008064999999995, -71.381401 42.018798, -71.23844199999999 41.665565, -71.390557 41.783708, -71.480875 41.360248, -71.8626677422346 41.310863612697)))") },
+{"id": 47, "name": "California", "boundary": st_geom_from_text("POLYGON ((-124.408601 40.443200999999995, -124.111756 41.026883, -124.21160499999999 41.99846, -119.999168 41.99454, -120.001014 38.999573999999996, -114.633487 35.001857, -114.386699 34.457910999999996, -114.131211 34.26273, -114.434181 34.087379, -114.524599 33.552231, -114.72528199999999 33.405048, -114.706488 33.08816, -114.469039 32.972294999999995, -114.526856 32.757093999999995, -117.12486200000001 32.534155999999996, -117.46979400000001 33.296417, -118.132698 33.753217, -118.411211 33.741985, -118.519514 34.027509, -119.129252 34.10084, -119.563986 34.41532, -120.63672199999999 34.560967, -120.644311 35.139616, -121.902729 36.306379, -121.862266 36.931551999999996, -122.405073 37.195791, -122.478083 37.810828, -122.044771 37.457249, -122.430087 37.963114999999995, -122.282824 38.082889, -122.489974 38.112013999999995, -122.527399 37.814947, -123.024066 37.994878, -122.977082 38.267902, -123.727976 38.919429, -123.851714 39.832041, -124.408601 40.443200999999995))") },
+{"id": 48, "name": "New York", "boundary": st_geom_from_text("POLYGON ((-79.762152 42.243054, -78.853455 42.783958, -79.070469 43.262454, -76.787949 43.311309, -76.209853 43.560136, -76.297392 43.855925, -76.059062 43.9857, -76.20164 44.079038999999995, -76.280677 43.959683, -76.312647 44.199044, -75.283136 44.849156, -74.826578 45.01585, -73.343124 45.010839999999995, -73.431229 43.588285, -73.24158899999999 43.534973, -73.264957 42.74594, -73.50814199999999 42.086256999999996, -73.482709 41.212759999999996, -73.727775 41.100696, -73.65597199999999 40.979597, -73.81520499999999 40.831075, -72.278789 41.158722, -72.11618064373404 41.00478221051866, -73.940591 40.542896, -74.042412 40.624846999999995, -73.893979 40.997197, -74.983341 41.480894, -75.072172 41.813731999999995, -75.359579 41.999445, -79.76137399999999 41.999067, -79.762152 42.243054))") },
+{"id": 49, "name": "District of Columbia", "boundary": st_geom_from_text("POLYGON ((-77.119759 38.934343, -76.909395 38.892812, -77.039006 38.791644999999995, -77.119759 38.934343))") }
+]);
+
+// ----------- Insert data into POIS
+
+use SJTest;
+
+INSERT INTO POIS ([
+{"id": 477940459990302720, "latitude": 39.60127691, "longitude": -75.08210896 },
+{"id": 477790131575730177, "latitude": 40.1514752, "longitude": -74.81236042 },
+{"id": 477710471814848512, "latitude": 33.43804694, "longitude": -95.68763757 },
+{"id": 477985242800996353, "latitude": 41.48453081, "longitude": -81.52077033 },
+{"id": 478030228334198784, "latitude": 41.39446897, "longitude": -81.77584306 },
+{"id": 477924841790922752, "latitude": 36.07628074, "longitude": -79.79048387 },
+{"id": 477970695121432576, "latitude": 41.53870828, "longitude": -74.08805952 },
+{"id": 477862757627592704, "latitude": 35.7346058, "longitude": -81.30704639 },
+{"id": 478023082238816256, "latitude": 35.3657508, "longitude": -119.0102226 },
+{"id": 477903515004993536, "latitude": 41.22221807, "longitude": -73.19275067 },
+{"id": 477938785984843776, "latitude": 36.24723439, "longitude": -115.05323046 },
+{"id": 477993123273252864, "latitude": 35.14151039, "longitude": -90.03867381 },
+{"id": 477951332804861952, "latitude": 33.87416095, "longitude": -118.35009287 },
+{"id": 477893925420957697, "latitude": 41.78922283, "longitude": -71.510315 },
+{"id": 477800021468151808, "latitude": 39.9847605, "longitude": -75.1554912 },
+{"id": 477976590979063809, "latitude": 31.3646508, "longitude": -110.95371188 },
+{"id": 477934575625043968, "latitude": 38.6834504, "longitude": -121.2700596 },
+{"id": 477895250854154240, "latitude": 34.31816446, "longitude": -118.48024392 },
+{"id": 477693917203165184, "latitude": 33.03928163, "longitude": -96.88638643 },
+{"id": 477987651723612161, "latitude": 36.9600352, "longitude": -120.0363063 },
+{"id": 477784297613430785, "latitude": 36.42884804, "longitude": -76.36722257 },
+{"id": 478001570689216512, "latitude": 41.63920715, "longitude": -87.56199455 },
+{"id": 477708824313204737, "latitude": 34.03503475, "longitude": -118.3755469 },
+{"id": 477697371217068033, "latitude": 40.69421039, "longitude": -73.70531126 },
+{"id": 477712027977150465, "latitude": 30.8511237, "longitude": -93.2950018 },
+{"id": 477893366458228736, "latitude": 37.62058509, "longitude": -109.47617019 },
+{"id": 478028298979536897, "latitude": 33.87267676, "longitude": -117.74513179 },
+{"id": 477985347683364864, "latitude": 33.84093782, "longitude": -118.39112049 },
+{"id": 477890422337847296, "latitude": 45.5816052, "longitude": -122.3602295 },
+{"id": 478015380498022401, "latitude": 39.28363031, "longitude": -76.62181533 },
+{"id": 477745839117721600, "latitude": 31.36526636, "longitude": -89.29842984 },
+{"id": 477816226769801216, "latitude": 40.06843354, "longitude": -74.86433339 },
+{"id": 477872171566391300, "latitude": 30.9128876, "longitude": -83.2392699 },
+{"id": 477866362866774016, "latitude": 38.9064913, "longitude": -77.0416139 },
+{"id": 477913033713659905, "latitude": 45.5057, "longitude": -73.5608 },
+{"id": 477721488758276096, "latitude": 34.1109045, "longitude": -117.3457078 },
+{"id": 477868622904172545, "latitude": 33.25337717, "longitude": -81.94603765 },
+{"id": 477838810013638656, "latitude": 33.89785895, "longitude": -118.31028598 },
+{"id": 477933599845810176, "latitude": 43.20424045, "longitude": -79.84003499 },
+{"id": 478007349232160768, "latitude": 30.49127254, "longitude": -90.4802828 },
+{"id": 477817866981371905, "latitude": 49.7371959, "longitude": -96.87045848 },
+{"id": 477987831831597057, "latitude": 40.05282192, "longitude": -75.13428426 },
+{"id": 477714409964331008, "latitude": 33.42498148, "longitude": -86.62742911 },
+{"id": 477997717017608192, "latitude": 39.56420777, "longitude": -76.99351121 },
+{"id": 477806861245091840, "latitude": 33.22548108, "longitude": -82.5657175 },
+{"id": 477931439271706625, "latitude": 40.261022, "longitude": -76.79750594 },
+{"id": 478040747015696384, "latitude": 37.3259826, "longitude": -77.4417275 },
+{"id": 477972909109936128, "latitude": 34.00351798, "longitude": -118.48500139 },
+{"id": 478021564642783232, "latitude": 33.92465156, "longitude": -118.13826456 },
+{"id": 477823067670847488, "latitude": 38.9824508, "longitude": -76.4949692 },
+{"id": 477990849377497089, "latitude": 41.52279668, "longitude": -81.62979662 },
+{"id": 477946764641964032, "latitude": 34.05945275, "longitude": -84.08681443 },
+{"id": 477942878690234369, "latitude": 33.88460724, "longitude": -117.49207187 },
+{"id": 477902253169184768, "latitude": 38.7696517, "longitude": -89.9656719 },
+{"id": 478021172173733888, "latitude": 39.0642047, "longitude": -84.4711534 },
+{"id": 478025089892372480, "latitude": 38.16958174, "longitude": -85.87278843 },
+{"id": 477806769431797761, "latitude": 33.94027909, "longitude": -118.40648334 },
+{"id": 477717169103331329, "latitude": 34.01226862, "longitude": -118.09915817 },
+{"id": 478014983616618497, "latitude": 39.60295926, "longitude": -105.02260514 },
+{"id": 477729349500563456, "latitude": 37.99818121, "longitude": -84.21115136 },
+{"id": 477685246914465792, "latitude": 40.09554088, "longitude": -82.95717482 },
+{"id": 477865432121348096, "latitude": 39.76031722, "longitude": -86.1575784 },
+{"id": 477689754977181696, "latitude": 47.23433434, "longitude": -122.15083003 },
+{"id": 477955520066617345, "latitude": 33.30739744, "longitude": -111.97886346 },
+{"id": 478031251405021184, "latitude": 40.33253123, "longitude": -76.80739109 },
+{"id": 477878662235435008, "latitude": 39.0890013, "longitude": -95.6884688 },
+{"id": 477807102228836353, "latitude": 34.74548518, "longitude": -81.94682579 },
+{"id": 477886749923094528, "latitude": 40.82511353, "longitude": -73.85390859 },
+{"id": 477864102120161280, "latitude": 35.8529023, "longitude": -95.42135767 },
+{"id": 477688305509597184, "latitude": 39.73074608, "longitude": -74.90067266 },
+{"id": 477844975372476416, "latitude": 39.32013315, "longitude": -84.54873581 },
+{"id": 478019144693329920, "latitude": 41.21432831, "longitude": -80.54341614 },
+{"id": 477963661403619329, "latitude": 33.6433111, "longitude": -112.118585 },
+{"id": 477969815672328194, "latitude": 33.72222418, "longitude": -92.64973803 },
+{"id": 477841342911885312, "latitude": 39.28065676, "longitude": -76.56681334 },
+{"id": 477889624178581506, "latitude": 36.95340629, "longitude": -120.06700063 },
+{"id": 477940249486565376, "latitude": 36.34748719, "longitude": -94.21057687 },
+{"id": 477933871195901952, "latitude": 35.58765105, "longitude": -78.35785957 },
+{"id": 477682766164000768, "latitude": 40.6634903, "longitude": -74.1110378 },
+{"id": 477958125048180736, "latitude": 40.79365954, "longitude": -77.85999475 },
+{"id": 477960384885051393, "latitude": 43.74163153, "longitude": -79.55500957 },
+{"id": 478018844431511552, "latitude": 40.85231804, "longitude": -73.97942773 },
+{"id": 477935367392620544, "latitude": 41.80048501, "longitude": -71.8879753 },
+{"id": 477798558406414336, "latitude": 43.5698948, "longitude": -79.6691208 },
+{"id": 478003401713025024, "latitude": 43.62132814, "longitude": -116.34896264 },
+{"id": 477679655206813696, "latitude": 40.74323463, "longitude": -73.98569682 },
+{"id": 477812331200913408, "latitude": 31.2783036, "longitude": -92.4307188 },
+{"id": 477968197203664896, "latitude": 33.04786211, "longitude": -117.29755178 },
+{"id": 477952606695088128, "latitude": 43.65619562, "longitude": -79.3805137 },
+{"id": 478021704476655616, "latitude": 37.6368882, "longitude": -122.0984205 },
+{"id": 477702943513534465, "latitude": 40.79229131, "longitude": -74.32971485 },
+{"id": 477860289577488384, "latitude": 42.43418214, "longitude": -83.04946121 },
+{"id": 477810175772856320, "latitude": 35.10625596, "longitude": -77.04778736 },
+{"id": 477780073995464704, "latitude": 43.6928174, "longitude": -87.7125278 },
+{"id": 477875035651457026, "latitude": 40.87373665, "longitude": -74.17612931 },
+{"id": 477833996580040705, "latitude": 32.4191746, "longitude": -81.7869042 },
+{"id": 477901252748001280, "latitude": 42.6607329, "longitude": -71.3024034 },
+{"id": 477993581190598656, "latitude": 34.61968507, "longitude": -79.00082282 },
+{"id": 477918999255474176, "latitude": 31.20446667, "longitude": -87.602705 },
+{"id": 478001129473597440, "latitude": 33.66173133, "longitude": -117.99760789 },
+{"id": 477741840129462272, "latitude": 41.72580357, "longitude": -83.60777901 },
+{"id": 478022065073946625, "latitude": 41.89967688, "longitude": -87.76559374 },
+{"id": 477899807814877184, "latitude": 42.2539919, "longitude": -71.82697122 },
+{"id": 477867002460401664, "latitude": 38.90826031, "longitude": -77.04231262 },
+{"id": 477714969463496705, "latitude": 32.59926679, "longitude": -96.76870759 },
+{"id": 478022159198326784, "latitude": 41.88569827, "longitude": -87.61582761 },
+{"id": 477871658762387456, "latitude": 41.59578485, "longitude": -93.66029367 },
+{"id": 477812649544417280, "latitude": 40.77836073, "longitude": -73.51308269 },
+{"id": 478016151213965312, "latitude": 35.6259867, "longitude": -105.9808948 },
+{"id": 477955249861173250, "latitude": 38.6633284, "longitude": -121.49791509 },
+{"id": 478018657273249792, "latitude": 41.10775421, "longitude": -85.05600725 },
+{"id": 478033837243772928, "latitude": 32.72854627, "longitude": -117.22587121 },
+{"id": 477939053807955969, "latitude": 42.74245328, "longitude": -84.39348084 },
+{"id": 477732786380365824, "latitude": 38.96133218, "longitude": -95.31464709 },
+{"id": 478008443056717825, "latitude": 39.96700038, "longitude": -74.26460732 },
+{"id": 477972538111188992, "latitude": 35.3887115, "longitude": -119.1309104 },
+{"id": 477962976960315392, "latitude": 33.8754065, "longitude": -117.7533813 },
+{"id": 477789249195225088, "latitude": 38.20716416, "longitude": -84.81514522 },
+{"id": 477802144251531264, "latitude": 37.44351927, "longitude": -77.66935801 },
+{"id": 477909725905174529, "latitude": 47.66852744, "longitude": -122.38511719 },
+{"id": 477891111860449280, "latitude": 32.90800808, "longitude": -117.13767925 },
+{"id": 477898631903973376, "latitude": 30.53614936, "longitude": -90.47798467 },
+{"id": 477943243775045632, "latitude": 36.15965854, "longitude": -115.24152072 },
+{"id": 477722811058778112, "latitude": 38.6748329, "longitude": -121.5338647 },
+{"id": 477709178467655680, "latitude": 38.88810024, "longitude": -121.31877659 },
+{"id": 477966296764542976, "latitude": 32.95175169, "longitude": -96.61296843 },
+{"id": 478000266617552897, "latitude": 36.02849819, "longitude": -115.0692286 },
+{"id": 477880874927931393, "latitude": 40.5444743, "longitude": -74.1574014 },
+{"id": 478028311302795264, "latitude": 40.551354, "longitude": -74.1802795 },
+{"id": 477950147872038915, "latitude": 39.84905545, "longitude": -75.4374292 },
+{"id": 477907262560694272, "latitude": 42.36665316, "longitude": -71.06260883 },
+{"id": 477869023099887616, "latitude": 39.96663803, "longitude": -75.12878428 },
+{"id": 478019511153487872, "latitude": 34.1518669, "longitude": -118.4227387 },
+{"id": 477924570469371904, "latitude": 41.28260956, "longitude": -96.16403055 },
+{"id": 477850569672261635, "latitude": 34.7919172, "longitude": -86.7285245 },
+{"id": 477684266776539136, "latitude": 32.6644321, "longitude": -116.96162864 },
+{"id": 477854131801972738, "latitude": 42.98238002, "longitude": -88.2592917 },
+{"id": 477991260201164801, "latitude": 45.18785731, "longitude": -93.33917043 },
+{"id": 477922136296681472, "latitude": 40.99099099, "longitude": -79.10798206 },
+{"id": 477866271548379136, "latitude": 42.1006064, "longitude": -80.09492712 },
+{"id": 477693328704540672, "latitude": 33.9731721, "longitude": -118.2353379 },
+{"id": 478032616969367552, "latitude": 33.0902049, "longitude": -81.9617307 },
+{"id": 477990464235122688, "latitude": 34.16036746, "longitude": -118.53782801 },
+{"id": 477890625056931841, "latitude": 36.85581682, "longitude": -119.67278405 },
+{"id": 477998138829987840, "latitude": 37.81051453, "longitude": -85.4503716 },
+{"id": 477833117374611456, "latitude": 44.11614436, "longitude": -103.06577797 },
+{"id": 478013367010209792, "latitude": 40.78099404, "longitude": -77.85107683 },
+{"id": 477846708165279744, "latitude": 41.06365442, "longitude": -80.055738 },
+{"id": 477990927747674112, "latitude": 34.0200987, "longitude": -118.3250881 },
+{"id": 477716262395867136, "latitude": 39.31396817, "longitude": -76.67261942 },
+{"id": 477958377176182784, "latitude": 32.20618574, "longitude": -110.90207834 },
+{"id": 477911776831369216, "latitude": 43.34336886, "longitude": -72.48533459 },
+{"id": 478011971321991169, "latitude": 40.8305601, "longitude": -73.93521622 },
+{"id": 477904525836046337, "latitude": 32.8397361, "longitude": -97.0111221 },
+{"id": 477917202776330241, "latitude": 40.78331945, "longitude": -73.21867713 },
+{"id": 477996255675576320, "latitude": 32.20294, "longitude": -98.2399004 },
+{"id": 477923101528043520, "latitude": 40.71424858, "longitude": -73.95375967 },
+{"id": 477809256251793408, "latitude": 33.94389398, "longitude": -118.40502262 },
+{"id": 478017662329823232, "latitude": 40.9444786, "longitude": -73.88956896 },
+{"id": 477836726254063616, "latitude": 33.53821433, "longitude": -84.56882506 },
+{"id": 478035545424019457, "latitude": 34.03297096, "longitude": -118.4859452 },
+{"id": 477867896610123776, "latitude": 41.50693914, "longitude": -82.94852757 },
+{"id": 478017597758136322, "latitude": 45.16530905, "longitude": -93.23935113 },
+{"id": 477967996007104512, "latitude": 41.2941292, "longitude": -72.9562981 },
+{"id": 477924929136906241, "latitude": 30.193654, "longitude": -97.7663298 },
+{"id": 477964119467757568, "latitude": 42.3404497, "longitude": -83.205684 },
+{"id": 478032297955168256, "latitude": 40.8503476, "longitude": -73.9401381 },
+{"id": 477859620191760384, "latitude": 32.23433126, "longitude": -110.79989419 },
+{"id": 477964238627958784, "latitude": 36.64489682, "longitude": -77.86652518 },
+{"id": 477783354054815744, "latitude": 42.9476236, "longitude": -78.8207316 },
+{"id": 477689514463211520, "latitude": 38.5300082, "longitude": -121.4826188 },
+{"id": 478005025185464320, "latitude": 42.86847305, "longitude": -78.79903624 },
+{"id": 477986693471952897, "latitude": 31.75989014, "longitude": -106.31561299 },
+{"id": 477957785909735424, "latitude": 39.81871193, "longitude": -75.53023171 },
+{"id": 477890178640384001, "latitude": 37.5688636, "longitude": -77.4540628 },
+{"id": 478004308827717632, "latitude": 39.14933024, "longitude": -84.43623134 },
+{"id": 478029048799846401, "latitude": 40.3030824, "longitude": -121.228368 },
+{"id": 477918605024428032, "latitude": 41.3849, "longitude": -71.8324 },
+{"id": 477704344804917248, "latitude": 34.04481618, "longitude": -117.64472726 },
+{"id": 478017044638474240, "latitude": 38.2475201, "longitude": -121.3203015 },
+{"id": 477858541366415360, "latitude": 39.9505, "longitude": -75.1665 },
+{"id": 478040810793861121, "latitude": 37.7854956, "longitude": -122.4156964 },
+{"id": 477699581149343746, "latitude": 34.14515531, "longitude": -118.10567066 },
+{"id": 477683342683676672, "latitude": 41.47743349, "longitude": -90.56032738 },
+{"id": 477692432629260288, "latitude": 34.2438489, "longitude": -77.896849 },
+{"id": 477793948002308096, "latitude": 32.219705, "longitude": -106.71692 },
+{"id": 477804456218984448, "latitude": 38.5387125, "longitude": -75.05452752 },
+{"id": 477911363524644864, "latitude": 34.178989, "longitude": -118.5970172 },
+{"id": 477878260219777024, "latitude": 40.74851475, "longitude": -73.04856675 },
+{"id": 477831827236073472, "latitude": 40.69118179, "longitude": -74.01678085 },
+{"id": 477962480053149697, "latitude": 41.52291792, "longitude": -73.9371046 },
+{"id": 477686459692883969, "latitude": 34.02691118, "longitude": -118.27842855 },
+{"id": 477935614818398208, "latitude": 32.6797551, "longitude": -117.0694914 },
+{"id": 478033494506209280, "latitude": 43.42099013, "longitude": -70.61556979 },
+{"id": 477966082389471233, "latitude": 31.8765154, "longitude": -102.3162344 },
+{"id": 477985210265763841, "latitude": 34.16597277, "longitude": -118.58611045 },
+{"id": 478039363477987328, "latitude": 33.07497432, "longitude": -96.72300287 },
+{"id": 477926631617478656, "latitude": 33.36047754, "longitude": -111.88298489 },
+{"id": 477893937122668544, "latitude": 30.47940996, "longitude": -84.16244847 },
+{"id": 477931754834386944, "latitude": 41.86256496, "longitude": -80.55896709 },
+{"id": 477908713605300224, "latitude": 42.14364648, "longitude": -72.49528381 },
+{"id": 477916962819829760, "latitude": 34.34233036, "longitude": -78.69287931 },
+{"id": 477704963724828672, "latitude": 38.30213214, "longitude": -122.31423652 },
+{"id": 478037978233966593, "latitude": 40.70936438, "longitude": -73.93719625 },
+{"id": 477689651637940224, "latitude": 32.51559582, "longitude": -96.37546796 },
+{"id": 478037659471056896, "latitude": 40.76366855, "longitude": -73.98612947 },
+{"id": 477871126472040448, "latitude": 45.3971171, "longitude": -75.8488709 },
+{"id": 477852612054634496, "latitude": 30.3019185, "longitude": -97.692977 },
+{"id": 477863043632619521, "latitude": 42.6427867, "longitude": -71.2991179 },
+{"id": 477885901591560192, "latitude": 32.62953799, "longitude": -83.66239552 },
+{"id": 477998671141679105, "latitude": 33.5644901, "longitude": -86.7190393 },
+{"id": 477681555175788544, "latitude": 35.43671433, "longitude": -80.73635268 },
+{"id": 477936827597193216, "latitude": 33.95645024, "longitude": -117.23417698 },
+{"id": 477871895246036992, "latitude": 43.25049676, "longitude": -83.77933309 },
+{"id": 477960076796252162, "latitude": 38.4268189, "longitude": -76.4814515 },
+{"id": 477949120313704448, "latitude": 43.7233751, "longitude": -79.668652 },
+{"id": 477960961555300352, "latitude": 36.71289148, "longitude": -121.62034351 },
+{"id": 477881528337190912, "latitude": 35.97806149, "longitude": -86.56547779 },
+{"id": 477940766870761473, "latitude": 40.03622527, "longitude": -76.31628094 },
+{"id": 478029034325311488, "latitude": 39.28700945, "longitude": -76.73488088 },
+{"id": 477989075082022912, "latitude": 41.7648877, "longitude": -87.8062973 },
+{"id": 477808000292233216, "latitude": 32.40897032, "longitude": -97.76105025 },
+{"id": 477828719420399616, "latitude": 37.98200177, "longitude": -87.63579244 },
+{"id": 477737856010440705, "latitude": 34.02855818, "longitude": -117.63851157 },
+{"id": 477889079963435008, "latitude": 32.91214498, "longitude": -96.98220881 },
+{"id": 477914081425244160, "latitude": 47.14988004, "longitude": -91.45364942 },
+{"id": 477901638854651904, "latitude": 39.64296332, "longitude": -106.38823919 },
+{"id": 477808001420906496, "latitude": 30.10967478, "longitude": -93.74674875 },
+{"id": 477916586230030336, "latitude": 35.08500927, "longitude": -80.63319144 },
+{"id": 477884640330788864, "latitude": 41.85938228, "longitude": -88.07347735 },
+{"id": 477708778998341632, "latitude": 36.28222236, "longitude": -115.28191601 },
+{"id": 478017816306925568, "latitude": 39.43706326, "longitude": -74.51741 },
+{"id": 477815334171598848, "latitude": 42.82934794, "longitude": -86.06309999 },
+{"id": 478023453639852032, "latitude": 38.64903476, "longitude": -85.32328796 },
+{"id": 477869761255469056, "latitude": 36.1290158, "longitude": -115.2039619 },
+{"id": 477961053658030081, "latitude": 33.63903871, "longitude": -111.97120395 },
+{"id": 477728353068544000, "latitude": 35.58133872, "longitude": -97.52597323 },
+{"id": 477814781617774592, "latitude": 30.3800325, "longitude": -97.6586255 },
+{"id": 477880679393263616, "latitude": 33.99768365, "longitude": -117.81735398 },
+{"id": 477864520723857409, "latitude": 40.4862157, "longitude": -74.4518188 },
+{"id": 477978576126758912, "latitude": 43.58957788, "longitude": -89.7833777 },
+{"id": 477886101089435648, "latitude": 41.5882, "longitude": -93.618 },
+{"id": 477991631627370498, "latitude": 33.848431, "longitude": -118.2463685 },
+{"id": 477885219609903104, "latitude": 32.4897064, "longitude": -117.0136619 },
+{"id": 478027685638471680, "latitude": 38.96354105, "longitude": -76.92634514 },
+{"id": 477694229112557568, "latitude": 33.47165251, "longitude": -112.20569405 },
+{"id": 478024512710385664, "latitude": 43.68755429, "longitude": -116.50153714 },
+{"id": 477747973481172992, "latitude": 36.78027399, "longitude": -119.8180974 },
+{"id": 477960059607998464, "latitude": 43.88204911, "longitude": -78.87653078 },
+{"id": 477955234455490560, "latitude": 42.5750374, "longitude": -87.8708912 },
+{"id": 478029562719113216, "latitude": 42.52663914, "longitude": -89.00081445 },
+{"id": 477881374448156672, "latitude": 42.1268789, "longitude": -71.0327824 },
+{"id": 478020628604149760, "latitude": 49.18901966, "longitude": -122.80202127 },
+{"id": 477718935035338752, "latitude": 33.87307559, "longitude": -118.33737052 },
+{"id": 478006089540067328, "latitude": 37.73652416, "longitude": -92.13005704 },
+{"id": 477946928043266048, "latitude": 38.6605063, "longitude": -121.5250138 },
+{"id": 477679903953805312, "latitude": 42.31194193, "longitude": -89.65558919 },
+{"id": 477910847822372864, "latitude": 40.73670468, "longitude": -74.17601316 },
+{"id": 477817401862397952, "latitude": 40.150897, "longitude": -79.894714 },
+{"id": 478015581082648576, "latitude": 40.73899742, "longitude": -73.6696815 },
+{"id": 478012257704480768, "latitude": 47.21223346, "longitude": -122.33665738 },
+{"id": 477938305670324224, "latitude": 32.3855, "longitude": -111.0466 },
+{"id": 477940480378810368, "latitude": 45.48674588, "longitude": -122.40913865 },
+{"id": 477895062404096002, "latitude": 33.50478853, "longitude": -86.70687112 },
+{"id": 478024163622264832, "latitude": 47.52106365, "longitude": -122.37185796 },
+{"id": 477841460553736192, "latitude": 35.0816837, "longitude": -78.9869528 },
+{"id": 477683062763831296, "latitude": 32.86618361, "longitude": -96.61647096 },
+{"id": 477922673616781312, "latitude": 36.7057576, "longitude": -76.3243453 },
+{"id": 477679237822808065, "latitude": 39.94075903, "longitude": -75.12939813 },
+{"id": 477874666712084480, "latitude": 30.44173236, "longitude": -81.6804043 },
+{"id": 477980125913694208, "latitude": 37.32848401, "longitude": -92.90783981 },
+{"id": 477823838739513344, "latitude": 33.82483429, "longitude": -118.24761714 },
+{"id": 477684417167515648, "latitude": 41.69814072, "longitude": -96.88632899 },
+{"id": 478007609312550912, "latitude": 40.70376663, "longitude": -82.17407774 },
+{"id": 477872254022582272, "latitude": 43.02719906, "longitude": -86.21020662 },
+{"id": 477916881102176257, "latitude": 34.42728759, "longitude": -117.37805184 },
+{"id": 477833128745385985, "latitude": 42.12772081, "longitude": -80.0672699 },
+{"id": 478015574547505152, "latitude": 30.0007518, "longitude": -90.217556 },
+{"id": 477971501677699072, "latitude": 43.02911891, "longitude": -87.97134476 },
+{"id": 477951280607166464, "latitude": 43.65606148, "longitude": -79.38195362 },
+{"id": 477999708829655041, "latitude": 40.7149536, "longitude": -74.0453423 },
+{"id": 477924226322956288, "latitude": 36.0158683, "longitude": -115.2972531 },
+{"id": 477847809715347457, "latitude": 37.75060823, "longitude": -80.48052168 },
+{"id": 477989669922435073, "latitude": 42.8669016, "longitude": -97.4728847 },
+{"id": 477846095716823042, "latitude": 30.21276451, "longitude": -85.69263827 },
+{"id": 477966887934902272, "latitude": 47.65861153, "longitude": -122.29909985 },
+{"id": 478038541537398784, "latitude": 35.20701204, "longitude": -101.88164148 },
+{"id": 478037318792925184, "latitude": 37.90198479, "longitude": -88.90947425 },
+{"id": 477944624179212289, "latitude": 37.97198685, "longitude": -121.32616046 },
+{"id": 477859473764016128, "latitude": 41.54484094, "longitude": -73.08943819 },
+{"id": 478039474820362241, "latitude": 39.2442511, "longitude": -84.7950769 },
+{"id": 477930677032058880, "latitude": 41.77537712, "longitude": -96.51016546 },
+{"id": 477906070375903233, "latitude": 33.4642541, "longitude": -81.858114 },
+{"id": 478014494703362048, "latitude": 38.76565397, "longitude": -77.61999165 },
+{"id": 477679191379292161, "latitude": 38.89951723, "longitude": -76.94373268 },
+{"id": 477857109627764736, "latitude": 35.93967971, "longitude": -77.81812104 },
+{"id": 477702281974910977, "latitude": 33.8708481, "longitude": -84.2592263 },
+{"id": 477708653127286784, "latitude": 42.2900927, "longitude": -71.115806 },
+{"id": 477967544435752960, "latitude": 38.93534126, "longitude": -94.66083593 },
+{"id": 477806076851916800, "latitude": 36.96689778, "longitude": -76.28720994 },
+{"id": 477923329069051904, "latitude": 40.6650153, "longitude": -74.2083941 },
+{"id": 478021449945718786, "latitude": 37.3259858, "longitude": -77.4416771 },
+{"id": 477835408584105984, "latitude": 35.4670096, "longitude": -86.04341405 },
+{"id": 477700960332939265, "latitude": 36.9881388, "longitude": -86.4781627 },
+{"id": 477696717539008512, "latitude": 45.519154, "longitude": -122.49645 },
+{"id": 477721971031949312, "latitude": 36.1070567, "longitude": -119.5711719 },
+{"id": 478012599649714176, "latitude": 41.70133207, "longitude": -88.11422568 },
+{"id": 477987657637560320, "latitude": 46.47830059, "longitude": -84.39180655 },
+{"id": 477875364107018240, "latitude": 32.36933551, "longitude": -99.77232272 },
+{"id": 477686220302991361, "latitude": 40.42093252, "longitude": -86.88332247 },
+{"id": 477887449600131073, "latitude": 40.57814206, "longitude": -75.5314676 },
+{"id": 478027760091553793, "latitude": 36.82417816, "longitude": -76.34214668 },
+{"id": 477842972419977216, "latitude": 32.525376, "longitude": -92.6474791 },
+{"id": 477973536825286656, "latitude": 34.6681647, "longitude": -82.7973042 },
+{"id": 478016419091599360, "latitude": 39.94987668, "longitude": -75.16192627 },
+{"id": 477984741216759808, "latitude": 37.71247565, "longitude": -97.44235258 },
+{"id": 477715971440791552, "latitude": 34.40640852, "longitude": -118.4558852 },
+{"id": 477953705749143552, "latitude": 37.73936105, "longitude": -122.47090816 },
+{"id": 477977501135015936, "latitude": 38.89622565, "longitude": -77.02710089 },
+{"id": 477922679081533440, "latitude": 34.0692007, "longitude": -117.1638077 },
+{"id": 477940900006334464, "latitude": 37.7463808, "longitude": -97.2505981 },
+{"id": 477802539425857536, "latitude": 42.18297032, "longitude": -71.22980197 },
+{"id": 477933301169000449, "latitude": 42.54147457, "longitude": -82.95565193 },
+{"id": 477943923617595397, "latitude": 40.52876834, "longitude": -79.91823689 },
+{"id": 477962491817783296, "latitude": 43.7771782, "longitude": -79.3229375 },
+{"id": 477701543886479361, "latitude": 38.36181239, "longitude": -75.08291012 },
+{"id": 477889349422297088, "latitude": 41.15297682, "longitude": -81.33378674 },
+{"id": 478026106399109120, "latitude": 41.5916251, "longitude": -81.5438 },
+{"id": 477920624904777728, "latitude": 39.1827483, "longitude": -84.2735235 },
+{"id": 478013550049247232, "latitude": 34.184408, "longitude": -118.6097329 },
+{"id": 477894254334050304, "latitude": 40.52618265, "longitude": -74.26938183 },
+{"id": 477910977766117376, "latitude": 30.359091, "longitude": -97.6832213 },
+{"id": 477989992061751296, "latitude": 32.95903647, "longitude": -80.17978705 },
+{"id": 477683852891025408, "latitude": 33.60219383, "longitude": -117.09650995 },
+{"id": 477895846122360832, "latitude": 32.69790599, "longitude": -95.10131946 },
+{"id": 477865794332684288, "latitude": 41.40713658, "longitude": -73.41789354 },
+{"id": 477715332476309504, "latitude": 45.0189949, "longitude": -85.6293453 },
+{"id": 477891425015980032, "latitude": 40.50608209, "longitude": -74.48621007 },
+{"id": 477740274676797441, "latitude": 40.03450759, "longitude": -75.10988387 },
+{"id": 477923380520161280, "latitude": 32.87843258, "longitude": -96.77219664 },
+{"id": 477937296453693440, "latitude": 42.074832, "longitude": -76.8014393 },
+{"id": 477959018078806017, "latitude": 40.77892456, "longitude": -73.20423195 },
+{"id": 478031294979260417, "latitude": 31.63027507, "longitude": -94.65266814 },
+{"id": 478018226123980800, "latitude": 39.59079377, "longitude": -77.65006762 },
+{"id": 477812515456704513, "latitude": 38.74031462, "longitude": -77.45205249 },
+{"id": 477949972286300160, "latitude": 33.75940171, "longitude": -84.39260365 },
+{"id": 477833044405915649, "latitude": 34.0647469, "longitude": -118.2960192 },
+{"id": 477710580308922368, "latitude": 49.1272886, "longitude": -125.8929667 },
+{"id": 477915909881729024, "latitude": 39.92360248, "longitude": -75.17068578 },
+{"id": 477804154220326912, "latitude": 39.31917015, "longitude": -76.56109658 },
+{"id": 477987885749403648, "latitude": 30.16022865, "longitude": -81.8395121 },
+{"id": 477949357421916160, "latitude": 43.86372963, "longitude": -79.01990986 },
+{"id": 478039843415810048, "latitude": 43.54679215, "longitude": -116.27268833 },
+{"id": 477820700083118080, "latitude": 38.46800746, "longitude": -77.40823852 },
+{"id": 477841919154728961, "latitude": 37.47434778, "longitude": -121.15318998 },
+{"id": 478006306104950786, "latitude": 41.36053392, "longitude": -72.11628047 },
+{"id": 477915536051810304, "latitude": 33.86704179, "longitude": -117.45907674 },
+{"id": 477700221951279104, "latitude": 37.30572696, "longitude": -79.97569383 },
+{"id": 477853463628771330, "latitude": 42.25562863, "longitude": -71.79026453 },
+{"id": 478033061599526912, "latitude": 36.07474708, "longitude": -91.91118264 },
+{"id": 477928443309355009, "latitude": 40.75664227, "longitude": -73.84577442 },
+{"id": 477840034188374017, "latitude": 43.1828623, "longitude": -86.2178809 },
+{"id": 478035831039733760, "latitude": 40.976158, "longitude": -78.5494174 },
+{"id": 477925041238073345, "latitude": 31.343054, "longitude": -110.9306708 },
+{"id": 477879667559395328, "latitude": 33.2068828, "longitude": -97.1441011 },
+{"id": 478031287563726848, "latitude": 32.7805712, "longitude": -96.8013638 },
+{"id": 477953389243138048, "latitude": 43.657183, "longitude": -79.4048378 },
+{"id": 477923509394366464, "latitude": 32.78399561, "longitude": -96.78645948 },
+{"id": 478026951266152448, "latitude": 36.95598761, "longitude": -78.96751297 },
+{"id": 478034571888304129, "latitude": 34.20523713, "longitude": -118.46686103 },
+{"id": 477849977520017409, "latitude": 40.8938323, "longitude": -74.1534622 },
+{"id": 477704957844398080, "latitude": 45.4481055, "longitude": -122.8030394 },
+{"id": 478013302510223361, "latitude": 39.99449054, "longitude": -83.00707676 },
+{"id": 477693051306250240, "latitude": 33.51003026, "longitude": -117.66815791 },
+{"id": 478006104094289921, "latitude": 35.25699106, "longitude": -81.04538179 },
+{"id": 477959740371120128, "latitude": 39.86039966, "longitude": -104.66651148 },
+{"id": 477737777425948672, "latitude": 36.13212297, "longitude": -115.20877856 },
+{"id": 477793105693769730, "latitude": 38.63529486, "longitude": -77.28855702 },
+{"id": 477834692268265472, "latitude": 44.89255, "longitude": -93.560317 },
+{"id": 478041279411281923, "latitude": 35.6151194, "longitude": -95.8587043 },
+{"id": 477931004557271041, "latitude": 38.83187312, "longitude": -77.20031518 },
+{"id": 477851224583049217, "latitude": 32.603274, "longitude": -82.1726845 },
+{"id": 477833345288523779, "latitude": 37.3909263, "longitude": -79.1511182 },
+{"id": 477881790594842624, "latitude": 35.91243722, "longitude": -81.5342698 },
+{"id": 477847264317431808, "latitude": 42.56238644, "longitude": -70.93504692 },
+{"id": 477681137700315136, "latitude": 34.75278481, "longitude": -77.37944689 },
+{"id": 478005539256164353, "latitude": 33.64044148, "longitude": -84.21523372 },
+{"id": 477720222481133568, "latitude": 32.73694721, "longitude": -117.18815526 },
+{"id": 477929756495589377, "latitude": 37.5482697, "longitude": -121.9885719 },
+{"id": 478014124442804224, "latitude": 35.4573321, "longitude": -97.6897606 },
+{"id": 478037377621843968, "latitude": 37.40124077, "longitude": -121.88260612 },
+{"id": 477689490903818240, "latitude": 34.01702946, "longitude": -117.70389905 },
+{"id": 477978053738758144, "latitude": 40.83293194, "longitude": -89.61476044 },
+{"id": 477876360883757056, "latitude": 40.81168398, "longitude": -77.89093791 },
+{"id": 478038216365203457, "latitude": 43.1005535, "longitude": -87.9765008 },
+{"id": 477922498940383232, "latitude": 41.29872066, "longitude": -86.10756881 },
+{"id": 477981779119587328, "latitude": 41.96992555, "longitude": -88.11481077 },
+{"id": 477869030083031040, "latitude": 35.6105322, "longitude": -87.1188455 },
+{"id": 477906243512983552, "latitude": 38.10951926, "longitude": -85.66456668 },
+{"id": 478011577271349248, "latitude": 39.01555629, "longitude": -84.44549088 },
+{"id": 477824714555924480, "latitude": 42.41110249, "longitude": -82.96886977 },
+{"id": 477844535293521920, "latitude": 40.75501062, "longitude": -72.87116395 },
+{"id": 478040453196300288, "latitude": 33.42546641, "longitude": -93.94259811 },
+{"id": 478021027218604032, "latitude": 42.00710863, "longitude": -88.376404 },
+{"id": 477969951924711425, "latitude": 36.84423907, "longitude": -76.02972834 },
+{"id": 478015268997070848, "latitude": 42.91019458, "longitude": -71.93817455 },
+{"id": 477872648941092865, "latitude": 31.83119168, "longitude": -106.4522023 },
+{"id": 477839945948598272, "latitude": 41.523705, "longitude": -88.0621931 },
+{"id": 477738578647474176, "latitude": 45.5239424, "longitude": -122.4934875 },
+{"id": 477804574137282560, "latitude": 35.85695515, "longitude": -86.45832902 },
+{"id": 477912240134172672, "latitude": 34.42363667, "longitude": -118.5700607 },
+{"id": 478023475656146944, "latitude": 42.11965623, "longitude": -83.21522175 },
+{"id": 477833462162796545, "latitude": 42.81293909, "longitude": -83.62068722 },
+{"id": 477994175926128641, "latitude": 39.9766276, "longitude": -82.8323485 },
+{"id": 477833593410957312, "latitude": 35.0029031, "longitude": -109.8994866 },
+{"id": 477909903764623361, "latitude": 46.999543, "longitude": -122.9154661 },
+{"id": 477977383757443073, "latitude": 32.94489976, "longitude": -96.71110036 },
+{"id": 477921634242682880, "latitude": 45.0715679, "longitude": -122.7956733 },
+{"id": 478039054777196545, "latitude": 38.8888726, "longitude": -121.3303283 },
+{"id": 477730927368355841, "latitude": 33.9552782, "longitude": -118.2408319 },
+{"id": 477819605260726273, "latitude": 39.35694782, "longitude": -74.43962706 },
+{"id": 478032564079190016, "latitude": 38.4482793, "longitude": -121.4077054 },
+{"id": 477681306726187009, "latitude": 33.5735512, "longitude": -92.8597276 },
+{"id": 477884541479055360, "latitude": 44.998804, "longitude": -93.3691498 },
+{"id": 478006814244876288, "latitude": 40.9059708, "longitude": -72.6504527 },
+{"id": 477870333102686212, "latitude": 40.85433896, "longitude": -73.96549076 },
+{"id": 477863486761209856, "latitude": 40.80429434, "longitude": -73.51790909 },
+{"id": 477720889057083392, "latitude": 41.88218853, "longitude": -71.35800177 },
+{"id": 478025907870113792, "latitude": 30.31159092, "longitude": -81.40721603 },
+{"id": 477701642171600896, "latitude": 35.42502865, "longitude": -78.06728159 },
+{"id": 477899848537366529, "latitude": 40.27521492, "longitude": -75.31873602 },
+{"id": 477955854990204928, "latitude": 30.59461799, "longitude": -81.4443254 },
+{"id": 477813748410089472, "latitude": 41.07650509, "longitude": -73.47121227 },
+{"id": 477967832891031553, "latitude": 39.88944784, "longitude": -75.05843529 },
+{"id": 477681877440925696, "latitude": 42.28811608, "longitude": -71.08818083 },
+{"id": 477892535227277312, "latitude": 30.10121455, "longitude": -81.71418034 },
+{"id": 477932514959302656, "latitude": 40.61480581, "longitude": -74.12057337 },
+{"id": 477830405140127744, "latitude": 39.16547145, "longitude": -74.68362831 },
+{"id": 478019244714893312, "latitude": 35.4877422, "longitude": -97.52519488 },
+{"id": 477721002646855680, "latitude": 33.0363669, "longitude": -96.58739288 },
+{"id": 477885357070229504, "latitude": 34.02587356, "longitude": -118.51587228 },
+{"id": 477688558275145730, "latitude": 33.03915996, "longitude": -86.06779662 },
+{"id": 478003749085265920, "latitude": 41.5750104, "longitude": -85.41041474 },
+{"id": 477807072944209920, "latitude": 39.7400441, "longitude": -104.9130089 },
+{"id": 477706138020892672, "latitude": 33.94631089, "longitude": -118.4043573 },
+{"id": 478006693155340289, "latitude": 41.05890327, "longitude": -111.91771414 },
+{"id": 477689569958051842, "latitude": 32.2389299, "longitude": -82.4742658 },
+{"id": 477975131432955904, "latitude": 40.272784, "longitude": -76.4388682 },
+{"id": 477685984440483842, "latitude": 43.0258155, "longitude": -63.95460103 },
+{"id": 478008926407888896, "latitude": 38.77914746, "longitude": -90.84552906 },
+{"id": 477905139823439872, "latitude": 33.68180385, "longitude": -83.70442416 },
+{"id": 477974987848945664, "latitude": 42.53526925, "longitude": -89.00805836 },
+{"id": 477926503598915584, "latitude": 34.13708715, "longitude": -118.19883757 },
+{"id": 477912631861600256, "latitude": 33.42076929, "longitude": -111.89644197 },
+{"id": 477931221280759808, "latitude": 37.80407791, "longitude": -97.42378542 },
+{"id": 477995918466494465, "latitude": 34.78536, "longitude": -84.29049214 },
+{"id": 477917785687744513, "latitude": 42.49227535, "longitude": -83.06498246 },
+{"id": 477848521379688448, "latitude": 39.95729031, "longitude": -82.87729983 },
+{"id": 477884092592037888, "latitude": 41.1029498, "longitude": -96.2632202 },
+{"id": 478004543964577792, "latitude": 35.3584555, "longitude": -118.9419798 },
+{"id": 477782571338575872, "latitude": 40.75262931, "longitude": -73.96454583 },
+{"id": 477703025273085952, "latitude": 40.73406911, "longitude": -73.98913616 },
+{"id": 477850385218957313, "latitude": 36.02145973, "longitude": -115.04584417 },
+{"id": 477696804117438464, "latitude": 36.82008078, "longitude": -79.35815811 },
+{"id": 478014221868101632, "latitude": 42.30332615, "longitude": -71.06604786 },
+{"id": 477987540532989954, "latitude": 40.9985147, "longitude": -73.6625676 },
+{"id": 477997038190469120, "latitude": 35.74327395, "longitude": -78.80894506 },
+{"id": 477927598212009984, "latitude": 33.97777343, "longitude": -117.63978576 },
+{"id": 477901135986978816, "latitude": 30.2804815, "longitude": -97.5916354 },
+{"id": 477825076641808385, "latitude": 41.6633483, "longitude": -72.7970243 },
+{"id": 477963020488818688, "latitude": 34.0279153, "longitude": -81.0495523 },
+{"id": 477962006050660352, "latitude": 42.2760525, "longitude": -83.7371931 },
+{"id": 477886472654049280, "latitude": 45.45785305, "longitude": -73.88391371 },
+{"id": 477941241405923328, "latitude": 36.3252727, "longitude": -119.2853102 },
+{"id": 477821395896770561, "latitude": 42.06442604, "longitude": -72.51375533 },
+{"id": 478000173617266688, "latitude": 41.90213216, "longitude": -87.68941599 },
+{"id": 478030516365828096, "latitude": 45.01114859, "longitude": -93.17993584 },
+{"id": 477892295128526848, "latitude": 31.9802205, "longitude": -81.1282672 },
+{"id": 477722149050789888, "latitude": 40.5788972, "longitude": -124.1531515 },
+{"id": 478036872804184064, "latitude": 41.53978015, "longitude": -82.74496501 },
+{"id": 477857985562042368, "latitude": 34.05129333, "longitude": -118.44551518 },
+{"id": 477956309493370880, "latitude": 40.44131579, "longitude": -79.75252542 },
+{"id": 477999944020684801, "latitude": 31.565754, "longitude": -93.4679604 },
+{"id": 477855636978671616, "latitude": 40.8290351, "longitude": -73.8708547 },
+{"id": 477875705225547779, "latitude": 43.59498378, "longitude": -79.63495071 },
+{"id": 477826739012648960, "latitude": 45.36749268, "longitude": -63.26566511 },
+{"id": 477852569922842624, "latitude": 38.90377527, "longitude": -95.07549802 },
+{"id": 477900745233424385, "latitude": 38.9375421, "longitude": -76.8653582 },
+{"id": 477960113756848128, "latitude": 41.7241626, "longitude": -86.2862621 },
+{"id": 478035142091743232, "latitude": 39.66848403, "longitude": -74.97448465 },
+{"id": 477937643569704960, "latitude": 42.7612236, "longitude": -84.6011455 },
+{"id": 477981869167112192, "latitude": 39.41689477, "longitude": -112.04119633 },
+{"id": 477932452292624384, "latitude": 40.8016407, "longitude": -73.96458146 },
+{"id": 477828413516820480, "latitude": 32.446395, "longitude": -81.79521189 },
+{"id": 478005197550010368, "latitude": 42.66080952, "longitude": -83.01093682 },
+{"id": 477862458233593857, "latitude": 33.62701309, "longitude": -116.19609073 },
+{"id": 478025850244190209, "latitude": 38.0249385, "longitude": -121.2692275 },
+{"id": 478015907525320704, "latitude": 36.00296147, "longitude": -84.05465191 },
+{"id": 477868388467740672, "latitude": 39.27877864, "longitude": -76.5540912 },
+{"id": 478008669427093504, "latitude": 40.92022351, "longitude": -81.11319998 },
+{"id": 477836954583580673, "latitude": 40.02967814, "longitude": -105.28528816 },
+{"id": 477892977935659010, "latitude": 30.13274593, "longitude": -85.21059028 },
+{"id": 477819758856138752, "latitude": 35.8654932, "longitude": -80.83020725 },
+{"id": 477970567337742336, "latitude": 42.1209816, "longitude": -86.4448199 },
+{"id": 478011502298157056, "latitude": 41.35714627, "longitude": -71.82063027 },
+{"id": 477967178713415681, "latitude": 38.9184209, "longitude": -75.4320446 },
+{"id": 477939969709715456, "latitude": 40.28459332, "longitude": -73.98832278 },
+{"id": 478013412983595009, "latitude": 33.6649998, "longitude": -117.98332926 },
+{"id": 477692062188699648, "latitude": 40.77465125, "longitude": -81.44462724 },
+{"id": 477841690909491200, "latitude": 40.66033578, "longitude": -74.21891073 },
+{"id": 477931242001031169, "latitude": 40.94763417, "longitude": -74.01744001 },
+{"id": 477689097222250496, "latitude": 38.42465473, "longitude": -78.86788293 },
+{"id": 477948871570886657, "latitude": 43.2239656, "longitude": -70.64082961 },
+{"id": 477986275245719552, "latitude": 39.62006841, "longitude": -74.62283981 },
+{"id": 478023135451951104, "latitude": 30.07024112, "longitude": -82.18488372 },
+{"id": 477921674612858880, "latitude": 32.7876578, "longitude": -97.2927821 },
+{"id": 477984868434202624, "latitude": 30.3703089, "longitude": -87.17656047 },
+{"id": 477992914044583939, "latitude": 41.43183868, "longitude": -81.88434938 },
+{"id": 478004192091848704, "latitude": 40.72297742, "longitude": -73.62874718 },
+{"id": 477860801240260609, "latitude": 40.60311741, "longitude": -74.1164178 },
+{"id": 477908448575635457, "latitude": 38.90723, "longitude": -77.03646 },
+{"id": 477750610884710400, "latitude": 36.21981122, "longitude": -115.27830704 },
+{"id": 477849506508070912, "latitude": 47.67013065, "longitude": -117.21192836 },
+{"id": 477810694029844480, "latitude": 39.18793704, "longitude": -77.2360181 },
+{"id": 477804335342968832, "latitude": 40.75797231, "longitude": -84.14871152 },
+{"id": 477762134231625728, "latitude": 38.56389577, "longitude": -90.23932911 },
+{"id": 477861457233580033, "latitude": 41.7607942, "longitude": -72.5786939 },
+{"id": 477863164609314816, "latitude": 39.98330946, "longitude": -74.90191153 },
+{"id": 477892492201693184, "latitude": 38.8991, "longitude": -77.029 },
+{"id": 477885272273977344, "latitude": 33.44147661, "longitude": -112.32643786 },
+{"id": 478029728470007808, "latitude": 45.42561567, "longitude": -122.63202214 },
+{"id": 477819673858174980, "latitude": 30.51889661, "longitude": -97.86986855 },
+{"id": 477911583423602688, "latitude": 45.57933146, "longitude": -122.11746066 },
+{"id": 478015512384110592, "latitude": 41.2123398, "longitude": -95.9605852 },
+{"id": 477697263058157569, "latitude": 35.27988499, "longitude": -106.6787443 },
+{"id": 477885770657566721, "latitude": 37.43760221, "longitude": -122.16855565 },
+{"id": 478032820376330240, "latitude": 41.58117395, "longitude": -87.43612762 },
+{"id": 477880676998733824, "latitude": 40.87303975, "longitude": -74.09579514 },
+{"id": 477693276544180225, "latitude": 30.90952018, "longitude": -95.28666623 },
+{"id": 478030723258286081, "latitude": 40.76852168, "longitude": -73.48983425 },
+{"id": 477804919169118209, "latitude": 33.8783047, "longitude": -84.3049567 },
+{"id": 477863507812032512, "latitude": 41.41113422, "longitude": -81.71346126 },
+{"id": 477941768063700992, "latitude": 47.62774309, "longitude": -122.15105284 },
+{"id": 478033930763792384, "latitude": 34.12235321, "longitude": -118.15340892 },
+{"id": 477973543112568832, "latitude": 39.42576236, "longitude": -76.93263529 },
+{"id": 477932768891244544, "latitude": 36.15721616, "longitude": -86.77653507 },
+{"id": 477960791329501184, "latitude": 32.70511077, "longitude": -80.87004475 },
+{"id": 477821007948812289, "latitude": 35.38283769, "longitude": -80.31561322 },
+{"id": 478020227348037632, "latitude": 35.4705269, "longitude": -78.60037993 },
+{"id": 477707911418744833, "latitude": 32.9195773, "longitude": -96.6785502 },
+{"id": 477941510550216704, "latitude": 40.33157069, "longitude": -74.59944787 },
+{"id": 477825651987062784, "latitude": 43.59777, "longitude": -79.62502 },
+{"id": 477796926084382720, "latitude": 30.36764925, "longitude": -81.49484334 },
+{"id": 477863819684106241, "latitude": 43.1242853, "longitude": -75.2157009 },
+{"id": 477975783395581952, "latitude": 43.47977147, "longitude": -80.52413855 },
+{"id": 477934425078894592, "latitude": 32.8546197, "longitude": -79.9748103 },
+{"id": 478038123885391872, "latitude": 34.9873433, "longitude": -79.09165991 },
+{"id": 477731655101075457, "latitude": 37.50232746, "longitude": -122.48263792 },
+{"id": 477998751806533632, "latitude": 37.1835168, "longitude": -93.303003 },
+{"id": 477862315287912449, "latitude": 41.1821897, "longitude": -73.1355297 },
+{"id": 478030812701806592, "latitude": 32.077289, "longitude": -96.4898988 },
+{"id": 477708916223000576, "latitude": 37.82562238, "longitude": -120.94786414 },
+{"id": 477990233620086784, "latitude": 38.57201579, "longitude": -77.33096931 },
+{"id": 478015341961166848, "latitude": 41.11509823, "longitude": -81.57664167 },
+{"id": 477903259316027392, "latitude": 40.42165692, "longitude": -86.74859656 },
+{"id": 477990401891377152, "latitude": 40.91099694, "longitude": -73.10071349 },
+{"id": 477788380873633792, "latitude": 40.82240336, "longitude": -73.95443214 },
+{"id": 478013152572211200, "latitude": 40.6813, "longitude": -73.9772 },
+{"id": 478005085646364673, "latitude": 30.21870724, "longitude": -81.5149449 },
+{"id": 477989136813817857, "latitude": 32.39123373, "longitude": -99.77180217 },
+{"id": 477889579865743360, "latitude": 33.81061287, "longitude": -118.07017612 },
+{"id": 477968103448801280, "latitude": 42.33012517, "longitude": -71.03232922 },
+{"id": 478011808897593344, "latitude": 37.2640163, "longitude": -121.8688395 },
+{"id": 477758509040144385, "latitude": 38.72438176, "longitude": -77.0773421 },
+{"id": 477713114411335680, "latitude": 40.2420244, "longitude": -74.77220717 },
+{"id": 478016024915099649, "latitude": 37.81640718, "longitude": -122.44062774 },
+{"id": 477933552948891648, "latitude": 30.47023882, "longitude": -97.80704132 },
+{"id": 477901679845588993, "latitude": 43.02468285, "longitude": -81.27731053 },
+{"id": 477991212830318593, "latitude": 32.9091537, "longitude": -117.1731764 },
+{"id": 478005940546183168, "latitude": 42.12469128, "longitude": -83.54129517 },
+{"id": 477767496796090368, "latitude": 41.95549963, "longitude": -87.70617485 },
+{"id": 477968094292238336, "latitude": 35.22967308, "longitude": -78.80528796 },
+{"id": 477957278054637568, "latitude": 40.64573715, "longitude": -89.57805924 },
+{"id": 477972195344265218, "latitude": 34.04119593, "longitude": -118.44737881 },
+{"id": 477714057063968769, "latitude": 33.930932, "longitude": -118.0913633 },
+{"id": 477737891519815680, "latitude": 38.47514735, "longitude": -121.34138185 },
+{"id": 477882778688978944, "latitude": 35.31220342, "longitude": -80.72476877 },
+{"id": 477845223918559232, "latitude": 41.4588129, "longitude": -71.44074403 },
+{"id": 477996774209359872, "latitude": 33.2612596, "longitude": -93.306991 },
+{"id": 477737627559272449, "latitude": 33.81986675, "longitude": -116.47563254 },
+{"id": 477738480446238721, "latitude": 41.9436127, "longitude": -87.83394444 },
+{"id": 477889879473262592, "latitude": 45.5382178, "longitude": -122.86769019 },
+{"id": 477904181789859840, "latitude": 41.010881, "longitude": -92.8801369 },
+{"id": 477696617244422144, "latitude": 30.217992, "longitude": -92.0194673 },
+{"id": 478008342619914241, "latitude": 37.83817496, "longitude": -79.37187071 },
+{"id": 477717547949641728, "latitude": 34.0433939, "longitude": -117.72152543 },
+{"id": 477879088984502273, "latitude": 31.11159351, "longitude": -97.88321212 },
+{"id": 478017561192574976, "latitude": 40.2521209, "longitude": -75.6058146 },
+{"id": 477950322816466944, "latitude": 41.69450051, "longitude": -87.70955401 },
+{"id": 477982616617250816, "latitude": 43.175325, "longitude": -79.2541103 },
+{"id": 477944187170455552, "latitude": 35.1338855, "longitude": -117.941223 },
+{"id": 477886190566141952, "latitude": 42.0123734, "longitude": -88.07639936 },
+{"id": 477928693785198592, "latitude": 37.6141639, "longitude": -97.3670183 },
+{"id": 477853453075902465, "latitude": 38.90873821, "longitude": -76.99758732 },
+{"id": 477962970899546112, "latitude": 45.28657677, "longitude": -75.66564241 },
+{"id": 477814423705645056, "latitude": 45.3998637, "longitude": -75.66678805 },
+{"id": 477740444278095872, "latitude": 41.88214094, "longitude": -71.34648912 },
+{"id": 478022723990745088, "latitude": 44.07194139, "longitude": -103.13581644 },
+{"id": 477995647409610757, "latitude": 43.03833982, "longitude": -81.22934064 },
+{"id": 477691017135542274, "latitude": 41.4306693, "longitude": -81.630174 },
+{"id": 477924963878313984, "latitude": 37.0237967, "longitude": -100.9201474 },
+{"id": 477850520213012480, "latitude": 41.4378804, "longitude": -72.22860553 },
+{"id": 478031952168366081, "latitude": 43.03706829, "longitude": -87.96871257 },
+{"id": 477857551615545344, "latitude": 32.7387327, "longitude": -117.0228189 },
+{"id": 477916119332683776, "latitude": 32.51694447, "longitude": -117.11921814 },
+{"id": 477853786761740290, "latitude": 40.72220649, "longitude": -73.9862584 },
+{"id": 477935814576709632, "latitude": 32.14172888, "longitude": -110.93639367 },
+{"id": 477854655574069248, "latitude": 33.7396304, "longitude": -118.2950553 },
+{"id": 477683703771299840, "latitude": 33.48867669, "longitude": -112.00146217 },
+{"id": 478003477835427841, "latitude": 40.70954468, "longitude": -74.07359789 },
+{"id": 477854185648439296, "latitude": 41.38155478, "longitude": -72.89586907 },
+{"id": 477882954698350592, "latitude": 43.636833, "longitude": -116.6081985 },
+{"id": 477951531950424064, "latitude": 34.15367199, "longitude": -80.76188721 },
+{"id": 477790307791409152, "latitude": 38.87639503, "longitude": -99.31503737 },
+{"id": 477944770296164353, "latitude": 34.0173164, "longitude": -117.6820219 },
+{"id": 477977320452419587, "latitude": 41.7005385, "longitude": -87.6606778 },
+{"id": 477979783847227393, "latitude": 33.89193915, "longitude": -96.6522511 },
+{"id": 477923446249500673, "latitude": 41.1753127, "longitude": -81.41732027 },
+{"id": 477887439504412673, "latitude": 38.04439954, "longitude": -78.78617274 },
+{"id": 478025282834952192, "latitude": 33.5942266, "longitude": -84.2738243 },
+{"id": 478011246433017856, "latitude": 42.87697485, "longitude": -78.70296103 },
+{"id": 477868623671721985, "latitude": 42.77767135, "longitude": -71.42280545 },
+{"id": 477810941762228224, "latitude": 40.65748915, "longitude": -80.57481911 },
+{"id": 477747449801363457, "latitude": 39.913157, "longitude": -104.9288839 },
+{"id": 477986092466319360, "latitude": 43.93554472, "longitude": -78.87971814 },
+{"id": 477860058873602048, "latitude": 35.66694346, "longitude": -80.56478172 },
+{"id": 477942380424097794, "latitude": 40.83317205, "longitude": -73.85393995 },
+{"id": 478026614555824128, "latitude": 40.1309557, "longitude": -74.9037905 },
+{"id": 477964393276506112, "latitude": 32.7608, "longitude": -117.121 },
+{"id": 477696199349125120, "latitude": 42.47716315, "longitude": -92.34704179 },
+{"id": 477771163406319616, "latitude": 35.16295649, "longitude": -78.9732162 },
+{"id": 477840693138698242, "latitude": 41.03906524, "longitude": -80.66457544 },
+{"id": 478019806164422656, "latitude": 32.96934105, "longitude": -96.74958262 },
+{"id": 477849680492384256, "latitude": 43.52657332, "longitude": -79.64547903 },
+{"id": 477716495447756800, "latitude": 34.6813581, "longitude": -118.1055898 },
+{"id": 477687590980964352, "latitude": 35.35639627, "longitude": -79.00794888 },
+{"id": 477819300443463680, "latitude": 39.9684619, "longitude": -83.00041402 },
+{"id": 477702458953584640, "latitude": 32.2013375, "longitude": -110.827568 },
+{"id": 477816653929910275, "latitude": 34.19162767, "longitude": -118.44020981 },
+{"id": 478010715010510848, "latitude": 34.74220988, "longitude": -86.7669327 },
+{"id": 477883834776559616, "latitude": 39.3468812, "longitude": -76.5596071 },
+{"id": 478037581695709184, "latitude": 33.65758858, "longitude": -118.00276822 },
+{"id": 478041281038274560, "latitude": 48.45309628, "longitude": -123.36459617 },
+{"id": 477736172538441729, "latitude": 42.8651071, "longitude": -85.6528495 },
+{"id": 478016209212805120, "latitude": 30.5759031, "longitude": -91.20427019 },
+{"id": 477853444930142209, "latitude": 37.76998675, "longitude": -97.4647456 },
+{"id": 478024996791795712, "latitude": 42.76627488, "longitude": -78.60814929 },
+{"id": 478038169221615616, "latitude": 41.77156029, "longitude": -87.62571596 },
+{"id": 477685727468077057, "latitude": 45.40471588, "longitude": -122.49310428 },
+{"id": 477916455078752256, "latitude": 41.31517201, "longitude": -74.12678395 },
+{"id": 477679511803162626, "latitude": 38.57482443, "longitude": -76.08221595 },
+{"id": 477866470173851650, "latitude": 41.54514269, "longitude": -72.04034147 },
+{"id": 477887299942756352, "latitude": 34.11466223, "longitude": -81.19090891 },
+{"id": 477912464169127937, "latitude": 41.75951836, "longitude": -87.57176657 },
+{"id": 477990295108194304, "latitude": 37.69884034, "longitude": -121.93238089 },
+{"id": 478016400326684672, "latitude": 40.69624563, "longitude": -73.30107922 },
+{"id": 477924283625115648, "latitude": 47.00681646, "longitude": -124.17239897 },
+{"id": 477936624827768833, "latitude": 42.03617491, "longitude": -87.67042538 },
+{"id": 477871360312487936, "latitude": 34.1027228, "longitude": -118.1995752 },
+{"id": 477691158873636864, "latitude": 36.1510926, "longitude": -115.2004735 },
+{"id": 477893139605508096, "latitude": 41.50286778, "longitude": -74.02146667 },
+{"id": 477917510285529088, "latitude": 44.79724076, "longitude": -122.78914221 },
+{"id": 477923185749278720, "latitude": 42.48401275, "longitude": -76.49159806 },
+{"id": 478039857370238976, "latitude": 35.87047468, "longitude": -80.08365866 },
+{"id": 477682971533520898, "latitude": 38.08044418, "longitude": -89.3167345 },
+{"id": 477870233882202113, "latitude": 43.041164, "longitude": -83.5834399 },
+{"id": 477925631892549632, "latitude": 42.76294535, "longitude": -71.22246003 },
+{"id": 477858304018751489, "latitude": 33.9615135, "longitude": -118.2883809 },
+{"id": 477916047711154176, "latitude": 36.20008818, "longitude": -115.17241909 },
+{"id": 477909862450343936, "latitude": 32.44381525, "longitude": -93.71638399 },
+{"id": 478030819336802304, "latitude": 30.39736192, "longitude": -91.17470813 },
+{"id": 477881840905121793, "latitude": 33.03875631, "longitude": -97.39374513 },
+{"id": 477754937590300672, "latitude": 34.05206397, "longitude": -118.2616448 },
+{"id": 477966891915677696, "latitude": 35.5667838, "longitude": -82.54394575 },
+{"id": 478013844078333955, "latitude": 41.9831109, "longitude": -91.59026341 },
+{"id": 477993642477760512, "latitude": 40.59777, "longitude": -74.69226215 },
+{"id": 477687347060817924, "latitude": 33.8977067, "longitude": -98.5042951 },
+{"id": 477823914920259584, "latitude": 41.26016555, "longitude": -72.79953241 },
+{"id": 477995019698454528, "latitude": 42.737329, "longitude": -92.4881123 },
+{"id": 477800972102541314, "latitude": 40.6233333, "longitude": -74.6011111 },
+{"id": 477962680221704192, "latitude": 34.23598105, "longitude": -77.85883579 },
+{"id": 477767820676452353, "latitude": 38.17036629, "longitude": -85.63743532 },
+{"id": 478008619729190914, "latitude": 41.50861365, "longitude": -72.98477451 },
+{"id": 477716576754360320, "latitude": 37.2415966, "longitude": -121.8484689 },
+{"id": 477840701577650176, "latitude": 39.4521242, "longitude": -75.0488449 },
+{"id": 478005517974253568, "latitude": 33.4171003, "longitude": -111.6210955 },
+{"id": 478009964649119745, "latitude": 33.91826951, "longitude": -117.88256493 },
+{"id": 477965746770042880, "latitude": 39.09991245, "longitude": -94.60176018 },
+{"id": 477857351861403648, "latitude": 44.94000487, "longitude": -92.93281722 },
+{"id": 477952241912254464, "latitude": 32.42628203, "longitude": -86.20760908 },
+{"id": 478024331701026816, "latitude": 39.08744548, "longitude": -76.86650128 },
+{"id": 477770132857835520, "latitude": 41.75770794, "longitude": -71.42729137 },
+{"id": 477924786283106304, "latitude": 46.86711921, "longitude": -96.84652896 },
+{"id": 477843614903447552, "latitude": 42.80845281, "longitude": -78.76962461 },
+{"id": 477708258740674560, "latitude": 41.89273784, "longitude": -87.72024488 },
+{"id": 477914770344267777, "latitude": 39.15904173, "longitude": -77.20414722 },
+{"id": 477964851130548224, "latitude": 31.71240828, "longitude": -106.4482849 },
+{"id": 477937141024952320, "latitude": 42.45694495, "longitude": -89.22906465 },
+{"id": 478015216501145601, "latitude": 33.57948145, "longitude": -85.10373398 },
+{"id": 477985687099408384, "latitude": 39.79673501, "longitude": -86.00981519 },
+{"id": 477860109234606080, "latitude": 35.52395005, "longitude": -97.56630309 },
+{"id": 477747057327738881, "latitude": 33.84282846, "longitude": -117.94108201 },
+{"id": 478023553292705792, "latitude": 34.04695248, "longitude": -90.02603705 },
+{"id": 477955056038207489, "latitude": 33.87835954, "longitude": -117.83692958 },
+{"id": 478041277632905216, "latitude": 34.28390473, "longitude": -118.51120491 },
+{"id": 477910848749305856, "latitude": 41.34215356, "longitude": -72.10550796 },
+{"id": 477995958198755328, "latitude": 38.79301084, "longitude": -121.23871074 },
+{"id": 477706808501338114, "latitude": 30.47900985, "longitude": -92.90025115 },
+{"id": 478030357892456448, "latitude": 37.69917668, "longitude": -97.47653894 },
+{"id": 478023027176009728, "latitude": 40.74186868, "longitude": -74.04647727 },
+{"id": 477963219399479296, "latitude": 39.26622863, "longitude": -77.1584765 },
+{"id": 477998036061147136, "latitude": 32.88563459, "longitude": -97.33560839 },
+{"id": 477912981884657664, "latitude": 33.74022641, "longitude": -84.70629068 },
+{"id": 478027407992324097, "latitude": 41.108509, "longitude": -83.347116 },
+{"id": 478041162864136192, "latitude": 32.65148959, "longitude": -115.4999187 },
+{"id": 477957656829632512, "latitude": 45.5086699, "longitude": -73.5539925 },
+{"id": 477856046514720768, "latitude": 45.5098431, "longitude": -122.68378924 },
+{"id": 478004514914844673, "latitude": 39.0902299, "longitude": -77.103741 },
+{"id": 477860001621352448, "latitude": 40.15338078, "longitude": -74.60297452 },
+{"id": 477711793427447810, "latitude": 32.4194548, "longitude": -96.8513628 },
+{"id": 477952550662991872, "latitude": 32.43543331, "longitude": -94.79776416 },
+{"id": 477959975524773889, "latitude": 47.10445093, "longitude": -122.29391904 },
+{"id": 477700164438982656, "latitude": 40.72243155, "longitude": -73.98623118 },
+{"id": 477743620389634048, "latitude": 37.274956, "longitude": -79.9637634 },
+{"id": 477679442089607169, "latitude": 40.50911374, "longitude": -111.84612086 },
+{"id": 478021184765063168, "latitude": 38.8303016, "longitude": -97.61020658 },
+{"id": 477858561721376769, "latitude": 39.7424454, "longitude": -104.99491802 },
+{"id": 477911797539028993, "latitude": 40.13503728, "longitude": -74.9131684 },
+{"id": 477889891510939648, "latitude": 43.00628767, "longitude": -83.73350996 },
+{"id": 477720496893870080, "latitude": 33.65960136, "longitude": -84.43571303 },
+{"id": 477917667769061376, "latitude": 34.45822821, "longitude": -118.50055438 },
+{"id": 478024704318787584, "latitude": 33.7664847, "longitude": -84.7595124 },
+{"id": 477732570759589888, "latitude": 34.43249778, "longitude": -118.42297553 },
+{"id": 478010941204746240, "latitude": 32.77298747, "longitude": -96.98846972 },
+{"id": 477868392469106688, "latitude": 41.95114492, "longitude": -87.88181458 },
+{"id": 477966865994506240, "latitude": 41.5512921, "longitude": -88.2806752 },
+{"id": 477976831485038592, "latitude": 33.62748856, "longitude": -78.9526619 },
+{"id": 477684294773522432, "latitude": 36.6986593, "longitude": -121.6025838 },
+{"id": 478008781897732097, "latitude": 32.843048, "longitude": -96.8272482 },
+{"id": 477954250115260416, "latitude": 40.70244742, "longitude": -75.27373431 },
+{"id": 478033040091152384, "latitude": 39.75692511, "longitude": -81.53793859 },
+{"id": 477728979927842817, "latitude": 45.52719744, "longitude": -122.94230431 },
+{"id": 477819040644096001, "latitude": 39.29668573, "longitude": -76.58678646 },
+{"id": 477844382117134336, "latitude": 33.44276078, "longitude": -84.57155756 },
+{"id": 477924519592464384, "latitude": 37.7726402, "longitude": -122.4099154 },
+{"id": 477940923746484225, "latitude": 40.8214923, "longitude": -81.3899254 },
+{"id": 477698358543257601, "latitude": 32.41622572, "longitude": -94.84791267 },
+{"id": 477876950808002561, "latitude": 34.1194691, "longitude": -117.35745563 },
+{"id": 477909274358976512, "latitude": 42.8017377, "longitude": -78.75619718 },
+{"id": 477963468776439809, "latitude": 42.96423859, "longitude": -85.90598739 },
+{"id": 477960784002048001, "latitude": 30.0947594, "longitude": -91.00207245 },
+{"id": 477873129964838912, "latitude": 47.04139972, "longitude": -122.75340633 },
+{"id": 477991720098209792, "latitude": 42.3502948, "longitude": -83.0432386 },
+{"id": 478036007682445312, "latitude": 30.06969858, "longitude": -95.41653706 },
+{"id": 477856328455430145, "latitude": 42.3636786, "longitude": -71.0825397 },
+{"id": 478032275263610880, "latitude": 47.40179034, "longitude": -120.30655878 },
+{"id": 477933223486300160, "latitude": 32.5167246, "longitude": -91.99739536 },
+{"id": 477946181080059904, "latitude": 43.0710258, "longitude": -87.9487872 },
+{"id": 477926910694281216, "latitude": 40.82611802, "longitude": -73.90181919 },
+{"id": 477990376356462592, "latitude": 39.6334668, "longitude": -77.7348643 },
+{"id": 477973887506849792, "latitude": 41.1217245, "longitude": -84.90149592 },
+{"id": 477948867657605120, "latitude": 34.49973776, "longitude": -79.30923652 },
+{"id": 477834556171505664, "latitude": 36.0020179, "longitude": -95.67961404 },
+{"id": 477865199177699329, "latitude": 35.22621056, "longitude": -81.11057606 },
+{"id": 477983507780608000, "latitude": 42.606919, "longitude": -113.1684622 },
+{"id": 477975492713144321, "latitude": 42.19833512, "longitude": -71.06179655 },
+{"id": 477888701801197568, "latitude": 33.4424075, "longitude": -117.61417128 },
+{"id": 477720011398590464, "latitude": 39.7814417, "longitude": -105.0225835 },
+{"id": 477891592972288000, "latitude": 40.02603788, "longitude": -75.05705919 },
+{"id": 477697557435801600, "latitude": 40.15909351, "longitude": -74.05171512 },
+{"id": 477975657683877888, "latitude": 47.22563807, "longitude": -122.35665939 },
+{"id": 478002488873078785, "latitude": 40.11960513, "longitude": -80.70100693 },
+{"id": 477983023804469248, "latitude": 40.64577187, "longitude": -73.71330374 },
+{"id": 477714883832594432, "latitude": 32.80656637, "longitude": -96.65052571 },
+{"id": 477871405040541697, "latitude": 42.99716276, "longitude": -88.05263353 },
+{"id": 477926044104933376, "latitude": 41.82953787, "longitude": -71.40081526 },
+{"id": 477871086722621440, "latitude": 42.31362257, "longitude": -71.20639926 },
+{"id": 477961246054944768, "latitude": 40.73896145, "longitude": -73.05441199 },
+{"id": 477983102372167680, "latitude": 40.67926549, "longitude": -73.92797102 },
+{"id": 478002474323038209, "latitude": 40.4374619, "longitude": -74.5494219 },
+{"id": 477994938345340929, "latitude": 33.60248588, "longitude": -112.34146812 },
+{"id": 477801517228240896, "latitude": 35.24043842, "longitude": -81.03893988 },
+{"id": 477973721362489344, "latitude": 44.96280289, "longitude": -93.23324585 },
+{"id": 477990537069613056, "latitude": 43.24215108, "longitude": -79.81596828 },
+{"id": 478002927010062336, "latitude": 35.61324251, "longitude": -82.31902975 },
+{"id": 478012769501859841, "latitude": 35.89629587, "longitude": -86.88264498 },
+{"id": 478025408907317248, "latitude": 37.9653027, "longitude": -87.5516888 },
+{"id": 477807251693264896, "latitude": 40.63413534, "longitude": -73.90717847 },
+{"id": 477859458891018240, "latitude": 40.5253497, "longitude": -91.4328378 },
+{"id": 478006701913018368, "latitude": 40.14677329, "longitude": -74.93967039 },
+{"id": 477840434669318144, "latitude": 33.55755365, "longitude": -81.71889561 },
+{"id": 477854415232040960, "latitude": 33.18732265, "longitude": -97.1065687 },
+{"id": 477954491971428352, "latitude": 41.6914339, "longitude": -87.53504017 },
+{"id": 477806120350666752, "latitude": 42.44354127, "longitude": -83.31779831 },
+{"id": 477783357825110016, "latitude": 40.2080235, "longitude": -111.6255726 },
+{"id": 478008050272731138, "latitude": 34.8192, "longitude": -82.388 },
+{"id": 477938688924450816, "latitude": 42.4165811, "longitude": -83.3101361 },
+{"id": 478034095050858496, "latitude": 41.75987336, "longitude": -111.84427139 },
+{"id": 477723591794884608, "latitude": 34.0653631, "longitude": -117.2532082 },
+{"id": 477811111149178880, "latitude": 30.32773106, "longitude": -81.48687364 },
+{"id": 477895351873966080, "latitude": 41.24210627, "longitude": -95.8219624 },
+{"id": 477996740415881217, "latitude": 41.80071839, "longitude": -72.55130558 },
+{"id": 477940500021133312, "latitude": 44.17862722, "longitude": -77.41224383 },
+{"id": 477724116602986499, "latitude": 41.0927125, "longitude": -85.4343322 },
+{"id": 477887499453595648, "latitude": 38.91615253, "longitude": -75.89677619 },
+{"id": 477968687430107136, "latitude": 41.84982196, "longitude": -71.40216124 },
+{"id": 477925793470099457, "latitude": 33.58646392, "longitude": -84.39121322 },
+{"id": 477898035230691328, "latitude": 40.6049826, "longitude": -79.1846658 },
+{"id": 477835240111890432, "latitude": 40.7011283, "longitude": -90.0089104 },
+{"id": 477815141296119809, "latitude": 38.63838031, "longitude": -121.2712777 },
+{"id": 478005500210974722, "latitude": 34.03253355, "longitude": -118.4896273 },
+{"id": 477958633594957824, "latitude": 41.39304201, "longitude": -72.9136892 },
+{"id": 477838145921515521, "latitude": 40.76320209, "longitude": -74.17583584 },
+{"id": 478004679692279810, "latitude": 33.6401662, "longitude": -84.3372063 },
+{"id": 477819837368893440, "latitude": 33.4583179, "longitude": -82.0681424 },
+{"id": 477844257227567106, "latitude": 35.03626327, "longitude": -118.97661477 },
+{"id": 477882675303174144, "latitude": 39.77915782, "longitude": -104.9859512 },
+{"id": 477906480222720000, "latitude": 37.01337632, "longitude": -76.37305118 },
+{"id": 477829276793053184, "latitude": 42.01377447, "longitude": -87.66851933 },
+{"id": 477925715225362432, "latitude": 39.65587234, "longitude": -79.77051998 },
+{"id": 477784793560535041, "latitude": 42.58400455, "longitude": -71.98178703 },
+{"id": 477702252757389312, "latitude": 33.48361571, "longitude": -112.07901583 },
+{"id": 478008266971025408, "latitude": 33.88478306, "longitude": -117.51252243 },
+{"id": 477843028682764288, "latitude": 32.8295, "longitude": -79.8238 },
+{"id": 477719445989629952, "latitude": 39.13642633, "longitude": -121.62002665 },
+{"id": 478028170835554304, "latitude": 34.23172432, "longitude": -118.40395544 },
+{"id": 477953530020392961, "latitude": 42.14138336, "longitude": -72.56764529 },
+{"id": 478007826972172288, "latitude": 39.09499641, "longitude": -77.07362917 },
+{"id": 477977897870053376, "latitude": 39.41825322, "longitude": -84.47142944 },
+{"id": 477954762592120832, "latitude": 32.9090117, "longitude": -96.8523687 },
+{"id": 477927083444674560, "latitude": 30.32986394, "longitude": -91.769592 },
+{"id": 478036451980886016, "latitude": 32.68202073, "longitude": -97.39411541 },
+{"id": 477973586712330240, "latitude": 41.92864493, "longitude": -71.92469909 },
+{"id": 478041131222327296, "latitude": 34.12048335, "longitude": -117.98989152 },
+{"id": 477967274209337345, "latitude": 32.6880722, "longitude": -96.7976064 },
+{"id": 477703727394979842, "latitude": 34.14178794, "longitude": -118.67441005 },
+{"id": 477906800998502400, "latitude": 34.03123283, "longitude": -117.79037458 },
+{"id": 477970674544562176, "latitude": 34.26844206, "longitude": -119.27291036 },
+{"id": 477873142254166016, "latitude": 30.10688223, "longitude": -90.94209893 },
+{"id": 478011481858912256, "latitude": 30.10497337, "longitude": -97.30787269 },
+{"id": 477917886875316224, "latitude": 30.50160009, "longitude": -97.73208464 },
+{"id": 477926403858395136, "latitude": 32.74026482, "longitude": -117.21180668 },
+{"id": 477822634558636032, "latitude": 30.18794223, "longitude": -85.69237506 },
+{"id": 477957132927524864, "latitude": 45.53165125, "longitude": -122.69481906 },
+{"id": 477694208933765121, "latitude": 40.41708476, "longitude": -80.1853684 },
+{"id": 477696484549226496, "latitude": 34.03418203, "longitude": -117.49746506 },
+{"id": 477980577417920512, "latitude": 39.7197945, "longitude": -104.9521158 },
+{"id": 477888410330222592, "latitude": 40.4376735, "longitude": -79.2165495 },
+{"id": 477697126273540097, "latitude": 33.938555, "longitude": -117.2319921 },
+{"id": 477689447668920320, "latitude": 38.727564, "longitude": -75.1298873 },
+{"id": 477905828755034112, "latitude": 40.7397292, "longitude": -81.3666992 },
+{"id": 477941128931852288, "latitude": 33.7361958, "longitude": -84.3887742 },
+{"id": 477753835423662080, "latitude": 40.38952284, "longitude": -90.16216108 },
+{"id": 477967126079475713, "latitude": 40.83688459, "longitude": -74.11455852 },
+{"id": 477716779125329920, "latitude": 40.7495269, "longitude": -73.9712674 },
+{"id": 477791726942175232, "latitude": 32.7074448, "longitude": -96.7870214 },
+{"id": 477711858862813184, "latitude": 46.5978252, "longitude": -120.576959 },
+{"id": 477892270515965953, "latitude": 34.0268538, "longitude": -117.3384829 },
+{"id": 477783612226420736, "latitude": 39.44644772, "longitude": -91.04822899 },
+{"id": 477831426318942209, "latitude": 35.28009677, "longitude": -107.98213935 },
+{"id": 478041225162141696, "latitude": 34.47871388, "longitude": -86.89536778 },
+{"id": 477988571304169473, "latitude": 40.75040383, "longitude": -74.40576215 },
+{"id": 477855872459104256, "latitude": 38.92446732, "longitude": -77.05472959 },
+{"id": 477984863732367361, "latitude": 30.5197895, "longitude": -96.7094647 },
+{"id": 477689131200294913, "latitude": 33.62165248, "longitude": -112.11514998 },
+{"id": 477912062031446016, "latitude": 36.08939892, "longitude": -115.17462273 },
+{"id": 477901235841142785, "latitude": 33.10726278, "longitude": -83.24583053 },
+{"id": 477844275795742720, "latitude": 30.5150136, "longitude": -81.6355519 },
+{"id": 477887332159213568, "latitude": 42.1335245, "longitude": -72.7850529 },
+{"id": 477855918592237568, "latitude": 45.5019451, "longitude": -122.4853043 },
+{"id": 477851789362864129, "latitude": 32.82051881, "longitude": -96.86759971 },
+{"id": 477994057319587840, "latitude": 43.0753188, "longitude": -89.51751349 },
+{"id": 478005623448428544, "latitude": 34.75272293, "longitude": -92.23983077 },
+{"id": 477872114859798531, "latitude": 40.65796667, "longitude": -74.27705191 },
+{"id": 477924070890037248, "latitude": 33.85588318, "longitude": -118.00228986 },
+{"id": 477935310454947840, "latitude": 33.59552389, "longitude": -84.09271642 },
+{"id": 477861992695611392, "latitude": 43.73021231, "longitude": -79.40181142 },
+{"id": 478025276304007168, "latitude": 34.11130425, "longitude": -117.64576051 },
+{"id": 477810914922471424, "latitude": 41.82315735, "longitude": -72.52744732 },
+{"id": 477843859485900800, "latitude": 41.45753493, "longitude": -83.36714118 },
+{"id": 478021090082820096, "latitude": 35.99637568, "longitude": -79.99050989 },
+{"id": 477980374044540930, "latitude": 38.944052, "longitude": -78.465815 },
+{"id": 477682314131288064, "latitude": 40.80077946, "longitude": -73.41087178 },
+{"id": 477833283036672000, "latitude": 39.1136334, "longitude": -94.6369857 },
+{"id": 477949820611461122, "latitude": 33.88167628, "longitude": -117.8960245 },
+{"id": 478011817072263168, "latitude": 39.28281407, "longitude": -76.59813163 },
+{"id": 477851194849636352, "latitude": 35.39877529, "longitude": -118.9684969 },
+{"id": 477997898701864960, "latitude": 36.0129276, "longitude": -86.6807238 },
+{"id": 477720493584158721, "latitude": 41.13093839, "longitude": -81.81031667 },
+{"id": 477906146012176384, "latitude": 44.6621051, "longitude": -74.9784274 },
+{"id": 477920544290271233, "latitude": 40.8623029, "longitude": -73.0332399 },
+{"id": 478004379803725824, "latitude": 33.37761959, "longitude": -84.71721249 },
+{"id": 477793226112634880, "latitude": 36.6564542, "longitude": -88.3010575 },
+{"id": 478013038117666817, "latitude": 33.62878737, "longitude": -111.93749333 },
+{"id": 477836074363133952, "latitude": 45.5025344, "longitude": -73.5708159 },
+{"id": 478022542629040128, "latitude": 34.8789039, "longitude": -82.4430189 },
+{"id": 477898419466682368, "latitude": 42.11699562, "longitude": -83.19813159 },
+{"id": 477873989617197056, "latitude": 41.31619857, "longitude": -81.44647921 },
+{"id": 477969499602554880, "latitude": 43.64605975, "longitude": -79.3797935 },
+{"id": 477973853738516480, "latitude": 35.5234805, "longitude": -84.38886032 },
+{"id": 478024849835950080, "latitude": 40.45316773, "longitude": -80.21897715 },
+{"id": 477680233374420992, "latitude": 34.74291954, "longitude": -87.64059694 },
+{"id": 477823275440287745, "latitude": 43.228417, "longitude": -79.85119742 },
+{"id": 478027773068718081, "latitude": 49.25255586, "longitude": -123.07894142 },
+{"id": 477934186381058050, "latitude": 33.8654823, "longitude": -117.2111045 },
+{"id": 477857964414357505, "latitude": 41.88229886, "longitude": -87.90562197 },
+{"id": 477994394813857792, "latitude": 32.74188459, "longitude": -117.09981393 },
+{"id": 477793630397026304, "latitude": 41.4419019, "longitude": -81.7435831 },
+{"id": 477986871302447104, "latitude": 39.95852396, "longitude": -75.27872297 },
+{"id": 478026948023574529, "latitude": 39.83968504, "longitude": -99.9001437 },
+{"id": 477916959712235520, "latitude": 40.0088029, "longitude": -82.7768254 },
+{"id": 478013684841988096, "latitude": 37.37141468, "longitude": -121.92255526 },
+{"id": 477908431010267138, "latitude": 38.89468451, "longitude": -77.45109956 },
+{"id": 478032577807134720, "latitude": 37.70018133, "longitude": -121.44848855 },
+{"id": 477805900095582209, "latitude": 40.11039718, "longitude": -74.06970781 },
+{"id": 478012775974055936, "latitude": 41.61943017, "longitude": -88.14623056 },
+{"id": 477953238474309632, "latitude": 41.99041229, "longitude": -87.69814926 },
+{"id": 477806316656660481, "latitude": 38.30754116, "longitude": -77.51818604 },
+{"id": 477860735578435585, "latitude": 35.55063937, "longitude": -97.58282271 },
+{"id": 477942762483228672, "latitude": 40.8396522, "longitude": -73.93923139 },
+{"id": 477714135522615296, "latitude": 36.73053603, "longitude": -119.80986192 },
+{"id": 478037506349608960, "latitude": 37.7251852, "longitude": -88.9384973 },
+{"id": 477952816695103488, "latitude": 42.76152052, "longitude": -71.08420894 },
+{"id": 477854562578362368, "latitude": 40.68058839, "longitude": -75.24127435 },
+{"id": 477984993197977600, "latitude": 33.75762427, "longitude": -84.3964286 },
+{"id": 477939743108251648, "latitude": 36.63098019, "longitude": -79.38030797 },
+{"id": 477954217907613696, "latitude": 38.93184447, "longitude": -74.92381211 },
+{"id": 477803926255710208, "latitude": 38.5969689, "longitude": -90.2731818 },
+{"id": 477967556586659840, "latitude": 33.92608706, "longitude": -98.51359251 },
+{"id": 477970234021019648, "latitude": 40.68637425, "longitude": -73.90854918 },
+{"id": 477843735225438208, "latitude": 31.84989614, "longitude": -106.45189301 },
+{"id": 477752054488641536, "latitude": 30.30301411, "longitude": -87.42596473 },
+{"id": 478026144474607617, "latitude": 41.8827041, "longitude": -87.7715596 },
+{"id": 477682604641361920, "latitude": 34.9987219, "longitude": -82.42219454 },
+{"id": 477695238212182016, "latitude": 42.2244245, "longitude": -83.61281745 },
+{"id": 477894607175307264, "latitude": 36.33875623, "longitude": -88.86574467 },
+{"id": 477804307698290690, "latitude": 43.6608713, "longitude": -79.3659713 },
+{"id": 477871744934760450, "latitude": 40.8852769, "longitude": -74.67702735 },
+{"id": 477684267950960643, "latitude": 33.43275104, "longitude": -112.3261626 },
+{"id": 477892037283680256, "latitude": 40.97347298, "longitude": -75.2618311 },
+{"id": 478029780928180224, "latitude": 42.9708326, "longitude": -75.0126497 },
+{"id": 477949881198608384, "latitude": 43.18174345, "longitude": -77.591468 },
+{"id": 477943278826819585, "latitude": 43.7137289, "longitude": -79.5555322 },
+{"id": 477946972528050177, "latitude": 41.58564898, "longitude": -87.6023239 },
+{"id": 477940846596091904, "latitude": 36.7336596, "longitude": -79.3059087 },
+{"id": 477719049434968064, "latitude": 33.6495332, "longitude": -117.96801701 },
+{"id": 477967500144283648, "latitude": 40.84373164, "longitude": -74.09780695 },
+{"id": 477946332976394241, "latitude": 41.23336131, "longitude": -95.8790523 },
+{"id": 478008454247100416, "latitude": 41.7617374, "longitude": -88.293187 },
+{"id": 477954034519642113, "latitude": 41.26713079, "longitude": -95.93170381 },
+{"id": 477889770832396288, "latitude": 49.27676626, "longitude": -123.1326484 },
+{"id": 477685614783893504, "latitude": 40.3044062, "longitude": -75.1276489 },
+{"id": 477919715914821632, "latitude": 30.4786801, "longitude": -87.3227319 },
+{"id": 477850362846937089, "latitude": 33.58379679, "longitude": -84.45362401 },
+{"id": 477941774317395968, "latitude": 33.80483188, "longitude": -117.91881316 },
+{"id": 478014367326535681, "latitude": 36.17812821, "longitude": -115.32341822 },
+{"id": 477875100574679040, "latitude": 37.36012256, "longitude": -121.9757628 },
+{"id": 478025741482094592, "latitude": 42.64122123, "longitude": -83.46136668 },
+{"id": 477855769728393217, "latitude": 30.18438814, "longitude": -81.73379097 },
+{"id": 477994122393837568, "latitude": 30.15434874, "longitude": -95.60582036 },
+{"id": 477959765834739712, "latitude": 38.31544291, "longitude": -122.30854237 },
+{"id": 477723331223764992, "latitude": 38.27284239, "longitude": -121.29270564 },
+{"id": 477788639951220736, "latitude": 32.96598732, "longitude": -96.80743613 },
+{"id": 478004281325273088, "latitude": 34.4871683, "longitude": -97.1922675 },
+{"id": 477708480204525568, "latitude": 44.03792567, "longitude": -75.82617412 },
+{"id": 478008199933865985, "latitude": 34.39983983, "longitude": -87.7525427 },
+{"id": 477883273939787776, "latitude": 38.98202363, "longitude": -84.52590887 },
+{"id": 477828565460078592, "latitude": 39.2526838, "longitude": -84.6098216 },
+{"id": 477865378777796608, "latitude": 33.64324854, "longitude": -117.80347672 },
+{"id": 477955362469842944, "latitude": 44.90860924, "longitude": -93.72830423 },
+{"id": 478000187361988608, "latitude": 39.2054555, "longitude": -76.6535671 },
+{"id": 477778253315596288, "latitude": 42.18900695, "longitude": -70.93179448 },
+{"id": 477986871143067648, "latitude": 36.00283739, "longitude": -75.66161066 },
+{"id": 477931410368376832, "latitude": 45.5318, "longitude": -122.896 },
+{"id": 477781295372967937, "latitude": 35.01579945, "longitude": -86.56565164 },
+{"id": 478038660558757888, "latitude": 34.01759414, "longitude": -118.30016276 },
+{"id": 477949836688236544, "latitude": 33.9344579, "longitude": -118.1881266 },
+{"id": 477977623604514816, "latitude": 36.11835959, "longitude": -115.17045021 },
+{"id": 478026078015873024, "latitude": 41.90258434, "longitude": -87.62852383 },
+{"id": 477964431234985984, "latitude": 38.4805033, "longitude": -78.0207609 },
+{"id": 477690680941109248, "latitude": 33.82966035, "longitude": -98.67529942 },
+{"id": 477845578064211968, "latitude": 33.76055869, "longitude": -118.13591499 },
+{"id": 477695600469630976, "latitude": 35.307094, "longitude": -80.7062965 },
+{"id": 477830707088474112, "latitude": 38.6541194, "longitude": -77.27384458 },
+{"id": 477992147501973504, "latitude": 39.52879318, "longitude": -119.81734248 },
+{"id": 477700243199238144, "latitude": 34.174836, "longitude": -118.2732806 },
+{"id": 477975893936066561, "latitude": 33.04736756, "longitude": -96.74070298 },
+{"id": 477987295061344256, "latitude": 41.41786502, "longitude": -81.57343118 },
+{"id": 477892902677676032, "latitude": 40.84411022, "longitude": -73.85532408 },
+{"id": 477697601945341952, "latitude": 36.82010193, "longitude": -79.35816579 }
+]);
+
+// --------- Run the spatial join query
+
+use SJTest;
+
+SELECT States.name, POIS.id
+FROM States, POIS
+WHERE st_contains(States.boundary, st_make_point(POIS.longitude, POIS.latitude));
diff --git a/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/autocomplete.png b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/autocomplete.png
new file mode 100644
index 0000000..99179be
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/autocomplete.png
Binary files differ
diff --git a/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/csv_header_sample_output.png b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/csv_header_sample_output.png
new file mode 100644
index 0000000..e30ef18
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/csv_header_sample_output.png
Binary files differ
diff --git a/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/input_component.png b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/input_component.png
new file mode 100644
index 0000000..9fd832e
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/input_component.png
Binary files differ
diff --git a/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/metadata_inspector.png b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/metadata_inspector.png
new file mode 100644
index 0000000..7f61f20
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/metadata_inspector.png
Binary files differ
diff --git a/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/plan_explorer.png b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/plan_explorer.png
new file mode 100644
index 0000000..2e48556
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/images/dashboard_screenshots/plan_explorer.png
Binary files differ
diff --git a/asterixdb/asterix-doc/src/site/resources/images/linestring.png b/asterixdb/asterix-doc/src/site/resources/images/linestring.png
new file mode 100644
index 0000000..1c50a9f
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/images/linestring.png
Binary files differ
diff --git a/asterixdb/asterix-doc/src/site/resources/images/linestring.svg b/asterixdb/asterix-doc/src/site/resources/images/linestring.svg
new file mode 100644
index 0000000..f775e79
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/resources/images/linestring.svg
@@ -0,0 +1,245 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+
+<!--
+ ! 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.
+ !-->
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+ xmlns:dc="http://purl.org/dc/elements/1.1/"
+ xmlns:cc="http://creativecommons.org/ns#"
+ xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+ xmlns:svg="http://www.w3.org/2000/svg"
+ xmlns="http://www.w3.org/2000/svg"
+ xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+ xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+ width="50"
+ height="50"
+ viewBox="0 0 13.229166 13.229167"
+ version="1.1"
+ id="svg8"
+ inkscape:version="0.92.4 (f8dce91, 2019-08-02)"
+ sodipodi:docname="linestring.svg"
+ inkscape:export-filename="/home/eldawy/workspace/asterixdb/asterixdb/asterix-doc/src/site/resources/images/linestring.png"
+ inkscape:export-xdpi="384"
+ inkscape:export-ydpi="384">
+ <defs
+ id="defs2">
+ <marker
+ inkscape:stockid="EmptyDiamondL"
+ orient="auto"
+ refY="0.0"
+ refX="0.0"
+ id="marker5096"
+ style="overflow:visible"
+ inkscape:isstock="true">
+ <path
+ id="path5094"
+ d="M 0,-7.0710768 L -7.0710894,0 L 0,7.0710589 L 7.0710462,0 L 0,-7.0710768 z "
+ style="fill-rule:evenodd;fill:#ffffff;stroke:#0000ff;stroke-width:1pt;stroke-opacity:1"
+ transform="scale(0.8)" />
+ </marker>
+ <marker
+ inkscape:stockid="SquareL"
+ orient="auto"
+ refY="0.0"
+ refX="0.0"
+ id="marker5050"
+ style="overflow:visible"
+ inkscape:isstock="true">
+ <path
+ id="path5048"
+ d="M -5.0,-5.0 L -5.0,5.0 L 5.0,5.0 L 5.0,-5.0 L -5.0,-5.0 z "
+ style="fill-rule:evenodd;stroke:#0000ff;stroke-width:1pt;stroke-opacity:1;fill:#0000ff;fill-opacity:1"
+ transform="scale(0.8)" />
+ </marker>
+ <marker
+ inkscape:stockid="EmptyDiamondL"
+ orient="auto"
+ refY="0.0"
+ refX="0.0"
+ id="EmptyDiamondL"
+ style="overflow:visible"
+ inkscape:isstock="true"
+ inkscape:collect="always">
+ <path
+ id="path4630"
+ d="M 0,-7.0710768 L -7.0710894,0 L 0,7.0710589 L 7.0710462,0 L 0,-7.0710768 z "
+ style="fill-rule:evenodd;fill:#ffffff;stroke:#0000ff;stroke-width:1pt;stroke-opacity:1"
+ transform="scale(0.8)" />
+ </marker>
+ <marker
+ inkscape:isstock="true"
+ style="overflow:visible"
+ id="marker4874"
+ refX="0.0"
+ refY="0.0"
+ orient="auto"
+ inkscape:stockid="EmptyDiamondLend">
+ <path
+ transform="scale(0.8) translate(-7,0)"
+ style="fill-rule:evenodd;fill:#ffffff;stroke:#000000;stroke-width:1pt;stroke-opacity:1"
+ d="M 0,-7.0710768 L -7.0710894,0 L 0,7.0710589 L 7.0710462,0 L 0,-7.0710768 z "
+ id="path4872" />
+ </marker>
+ <marker
+ inkscape:stockid="SquareL"
+ orient="auto"
+ refY="0.0"
+ refX="0.0"
+ id="SquareL"
+ style="overflow:visible"
+ inkscape:isstock="true"
+ inkscape:collect="always">
+ <path
+ id="path4594"
+ d="M -5.0,-5.0 L -5.0,5.0 L 5.0,5.0 L 5.0,-5.0 L -5.0,-5.0 z "
+ style="fill-rule:evenodd;stroke:#0000ff;stroke-width:1pt;stroke-opacity:1;fill:#0000ff;fill-opacity:1"
+ transform="scale(0.8)" />
+ </marker>
+ </defs>
+ <sodipodi:namedview
+ id="base"
+ pagecolor="#ffffff"
+ bordercolor="#666666"
+ borderopacity="1.0"
+ inkscape:pageopacity="0.0"
+ inkscape:pageshadow="2"
+ inkscape:zoom="15.31462"
+ inkscape:cx="18.561092"
+ inkscape:cy="9.425358"
+ inkscape:document-units="mm"
+ inkscape:current-layer="layer1"
+ showgrid="true"
+ units="px"
+ inkscape:snap-grids="false"
+ inkscape:snap-to-guides="false"
+ inkscape:window-width="1672"
+ inkscape:window-height="1248"
+ inkscape:window-x="1956"
+ inkscape:window-y="520"
+ inkscape:window-maximized="0"
+ inkscape:snap-page="true">
+ <inkscape:grid
+ type="xygrid"
+ id="grid815"
+ spacingx="2.6458333"
+ spacingy="2.6458333" />
+ </sodipodi:namedview>
+ <metadata
+ id="metadata5">
+ <rdf:RDF>
+ <cc:Work
+ rdf:about="">
+ <dc:format>image/svg+xml</dc:format>
+ <dc:type
+ rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+ <dc:title />
+ </cc:Work>
+ </rdf:RDF>
+ </metadata>
+ <g
+ inkscape:label="Layer 1"
+ inkscape:groupmode="layer"
+ id="layer1"
+ transform="translate(0,-283.77082)">
+ <g
+ id="g948"
+ style="stroke:#e6e6e6">
+ <g
+ style="stroke:#e6e6e6;stroke-width:0.05;stroke-miterlimit:4;stroke-dasharray:none"
+ id="g932">
+ <g
+ style="stroke:#e6e6e6;stroke-width:0.05;stroke-miterlimit:4;stroke-dasharray:none"
+ id="g910">
+ <path
+ inkscape:connector-curvature="0"
+ id="path894"
+ d="m 10.583333,283.77082 v 13.22917"
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none" />
+ <path
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none"
+ d="m 7.9375,283.77082 v 13.22917"
+ id="path896"
+ inkscape:connector-curvature="0" />
+ <path
+ inkscape:connector-curvature="0"
+ id="path898"
+ d="m 5.2916667,283.77082 v 13.22917"
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none" />
+ <path
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none"
+ d="m 2.6458333,283.77082 v 13.22917"
+ id="path900"
+ inkscape:connector-curvature="0" />
+ </g>
+ <g
+ style="stroke:#e6e6e6;stroke-width:0.05;stroke-miterlimit:4;stroke-dasharray:none"
+ id="g920"
+ transform="rotate(90,6.6145832,290.38541)">
+ <path
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none"
+ d="m 10.583333,283.77082 v 13.22917"
+ id="path912"
+ inkscape:connector-curvature="0" />
+ <path
+ inkscape:connector-curvature="0"
+ id="path914"
+ d="m 7.9375,283.77082 v 13.22917"
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none" />
+ <path
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none"
+ d="m 5.2916667,283.77082 v 13.22917"
+ id="path916"
+ inkscape:connector-curvature="0" />
+ <path
+ inkscape:connector-curvature="0"
+ id="path918"
+ d="m 2.6458333,283.77082 v 13.22917"
+ style="fill:none;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none" />
+ </g>
+ </g>
+ <rect
+ y="283.77081"
+ x="0"
+ height="13.229166"
+ width="13.229166"
+ id="rect934"
+ style="opacity:1;fill:none;fill-opacity:1;stroke:#e6e6e6;stroke-width:0.05;stroke-linecap:round;stroke-linejoin:miter;stroke-miterlimit:4;stroke-dasharray:none;stroke-dashoffset:0;stroke-opacity:1;paint-order:stroke fill markers" />
+ </g>
+ <path
+ style="fill:#aaeeff;stroke:#000000;stroke-width:0.18897638;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-dasharray:none;stroke-opacity:0.50196078;fill-opacity:0.50196081"
+ d="M 45,5 15,10 10,30 35,40 Z M 35,15 30,30 20,20 Z"
+ transform="matrix(0.26458333,0,0,0.26458333,0,283.77082)"
+ id="path5044"
+ inkscape:connector-curvature="0"
+ sodipodi:nodetypes="ccccccccc" />
+ <path
+ style="fill:none;stroke:#0000ff;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-opacity:1;stroke-miterlimit:4;stroke-dasharray:none;marker-start:url(#SquareL);marker-mid:url(#EmptyDiamondL)"
+ d="m 5.2916667,289.06249 3.96875,-1.32292 -1.3229167,3.96875 z"
+ id="path4522"
+ inkscape:connector-curvature="0"
+ sodipodi:nodetypes="cccc" />
+ <path
+ style="fill:none;stroke:#0000ff;stroke-width:0.05;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-dasharray:none;stroke-opacity:1;marker-start:url(#marker5050);marker-mid:url(#marker5096)"
+ d="m 9.2604166,294.35415 -6.6145833,-2.64583 1.3229167,-5.29167 7.9375,-1.32291 z"
+ id="path5367"
+ inkscape:connector-curvature="0"
+ sodipodi:nodetypes="ccccc" />
+ </g>
+</svg>
diff --git a/asterixdb/asterix-doc/src/site/site.xml b/asterixdb/asterix-doc/src/site/site.xml
index 45193fc..5c837d2 100644
--- a/asterixdb/asterix-doc/src/site/site.xml
+++ b/asterixdb/asterix-doc/src/site/site.xml
@@ -96,7 +96,10 @@
<item name="Filter-Based LSM Index Acceleration" href="sqlpp/filters.html"/>
<item name="Support of Full-text Queries" href="sqlpp/fulltext.html"/>
<item name="Support of Similarity Queries" href="sqlpp/similarity.html"/>
+ <item name="GIS Support Overview" href="geo/quickstart.html"/>
+ <item name="GIS Functions" href="geo/functions.html"/>
<item name="Support of Interval Joins" href="interval_join.html"/>
+ <item name="Support of Spatial Joins" href="spatial_join.html"/>
<item name="Support of Array Indexes" href="sqlpp/arrayindex.html"/>
</menu>
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index c0ee300..a5ce71d 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -207,6 +207,10 @@
<usedDependency>com.sun.xml.bind:jaxb-impl</usedDependency>
<usedDependency>com.sun.activation:javax.activation</usedDependency>
<usedDependency>net.razorvine:serpent</usedDependency>
+ <usedDependency>io.netty:netty-resolver-dns</usedDependency>
+ <usedDependency>io.netty:netty-codec-http2</usedDependency>
+ <usedDependency>io.netty:netty-transport-native-unix-common</usedDependency>
+ <usedDependency>org.apache.tomcat:tomcat-annotations-api</usedDependency>
</usedDependencies>
</configuration>
</plugin>
@@ -246,11 +250,6 @@
<version>${project.version}</version>
</dependency>
<dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-hivecompat</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-core</artifactId>
<version>4.0.3</version>
@@ -271,10 +270,6 @@
<artifactId>rome</artifactId>
</dependency>
<dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-serde</artifactId>
- </dependency>
- <dependency>
<groupId>com.e-movimento.tinytools</groupId>
<artifactId>privilegedaccessor</artifactId>
<version>1.2.2</version>
@@ -433,10 +428,6 @@
<artifactId>hyracks-http</artifactId>
</dependency>
<dependency>
- <groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
- </dependency>
- <dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
@@ -469,6 +460,22 @@
<artifactId>azure-storage-blob</artifactId>
</dependency>
<dependency>
+ <groupId>com.azure</groupId>
+ <artifactId>azure-storage-file-datalake</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.azure</groupId>
+ <artifactId>azure-identity</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-storage</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tomcat</groupId>
+ <artifactId>tomcat-annotations-api</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.msgpack</groupId>
<artifactId>msgpack-core</artifactId>
</dependency>
@@ -484,5 +491,33 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-1.2-api</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-s3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-dynamodb</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-aws</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-azure</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http2</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-unix-common</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReaderFactory.java
index a936e86..8849508 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReaderFactory.java
@@ -19,22 +19,34 @@
package org.apache.asterix.external.api;
import java.util.List;
+import java.util.Set;
+import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IRecordReaderFactory<T> extends IExternalDataSourceFactory {
- public IRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition)
- throws HyracksDataException;
+ IRecordReader<? extends T> createRecordReader(IHyracksTaskContext ctx, int partition) throws HyracksDataException;
- public Class<?> getRecordClass();
+ Class<?> getRecordClass();
@Override
- public default DataSourceType getDataSourceType() {
+ default DataSourceType getDataSourceType() {
return DataSourceType.RECORDS;
}
- public List<String> getRecordReaderNames();
+ List<String> getRecordReaderNames();
+
+ /**
+ * Usually there is only a single adapter with a specific name.
+ * When two or more adapters share the same name, only one can support {@link ExternalDataConstants#ALL_FORMATS}.
+ * Other adapters must have only a subset of {@link ExternalDataConstants#ALL_FORMATS}.
+ *
+ * @return adapter's supported formats
+ */
+ default Set<String> getReaderSupportedFormats() {
+ return ExternalDataConstants.ALL_FORMATS;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java
index e26b925..824ae56 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/IndexingConstants.java
@@ -41,8 +41,6 @@
public static final String EXTERNAL_FILE_INDEX_NAME_SUFFIX = "FilesIndex";
public static final String KEY_INPUT_FORMAT = "input-format";
- public static final String INPUT_FORMAT_RC = "rc-input-format";
- public static final String INPUT_FORMAT_RC_FULLY_QUALIFIED = "org.apache.asterix.hivecompat.io.RCFileInputFormat";
//Field Types
public static final IAType FILE_NUMBER_FIELD_TYPE = BuiltinType.AINT32;
@@ -56,9 +54,6 @@
BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT64, true);
private static final IBinaryComparatorFactory rowNumberCompFactory =
BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true);
-
- private static final IBinaryComparatorFactory[] rCFileRIDComparatorFactories =
- { fileNumberCompFactory, recordOffsetCompFactory, rowNumberCompFactory };
private static final IBinaryComparatorFactory[] txtSeqFileRIDComparatorFactories =
{ fileNumberCompFactory, recordOffsetCompFactory };
@@ -114,20 +109,14 @@
// This function returns the size of the RID for the passed file input format
public static int getRIDSize(String fileInputFormat) {
- if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED)) {
- return 3;
- } else {
- return 2;
- }
+ //Only text format is supported.
+ return 2;
}
// This function returns the size of the RID for the passed file input format
public static IBinaryComparatorFactory[] getComparatorFactories(String fileInputFormat) {
- if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED)) {
- return rCFileRIDComparatorFactories;
- } else {
- return txtSeqFileRIDComparatorFactories;
- }
+ //Only text format is supported.
+ return txtSeqFileRIDComparatorFactories;
}
public static IAType getFieldType(int fieldNumber) throws AsterixException {
@@ -204,12 +193,8 @@
}
public static List<List<String>> getRIDKeys(Map<String, String> properties) {
- String fileInputFormat = properties.get(KEY_INPUT_FORMAT);
- if (fileInputFormat.equals(INPUT_FORMAT_RC) || fileInputFormat.equals(INPUT_FORMAT_RC_FULLY_QUALIFIED)) {
- return RECORD_ID_FIELDS;
- } else {
- return RECORD_ID_FIELDS.subList(0, ROW_NUMBER_FIELD_INDEX);
- }
+ //Only text format is supported.
+ return RECORD_ID_FIELDS.subList(0, ROW_NUMBER_FIELD_INDEX);
}
public static String getFilesIndexName(String datasetName) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index f436aa1..8ea9ed4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -48,18 +48,22 @@
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.ICCServiceContext;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.hyracks.hdfs.dataflow.ConfFactory;
import org.apache.hyracks.hdfs.dataflow.InputSplitsFactory;
import org.apache.hyracks.hdfs.scheduler.Scheduler;
public class HDFSDataSourceFactory implements IRecordReaderFactory<Object>, IIndexibleExternalDataSource {
- protected static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
+ private static final List<String> recordReaderNames = Collections.singletonList("hdfs");
+
protected transient AlgebricksAbsolutePartitionConstraint clusterLocations;
protected transient IServiceContext serviceCtx;
protected String[] readSchedule;
@@ -79,23 +83,31 @@
private InputSplit[] inputSplits;
private String nodeName;
private Class recordReaderClazz;
- private static final List<String> recordReaderNames = Collections.unmodifiableList(Arrays.asList("hdfs"));
@Override
public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
- IWarningCollector warningCollector) throws AsterixException {
+ IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
+ JobConf hdfsConf = createHdfsConf(serviceCtx, configuration);
+ configureHdfsConf(hdfsConf, configuration);
+ }
+
+ protected JobConf createHdfsConf(IServiceContext serviceCtx, Map<String, String> configuration)
+ throws HyracksDataException {
+ this.serviceCtx = serviceCtx;
+ this.configuration = configuration;
+ init((ICCServiceContext) serviceCtx);
+ return HDFSUtils.configureHDFSJobConf(configuration);
+ }
+
+ protected void configureHdfsConf(JobConf conf, Map<String, String> configuration) throws AlgebricksException {
try {
- this.serviceCtx = serviceCtx;
- this.configuration = configuration;
- init((ICCServiceContext) serviceCtx);
- JobConf conf = HDFSUtils.configureHDFSJobConf(configuration);
confFactory = new ConfFactory(conf);
clusterLocations = getPartitionConstraint();
int numPartitions = clusterLocations.getLocations().length;
// if files list was set, we restrict the splits to the list
InputSplit[] inputSplits;
if (files == null) {
- inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+ inputSplits = getInputSplits(conf, numPartitions);
} else {
inputSplits = HDFSUtils.getSplits(conf, files);
}
@@ -108,11 +120,12 @@
read = new boolean[readSchedule.length];
Arrays.fill(read, false);
String formatString = configuration.get(ExternalDataConstants.KEY_FORMAT);
- if (formatString == null || formatString.equals(ExternalDataConstants.FORMAT_HDFS_WRITABLE)
- || formatString.equals(ExternalDataConstants.FORMAT_NOOP)) {
+ if (formatString == null || formatString.equals(ExternalDataConstants.FORMAT_HDFS_WRITABLE)) {
RecordReader<?, ?> reader = conf.getInputFormat().getRecordReader(inputSplits[0], conf, Reporter.NULL);
this.recordClass = reader.createValue().getClass();
reader.close();
+ } else if (formatString.equals(ExternalDataConstants.FORMAT_PARQUET)) {
+ recordClass = IValueReference.class;
} else {
recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
this.recordClass = char[].class;
@@ -122,6 +135,13 @@
}
}
+ private InputSplit[] getInputSplits(JobConf conf, int numPartitions) throws IOException {
+ if (HDFSUtils.isEmpty(conf)) {
+ return Scheduler.EMPTY_INPUT_SPLITS;
+ }
+ return conf.getInputFormat().getSplits(conf, numPartitions);
+ }
+
// Used to tell the factory to restrict the splits to the intersection between this list a
// actual files on hde
@Override
@@ -217,7 +237,19 @@
}
}
restoreConfig(ctx);
- return createRecordReader(configuration, read, inputSplits, readSchedule, nodeName, conf, files, indexer);
+ JobConf readerConf = conf;
+ if (ctx.getWarningCollector().shouldWarn()
+ && configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT.trim())
+ .equals(ExternalDataConstants.INPUT_FORMAT_PARQUET)) {
+ /*
+ * JobConf is used to pass warnings from the ParquetReadSupport to ParquetReader. As multiple
+ * partitions can issue different warnings, we might have a race condition on JobConf. Thus, we
+ * should create a copy when warnings are enabled.
+ */
+ readerConf = confFactory.getConf();
+ }
+ return createRecordReader(configuration, read, inputSplits, readSchedule, nodeName, readerConf, files,
+ indexer, ctx.getWarningCollector());
} catch (Exception e) {
throw HyracksDataException.create(e);
}
@@ -245,10 +277,10 @@
private static IRecordReader<? extends Object> createRecordReader(Map<String, String> configuration, boolean[] read,
InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf, List<ExternalFile> files,
- IExternalIndexer indexer) throws IOException {
+ IExternalIndexer indexer, IWarningCollector warningCollector) throws IOException {
if (configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT.trim())
.equals(ExternalDataConstants.INPUT_FORMAT_PARQUET)) {
- return new ParquetFileRecordReader<>(read, inputSplits, readSchedule, nodeName, conf);
+ return new ParquetFileRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, warningCollector);
} else {
return new HDFSRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, files, indexer);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java
index 2a063bf..eac4835 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/abstracts/AbstractExternalInputStreamFactory.java
@@ -18,24 +18,16 @@
*/
package org.apache.asterix.external.input.record.reader.abstracts;
-import static org.apache.asterix.external.util.ExternalDataConstants.*;
-
import java.io.Serializable;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.function.BiPredicate;
import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
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.external.api.AsterixInputStream;
import org.apache.asterix.external.api.IInputStreamFactory;
-import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
@@ -82,39 +74,6 @@
((ICcApplicationContext) ctx.getApplicationContext()).getClusterStateManager().getClusterLocations();
}
- protected IncludeExcludeMatcher getIncludeExcludeMatchers() throws CompilationException {
- // Get and compile the patterns for include/exclude if provided
- List<Matcher> includeMatchers = new ArrayList<>();
- List<Matcher> excludeMatchers = new ArrayList<>();
- String pattern = null;
- try {
- for (Map.Entry<String, String> entry : configuration.entrySet()) {
- if (entry.getKey().startsWith(KEY_INCLUDE)) {
- pattern = entry.getValue();
- includeMatchers.add(Pattern.compile(ExternalDataUtils.patternToRegex(pattern)).matcher(""));
- } else if (entry.getKey().startsWith(KEY_EXCLUDE)) {
- pattern = entry.getValue();
- excludeMatchers.add(Pattern.compile(ExternalDataUtils.patternToRegex(pattern)).matcher(""));
- }
- }
- } catch (PatternSyntaxException ex) {
- throw new CompilationException(ErrorCode.INVALID_REGEX_PATTERN, pattern);
- }
-
- IncludeExcludeMatcher includeExcludeMatcher;
- if (!includeMatchers.isEmpty()) {
- includeExcludeMatcher = new IncludeExcludeMatcher(includeMatchers,
- (matchers1, key) -> ExternalDataUtils.matchPatterns(matchers1, key));
- } else if (!excludeMatchers.isEmpty()) {
- includeExcludeMatcher = new IncludeExcludeMatcher(excludeMatchers,
- (matchers1, key) -> !ExternalDataUtils.matchPatterns(matchers1, key));
- } else {
- includeExcludeMatcher = new IncludeExcludeMatcher(Collections.emptyList(), (matchers1, key) -> true);
- }
-
- return includeExcludeMatcher;
- }
-
public static class PartitionWorkLoadBasedOnSize implements Serializable {
private static final long serialVersionUID = 1L;
private final List<String> filePaths = new ArrayList<>();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
index 5eb8475..f14af53 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -19,6 +19,7 @@
package org.apache.asterix.external.input.record.reader.aws;
import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
import java.io.IOException;
import java.util.List;
@@ -90,7 +91,7 @@
return false;
} catch (S3Exception ex) {
if (!shouldRetry(ex.awsErrorDetails().errorCode(), retries++)) {
- throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
}
LOGGER.debug(() -> "S3 retryable error: " + LogRedactionUtil.userData(ex.getMessage()));
@@ -101,7 +102,7 @@
Thread.currentThread().interrupt();
}
} catch (SdkException ex) {
- throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
}
}
return true;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
index 8197524..89ea39e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -18,35 +18,20 @@
*/
package org.apache.asterix.external.input.record.reader.aws;
-import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
-import java.util.function.BiPredicate;
-import java.util.regex.Matcher;
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
-import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.util.CleanupUtils;
-import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.services.s3.S3Client;
-import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
-import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
-import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
-import software.amazon.awssdk.services.s3.model.S3Exception;
import software.amazon.awssdk.services.s3.model.S3Object;
public class AwsS3InputStreamFactory extends AbstractExternalInputStreamFactory {
@@ -65,156 +50,27 @@
// Ensure the validity of include/exclude
ExternalDataUtils.validateIncludeExclude(configuration);
- IncludeExcludeMatcher includeExcludeMatcher = getIncludeExcludeMatchers();
+ IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
- // Prepare to retrieve the objects
- List<S3Object> filesOnly;
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- S3Client s3Client = ExternalDataUtils.AwsS3.buildAwsS3Client(configuration);
-
- try {
- filesOnly = listS3Objects(s3Client, container, includeExcludeMatcher);
- } catch (S3Exception ex) {
- // New API is not implemented, try falling back to old API
- try {
- // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
- if (ex.awsErrorDetails().errorCode().equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
- filesOnly = oldApiListS3Objects(s3Client, container, includeExcludeMatcher);
- } else {
- throw ex;
- }
- } catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex2.getMessage());
- }
- } catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
- } finally {
- if (s3Client != null) {
- CleanupUtils.close(s3Client, null);
- }
- }
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
-
+ //Get a list of S3 objects
+ List<S3Object> filesOnly =
+ ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
// Distribute work load amongst the partitions
distributeWorkLoad(filesOnly, getPartitionsCount());
}
/**
- * Uses the latest API to retrieve the objects from the storage.
- *
- * @param s3Client S3 client
- * @param container container name
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private List<S3Object> listS3Objects(S3Client s3Client, String container,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsV2Response listObjectsResponse;
- ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
- listObjectsBuilder.prefix(ExternalDataUtils.getPrefix(configuration));
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
- } else {
- listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextContinuationToken();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
- *
- * @param s3Client S3 client
- * @param container container name
- * @param includeExcludeMatcher include/exclude matchers to apply
- */
- private List<S3Object> oldApiListS3Objects(S3Client s3Client, String container,
- IncludeExcludeMatcher includeExcludeMatcher) {
- String newMarker = null;
- List<S3Object> filesOnly = new ArrayList<>();
-
- ListObjectsResponse listObjectsResponse;
- ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
- listObjectsBuilder.prefix(ExternalDataUtils.getPrefix(configuration));
-
- while (true) {
- // List the objects from the start, or from the last marker in case of truncated result
- if (newMarker == null) {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
- } else {
- listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
- }
-
- // Collect the paths to files only
- collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
- if (!listObjectsResponse.isTruncated()) {
- break;
- } else {
- newMarker = listObjectsResponse.nextMarker();
- }
- }
-
- return filesOnly;
- }
-
- /**
- * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
- * a file if it does not end up with a "/" which is the separator in a folder structure.
- *
- * @param s3Objects List of returned objects
- */
- private void collectAndFilterFiles(List<S3Object> s3Objects, BiPredicate<List<Matcher>, String> predicate,
- List<Matcher> matchers, List<S3Object> filesOnly) {
- for (S3Object object : s3Objects) {
- // skip folders
- if (object.key().endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, object.key())) {
- filesOnly.add(object);
- }
- }
- }
-
- /**
* To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
* size.
- *
+ * <p>
* Example:
* File1 1mb, File2 300kb, File3 300kb, File4 300kb
- *
+ * <p>
* Distribution:
* Partition1: [File1]
* Partition2: [File2, File3, File4]
*
- * @param fileObjects AWS S3 file objects
+ * @param fileObjects AWS S3 file objects
* @param partitionsCount Partitions count
*/
private void distributeWorkLoad(List<S3Object> fileObjects, int partitionsCount) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
new file mode 100644
index 0000000..803e657
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/parquet/AwsS3ParquetReaderFactory.java
@@ -0,0 +1,114 @@
+/*
+ * 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.external.input.record.reader.aws.parquet;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.external.input.HDFSDataSourceFactory;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+public class AwsS3ParquetReaderFactory extends HDFSDataSourceFactory {
+ private static final long serialVersionUID = -6140824803254158253L;
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3);
+
+ @Override
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
+ //Get path
+ String path = buildPathURIs(configuration, warningCollector);
+ //Put S3 configurations to AsterixDB's Hadoop configuration
+ putS3ConfToHadoopConf(configuration, path);
+
+ //Configure Hadoop S3 input splits
+ JobConf conf = createHdfsConf(serviceCtx, configuration);
+ int numberOfPartitions = getPartitionConstraint().getLocations().length;
+ ExternalDataUtils.AwsS3.configureAwsS3HdfsJobConf(conf, configuration, numberOfPartitions);
+ configureHdfsConf(conf, configuration);
+ }
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public Set<String> getReaderSupportedFormats() {
+ return Collections.singleton(ExternalDataConstants.FORMAT_PARQUET);
+ }
+
+ /**
+ * Prepare Hadoop configurations to read parquet files
+ *
+ * @param path Comma-delimited paths
+ */
+ private static void putS3ConfToHadoopConf(Map<String, String> configuration, String path) {
+ configuration.put(ExternalDataConstants.KEY_PATH, path);
+ configuration.put(ExternalDataConstants.KEY_INPUT_FORMAT, ExternalDataConstants.INPUT_FORMAT_PARQUET);
+ configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_NOOP);
+ }
+
+ /**
+ * Build S3 path-style for the requested files
+ *
+ * @param configuration properties
+ * @param warningCollector warning collector
+ * @return Comma-delimited paths (e.g., "s3a://bucket/file1.parquet,s3a://bucket/file2.parquet")
+ * @throws CompilationException Compilation exception
+ */
+ private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector)
+ throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+ List<S3Object> filesOnly =
+ ExternalDataUtils.AwsS3.listS3Objects(configuration, includeExcludeMatcher, warningCollector);
+ StringBuilder builder = new StringBuilder();
+
+ if (!filesOnly.isEmpty()) {
+ appendFileURI(builder, container, filesOnly.get(0));
+ for (int i = 1; i < filesOnly.size(); i++) {
+ builder.append(',');
+ appendFileURI(builder, container, filesOnly.get(i));
+ }
+ }
+
+ return builder.toString();
+ }
+
+ private static void appendFileURI(StringBuilder builder, String container, S3Object file) {
+ builder.append(ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL);
+ builder.append("://");
+ builder.append(container);
+ builder.append('/');
+ builder.append(file.key());
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobInputStream.java
deleted file mode 100644
index 3fb3395..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobInputStream.java
+++ /dev/null
@@ -1,89 +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.
- */
-package org.apache.asterix.external.input.record.reader.azure;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.zip.GZIPInputStream;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.util.LogRedactionUtil;
-
-import com.azure.storage.blob.BlobClient;
-import com.azure.storage.blob.BlobContainerClient;
-import com.azure.storage.blob.BlobServiceClient;
-import com.azure.storage.blob.models.BlobErrorCode;
-import com.azure.storage.blob.models.BlobStorageException;
-
-public class AzureBlobInputStream extends AbstractExternalInputStream {
-
- private final BlobServiceClient client;
- private final String container;
-
- public AzureBlobInputStream(Map<String, String> configuration, List<String> filePaths) throws HyracksDataException {
- super(configuration, filePaths);
- this.client = buildAzureClient(configuration);
- this.container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- }
-
- @Override
- protected boolean getInputStream() throws IOException {
- String fileName = filePaths.get(nextFileIndex);
- BlobContainerClient blobContainerClient;
- BlobClient blob;
- try {
- blobContainerClient = client.getBlobContainerClient(container);
- blob = blobContainerClient.getBlobClient(filePaths.get(nextFileIndex));
- in = blob.openInputStream();
-
- // Use gzip stream if needed
- String lowerCaseFileName = fileName.toLowerCase();
- if (lowerCaseFileName.endsWith(".gz") || lowerCaseFileName.endsWith(".gzip")) {
- in = new GZIPInputStream(in, ExternalDataConstants.DEFAULT_BUFFER_SIZE);
- }
- } catch (BlobStorageException ex) {
- if (ex.getErrorCode().equals(BlobErrorCode.BLOB_NOT_FOUND)) {
- LOGGER.debug(() -> "Key " + LogRedactionUtil.userData(filePaths.get(nextFileIndex)) + " was not "
- + "found in container " + container);
- return false;
- } else {
- throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
- }
- } catch (Exception ex) {
- throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
- }
-
- return true;
- }
-
- private BlobServiceClient buildAzureClient(Map<String, String> configuration) throws HyracksDataException {
- try {
- return ExternalDataUtils.Azure.buildAzureClient(configuration);
- } catch (CompilationException ex) {
- throw HyracksDataException.create(ex);
- }
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobInputStreamFactory.java
deleted file mode 100644
index 803694a..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobInputStreamFactory.java
+++ /dev/null
@@ -1,151 +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.
- */
-package org.apache.asterix.external.input.record.reader.azure;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.function.BiPredicate;
-import java.util.regex.Matcher;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.external.api.AsterixInputStream;
-import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.ExternalDataUtils;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.application.IServiceContext;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
-
-import com.azure.storage.blob.BlobContainerClient;
-import com.azure.storage.blob.BlobServiceClient;
-import com.azure.storage.blob.models.BlobItem;
-import com.azure.storage.blob.models.ListBlobsOptions;
-
-public class AzureBlobInputStreamFactory extends AbstractExternalInputStreamFactory {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public AsterixInputStream createInputStream(IHyracksTaskContext ctx, int partition) throws HyracksDataException {
- return new AzureBlobInputStream(configuration, partitionWorkLoadsBasedOnSize.get(partition).getFilePaths());
- }
-
- @Override
- public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
- throws AlgebricksException {
- super.configure(ctx, configuration, warningCollector);
-
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-
- List<BlobItem> filesOnly = new ArrayList<>();
-
- // Ensure the validity of include/exclude
- ExternalDataUtils.validateIncludeExclude(configuration);
-
- BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureClient(configuration);
- BlobContainerClient blobContainer;
- try {
- blobContainer = blobServiceClient.getBlobContainerClient(container);
-
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(ExternalDataUtils.getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- // Collect the paths to files only
- IncludeExcludeMatcher includeExcludeMatcher = getIncludeExcludeMatchers();
- collectAndFilterFiles(blobItems, includeExcludeMatcher.getPredicate(),
- includeExcludeMatcher.getMatchersList(), filesOnly);
-
- // Warn if no files are returned
- if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
- Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
- warningCollector.warn(warning);
- }
-
- // Distribute work load amongst the partitions
- distributeWorkLoad(filesOnly, getPartitionsCount());
- } catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
- }
- }
-
- /**
- * Collects and filters the files only, and excludes any folders
- *
- * @param items storage items
- * @param predicate predicate to test with for file filtration
- * @param matchers include/exclude matchers to test against
- * @param filesOnly List containing the files only (excluding folders)
- */
- private void collectAndFilterFiles(Iterable<BlobItem> items, BiPredicate<List<Matcher>, String> predicate,
- List<Matcher> matchers, List<BlobItem> filesOnly) {
- for (BlobItem item : items) {
- String uri = item.getName();
-
- // skip folders
- if (uri.endsWith("/")) {
- continue;
- }
-
- // No filter, add file
- if (predicate.test(matchers, uri)) {
- filesOnly.add(item);
- }
- }
- }
-
- /**
- * To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
- * size.
- *
- * Example:
- * File1 1mb, File2 300kb, File3 300kb, File4 300kb
- *
- * Distribution:
- * Partition1: [File1]
- * Partition2: [File2, File3, File4]
- *
- * @param items items
- * @param partitionsCount Partitions count
- */
- private void distributeWorkLoad(List<BlobItem> items, int partitionsCount) {
- PriorityQueue<PartitionWorkLoadBasedOnSize> workloadQueue = new PriorityQueue<>(partitionsCount,
- Comparator.comparingLong(PartitionWorkLoadBasedOnSize::getTotalSize));
-
- // Prepare the workloads based on the number of partitions
- for (int i = 0; i < partitionsCount; i++) {
- workloadQueue.add(new PartitionWorkLoadBasedOnSize());
- }
-
- for (BlobItem object : items) {
- PartitionWorkLoadBasedOnSize workload = workloadQueue.poll();
- workload.addFilePath(object.getName(), object.getProperties().getContentLength());
- workloadQueue.add(workload);
- }
- partitionWorkLoadsBasedOnSize.addAll(workloadQueue);
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobReaderFactory.java
deleted file mode 100644
index 27e1b02..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/AzureBlobReaderFactory.java
+++ /dev/null
@@ -1,73 +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.
- */
-package org.apache.asterix.external.input.record.reader.azure;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.input.record.reader.stream.StreamRecordReaderFactory;
-import org.apache.asterix.external.provider.StreamRecordReaderProvider;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.application.IServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-
-public class AzureBlobReaderFactory extends StreamRecordReaderFactory {
-
- private static final long serialVersionUID = 1L;
-
- private static final List<String> recordReaderNames =
- Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB);
-
- @Override
- public List<String> getRecordReaderNames() {
- return recordReaderNames;
- }
-
- @Override
- public DataSourceType getDataSourceType() {
- return DataSourceType.RECORDS;
- }
-
- @Override
- public Class<?> getRecordClass() {
- return char[].class;
- }
-
- @Override
- public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException {
- return streamFactory.getPartitionConstraint();
- }
-
- @Override
- public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
- throws AlgebricksException, HyracksDataException {
- this.configuration = configuration;
-
- // Stream factory
- streamFactory = new AzureBlobInputStreamFactory();
- streamFactory.configure(ctx, configuration, warningCollector);
-
- // record reader
- recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
new file mode 100644
index 0000000..b402f25
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStream.java
@@ -0,0 +1,91 @@
+/*
+ * 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.external.input.record.reader.azure.blob;
+
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.LogRedactionUtil;
+
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobContainerClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobErrorCode;
+import com.azure.storage.blob.models.BlobStorageException;
+
+public class AzureBlobInputStream extends AbstractExternalInputStream {
+
+ private final BlobServiceClient client;
+ private final String container;
+
+ public AzureBlobInputStream(Map<String, String> configuration, List<String> filePaths) throws HyracksDataException {
+ super(configuration, filePaths);
+ this.client = buildAzureClient(configuration);
+ this.container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ }
+
+ @Override
+ protected boolean getInputStream() throws IOException {
+ String fileName = filePaths.get(nextFileIndex);
+ BlobContainerClient blobContainerClient;
+ BlobClient blob;
+ try {
+ blobContainerClient = client.getBlobContainerClient(container);
+ blob = blobContainerClient.getBlobClient(filePaths.get(nextFileIndex));
+ in = blob.openInputStream();
+
+ // Use gzip stream if needed
+ String lowerCaseFileName = fileName.toLowerCase();
+ if (lowerCaseFileName.endsWith(".gz") || lowerCaseFileName.endsWith(".gzip")) {
+ in = new GZIPInputStream(in, ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+ }
+ } catch (BlobStorageException ex) {
+ if (ex.getErrorCode().equals(BlobErrorCode.BLOB_NOT_FOUND)) {
+ LOGGER.debug(() -> "Key " + LogRedactionUtil.userData(filePaths.get(nextFileIndex)) + " was not "
+ + "found in container " + container);
+ return false;
+ } else {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (Exception ex) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return true;
+ }
+
+ private BlobServiceClient buildAzureClient(Map<String, String> configuration) throws HyracksDataException {
+ try {
+ return ExternalDataUtils.Azure.buildAzureBlobClient(configuration);
+ } catch (CompilationException ex) {
+ throw HyracksDataException.create(ex);
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
new file mode 100644
index 0000000..bf904a4
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobInputStreamFactory.java
@@ -0,0 +1,93 @@
+/*
+ * 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.external.input.record.reader.azure.blob;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+import org.apache.asterix.external.api.AsterixInputStream;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobItem;
+
+public class AzureBlobInputStreamFactory extends AbstractExternalInputStreamFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public AsterixInputStream createInputStream(IHyracksTaskContext ctx, int partition) throws HyracksDataException {
+ return new AzureBlobInputStream(configuration, partitionWorkLoadsBasedOnSize.get(partition).getFilePaths());
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException {
+ super.configure(ctx, configuration, warningCollector);
+
+ // Ensure the validity of include/exclude
+ ExternalDataUtils.validateIncludeExclude(configuration);
+ IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+ BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(configuration);
+ List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
+ includeExcludeMatcher, warningCollector);
+
+ // Distribute work load amongst the partitions
+ distributeWorkLoad(filesOnly, getPartitionsCount());
+ }
+
+ /**
+ * To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
+ * size.
+ * <p>
+ * Example:
+ * File1 1mb, File2 300kb, File3 300kb, File4 300kb
+ * <p>
+ * Distribution:
+ * Partition1: [File1]
+ * Partition2: [File2, File3, File4]
+ *
+ * @param items items
+ * @param partitionsCount Partitions count
+ */
+ private void distributeWorkLoad(List<BlobItem> items, int partitionsCount) {
+ PriorityQueue<PartitionWorkLoadBasedOnSize> workloadQueue = new PriorityQueue<>(partitionsCount,
+ Comparator.comparingLong(PartitionWorkLoadBasedOnSize::getTotalSize));
+
+ // Prepare the workloads based on the number of partitions
+ for (int i = 0; i < partitionsCount; i++) {
+ workloadQueue.add(new PartitionWorkLoadBasedOnSize());
+ }
+
+ for (BlobItem object : items) {
+ PartitionWorkLoadBasedOnSize workload = workloadQueue.poll();
+ workload.addFilePath(object.getName(), object.getProperties().getContentLength());
+ workloadQueue.add(workload);
+ }
+ partitionWorkLoadsBasedOnSize.addAll(workloadQueue);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobReaderFactory.java
new file mode 100644
index 0000000..525ee63
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/blob/AzureBlobReaderFactory.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.azure.blob;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.input.record.reader.stream.StreamRecordReaderFactory;
+import org.apache.asterix.external.provider.StreamRecordReaderProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+public class AzureBlobReaderFactory extends StreamRecordReaderFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB);
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public DataSourceType getDataSourceType() {
+ return DataSourceType.RECORDS;
+ }
+
+ @Override
+ public Class<?> getRecordClass() {
+ return char[].class;
+ }
+
+ @Override
+ public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException {
+ return streamFactory.getPartitionConstraint();
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException, HyracksDataException {
+ this.configuration = configuration;
+
+ // Stream factory
+ streamFactory = new AzureBlobInputStreamFactory();
+ streamFactory.configure(ctx, configuration, warningCollector);
+
+ // record reader
+ recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
new file mode 100644
index 0000000..b7d142f
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStream.java
@@ -0,0 +1,92 @@
+/*
+ * 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.external.input.record.reader.azure.datalake;
+
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.LogRedactionUtil;
+
+import com.azure.storage.blob.models.BlobErrorCode;
+import com.azure.storage.blob.models.BlobStorageException;
+import com.azure.storage.file.datalake.DataLakeFileClient;
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+
+public class AzureDataLakeInputStream extends AbstractExternalInputStream {
+
+ private final DataLakeServiceClient client;
+ private final String container;
+
+ public AzureDataLakeInputStream(Map<String, String> configuration, List<String> filePaths)
+ throws HyracksDataException {
+ super(configuration, filePaths);
+ this.client = buildAzureClient(configuration);
+ this.container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ }
+
+ @Override
+ protected boolean getInputStream() throws IOException {
+ String fileName = filePaths.get(nextFileIndex);
+ DataLakeFileSystemClient fileSystemClient;
+ DataLakeFileClient fileClient;
+ try {
+ fileSystemClient = client.getFileSystemClient(container);
+ fileClient = fileSystemClient.getFileClient(filePaths.get(nextFileIndex));
+ in = fileClient.openInputStream().getInputStream();
+
+ // Use gzip stream if needed
+ String lowerCaseFileName = fileName.toLowerCase();
+ if (lowerCaseFileName.endsWith(".gz") || lowerCaseFileName.endsWith(".gzip")) {
+ in = new GZIPInputStream(in, ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+ }
+ } catch (BlobStorageException ex) {
+ if (ex.getErrorCode().equals(BlobErrorCode.BLOB_NOT_FOUND)) {
+ LOGGER.debug(() -> "Key " + LogRedactionUtil.userData(filePaths.get(nextFileIndex)) + " was not "
+ + "found in container " + container);
+ return false;
+ } else {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (Exception ex) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return true;
+ }
+
+ private DataLakeServiceClient buildAzureClient(Map<String, String> configuration) throws HyracksDataException {
+ try {
+ return ExternalDataUtils.Azure.buildAzureDatalakeClient(configuration);
+ } catch (CompilationException ex) {
+ throw HyracksDataException.create(ex);
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
new file mode 100644
index 0000000..e145e1f
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeInputStreamFactory.java
@@ -0,0 +1,93 @@
+/*
+ * 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.external.input.record.reader.azure.datalake;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+import org.apache.asterix.external.api.AsterixInputStream;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.PathItem;
+
+public class AzureDataLakeInputStreamFactory extends AbstractExternalInputStreamFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public AsterixInputStream createInputStream(IHyracksTaskContext ctx, int partition) throws HyracksDataException {
+ return new AzureDataLakeInputStream(configuration, partitionWorkLoadsBasedOnSize.get(partition).getFilePaths());
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException {
+ super.configure(ctx, configuration, warningCollector);
+
+ // Ensure the validity of include/exclude
+ ExternalDataUtils.validateIncludeExclude(configuration);
+ IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+ DataLakeServiceClient client = ExternalDataUtils.Azure.buildAzureDatalakeClient(configuration);
+ List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(client, configuration,
+ includeExcludeMatcher, warningCollector);
+
+ // Distribute work load amongst the partitions
+ distributeWorkLoad(filesOnly, getPartitionsCount());
+ }
+
+ /**
+ * To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
+ * size.
+ * <p>
+ * Example:
+ * File1 1mb, File2 300kb, File3 300kb, File4 300kb
+ * <p>
+ * Distribution:
+ * Partition1: [File1]
+ * Partition2: [File2, File3, File4]
+ *
+ * @param items items
+ * @param partitionsCount Partitions count
+ */
+ private void distributeWorkLoad(List<PathItem> items, int partitionsCount) {
+ PriorityQueue<PartitionWorkLoadBasedOnSize> workloadQueue = new PriorityQueue<>(partitionsCount,
+ Comparator.comparingLong(PartitionWorkLoadBasedOnSize::getTotalSize));
+
+ // Prepare the workloads based on the number of partitions
+ for (int i = 0; i < partitionsCount; i++) {
+ workloadQueue.add(new PartitionWorkLoadBasedOnSize());
+ }
+
+ for (PathItem object : items) {
+ PartitionWorkLoadBasedOnSize workload = workloadQueue.poll();
+ workload.addFilePath(object.getName(), object.getContentLength());
+ workloadQueue.add(workload);
+ }
+ partitionWorkLoadsBasedOnSize.addAll(workloadQueue);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeReaderFactory.java
new file mode 100644
index 0000000..594bacf
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/datalake/AzureDataLakeReaderFactory.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.azure.datalake;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.input.record.reader.stream.StreamRecordReaderFactory;
+import org.apache.asterix.external.provider.StreamRecordReaderProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+public class AzureDataLakeReaderFactory extends StreamRecordReaderFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_DATA_LAKE);
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public DataSourceType getDataSourceType() {
+ return DataSourceType.RECORDS;
+ }
+
+ @Override
+ public Class<?> getRecordClass() {
+ return char[].class;
+ }
+
+ @Override
+ public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException {
+ return streamFactory.getPartitionConstraint();
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException, HyracksDataException {
+ this.configuration = configuration;
+
+ // Stream factory
+ streamFactory = new AzureDataLakeInputStreamFactory();
+ streamFactory.configure(ctx, configuration, warningCollector);
+
+ // record reader
+ recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
new file mode 100644
index 0000000..1f82dae
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureBlobParquetReaderFactory.java
@@ -0,0 +1,127 @@
+/*
+ * 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.external.input.record.reader.azure.parquet;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.external.input.HDFSDataSourceFactory;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.models.BlobItem;
+
+public class AzureBlobParquetReaderFactory extends HDFSDataSourceFactory {
+ private static final long serialVersionUID = -6140824803254158253L;
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB);
+
+ @Override
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
+ BlobServiceClient blobServiceClient = ExternalDataUtils.Azure.buildAzureBlobClient(configuration);
+ //Get endpoint
+ String endPoint = extractEndPoint(blobServiceClient.getAccountUrl());
+ //Get path
+ String path = buildPathURIs(configuration, warningCollector, blobServiceClient, endPoint);
+ //Put Azure configurations to AsterixDB's Hadoop configuration
+ putAzureBlobConfToHadoopConf(configuration, path);
+
+ //Configure Hadoop Azure input splits
+ JobConf conf = createHdfsConf(serviceCtx, configuration);
+ ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureHdfsConf(conf, configuration);
+ }
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public Set<String> getReaderSupportedFormats() {
+ return Collections.singleton(ExternalDataConstants.FORMAT_PARQUET);
+ }
+
+ /**
+ * Prepare Hadoop configurations to read parquet files
+ *
+ * @param path Comma-delimited paths
+ */
+ private static void putAzureBlobConfToHadoopConf(Map<String, String> configuration, String path) {
+ configuration.put(ExternalDataConstants.KEY_PATH, path);
+ configuration.put(ExternalDataConstants.KEY_INPUT_FORMAT, ExternalDataConstants.INPUT_FORMAT_PARQUET);
+ configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_NOOP);
+ }
+
+ /**
+ * Build Azure Blob Storage path-style for the requested files
+ *
+ * @param configuration properties
+ * @param warningCollector warning collector
+ * @return Comma-delimited paths (e.g., "wasbs://container@accountName.blob.core.windows.net/file1.parquet,
+ * wasbs://container@accountName.blob.core.windows.net/file2.parquet")
+ * @throws CompilationException Compilation exception
+ */
+ private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
+ BlobServiceClient blobServiceClient, String endPoint) throws CompilationException {
+ IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+ List<BlobItem> filesOnly = ExternalDataUtils.Azure.listBlobItems(blobServiceClient, configuration,
+ includeExcludeMatcher, warningCollector);
+
+ StringBuilder builder = new StringBuilder();
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ if (!filesOnly.isEmpty()) {
+ appendFileURI(builder, container, endPoint, filesOnly.get(0));
+ for (int i = 1; i < filesOnly.size(); i++) {
+ builder.append(',');
+ appendFileURI(builder, container, endPoint, filesOnly.get(i));
+ }
+ }
+
+ return builder.toString();
+ }
+
+ private static String extractEndPoint(String uri) {
+ //The URI is in the form http(s)://<accountName>.blob.core.windows.net
+ //We need to Remove the protocol (i.e., http(s)://) from the URI
+ return uri.substring(uri.indexOf("//") + "//".length());
+ }
+
+ private static void appendFileURI(StringBuilder builder, String container, String endPoint, BlobItem file) {
+ builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL);
+ builder.append("://");
+ builder.append(container);
+ builder.append('@');
+ builder.append(endPoint);
+ builder.append('/');
+ builder.append(file.getName());
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
new file mode 100644
index 0000000..8474a74
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/azure/parquet/AzureDataLakeParquetReaderFactory.java
@@ -0,0 +1,130 @@
+/*
+ * 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.external.input.record.reader.azure.parquet;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.external.input.HDFSDataSourceFactory;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.PathItem;
+
+public class AzureDataLakeParquetReaderFactory extends HDFSDataSourceFactory {
+ private static final long serialVersionUID = -6140824803254158253L;
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_DATA_LAKE);
+
+ @Override
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AlgebricksException, HyracksDataException {
+ DataLakeServiceClient dataLakeServiceClient = ExternalDataUtils.Azure.buildAzureDatalakeClient(configuration);
+
+ //Get endpoint
+ String endPoint = extractEndPoint(dataLakeServiceClient.getAccountUrl());
+
+ //Get path
+ String path = buildPathURIs(configuration, warningCollector, dataLakeServiceClient, endPoint);
+
+ //Put Azure configurations to AsterixDB's Hadoop configuration
+ putAzureDataLakeConfToHadoopConf(configuration, path);
+
+ //Configure Hadoop Azure input splits
+ JobConf conf = createHdfsConf(serviceCtx, configuration);
+ ExternalDataUtils.Azure.configureAzureHdfsJobConf(conf, configuration, endPoint);
+ configureHdfsConf(conf, configuration);
+ }
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public Set<String> getReaderSupportedFormats() {
+ return Collections.singleton(ExternalDataConstants.FORMAT_PARQUET);
+ }
+
+ /**
+ * Prepare Hadoop configurations to read parquet files
+ *
+ * @param path Comma-delimited paths
+ */
+ private static void putAzureDataLakeConfToHadoopConf(Map<String, String> configuration, String path) {
+ configuration.put(ExternalDataConstants.KEY_PATH, path);
+ configuration.put(ExternalDataConstants.KEY_INPUT_FORMAT, ExternalDataConstants.INPUT_FORMAT_PARQUET);
+ configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_NOOP);
+ }
+
+ /**
+ * Build Azure Datalake Storage path-style for the requested files
+ *
+ * @param configuration properties
+ * @param warningCollector warning collector
+ * @return Comma-delimited paths (e.g., "abfss://<container-name>@<accountName>.dfs.core.windows.net/file1.parquet,
+ * abfss://<container-name>@<accountName>.dfs.core.windows.net//file2.parquet")
+ * @throws CompilationException Compilation exception
+ */
+ private static String buildPathURIs(Map<String, String> configuration, IWarningCollector warningCollector,
+ DataLakeServiceClient dataLakeServiceClient, String endPoint) throws CompilationException {
+ IncludeExcludeMatcher includeExcludeMatcher = ExternalDataUtils.getIncludeExcludeMatchers(configuration);
+ List<PathItem> filesOnly = ExternalDataUtils.Azure.listDatalakePathItems(dataLakeServiceClient, configuration,
+ includeExcludeMatcher, warningCollector);
+
+ StringBuilder builder = new StringBuilder();
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ if (!filesOnly.isEmpty()) {
+ appendFileURI(builder, container, endPoint, filesOnly.get(0));
+ for (int i = 1; i < filesOnly.size(); i++) {
+ builder.append(',');
+ appendFileURI(builder, container, endPoint, filesOnly.get(i));
+ }
+ }
+
+ return builder.toString();
+ }
+
+ private static String extractEndPoint(String uri) {
+ //The URI is in the form http(s)://<accountName>.dfs.core.windows.net
+ //We need to Remove the protocol (i.e., http(s)://) from the URI
+ return uri.substring(uri.indexOf("//") + "//".length());
+ }
+
+ private static void appendFileURI(StringBuilder builder, String container, String endPoint, PathItem file) {
+ builder.append(ExternalDataConstants.Azure.HADOOP_AZURE_DATALAKE_PROTOCOL);
+ builder.append("://");
+ builder.append(container);
+ builder.append('@');
+ builder.append(endPoint);
+ builder.append('/');
+ builder.append(file.getName());
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
new file mode 100644
index 0000000..652fa3e
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStream.java
@@ -0,0 +1,142 @@
+/*
+ * 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.external.input.record.reader.gcs;
+
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStream;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
+import org.apache.hyracks.util.LogRedactionUtil;
+
+import com.google.cloud.BaseServiceException;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.BlobId;
+import com.google.cloud.storage.Storage;
+
+public class GCSInputStream extends AbstractExternalInputStream {
+
+ private final Storage client;
+ private final String container;
+ private static final int MAX_RETRIES = 5; // We will retry 5 times in case of retryable errors
+
+ public GCSInputStream(Map<String, String> configuration, List<String> filePaths) throws HyracksDataException {
+ super(configuration, filePaths);
+ this.client = buildClient(configuration);
+ this.container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ }
+
+ @Override
+ protected boolean getInputStream() throws IOException {
+ String fileName = filePaths.get(nextFileIndex);
+
+ // Have a reference to the input stream to ensure that if GZipInputStream causes an IOException because of
+ // reading the header, then the input stream gets closed in the close method
+ if (!doGetInputStream(fileName)) {
+ return false;
+ }
+
+ // Use gzip stream if needed
+ if (StringUtils.endsWithIgnoreCase(fileName, ".gz") || StringUtils.endsWithIgnoreCase(fileName, ".gzip")) {
+ in = new GZIPInputStream(in, ExternalDataConstants.DEFAULT_BUFFER_SIZE);
+ }
+ return true;
+ }
+
+ /**
+ * Get the input stream. If an error is encountered, depending on the error, a retry might be favorable.
+ *
+ * @return true
+ */
+ private boolean doGetInputStream(String fileName) throws RuntimeDataException {
+ int retries = 0;
+ BlobId blobId = BlobId.of(container, fileName);
+
+ while (retries < MAX_RETRIES) {
+ try {
+ Blob blob = client.get(blobId);
+ if (blob == null) {
+ // Object not found
+ LOGGER.debug(() -> "Key " + LogRedactionUtil.userData(fileName) + " was not found in container "
+ + container);
+ return false;
+ }
+ in = new ByteArrayInputStream(blob.getContent());
+ break;
+ } catch (BaseServiceException ex) {
+ if (!shouldRetry(retries++) && ex.isRetryable()) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ LOGGER.debug(() -> "Retryable error: " + LogRedactionUtil.userData(ex.getMessage()));
+
+ // Backoff for 1 sec for the first 2 retries, and 2 seconds from there onward
+ try {
+ Thread.sleep(TimeUnit.SECONDS.toMillis(retries < 3 ? 1 : 2));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ } catch (Exception ex) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+ return true;
+ }
+
+ private boolean shouldRetry(int currentRetry) {
+ return currentRetry < MAX_RETRIES;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (in != null) {
+ CleanupUtils.close(in, null);
+ }
+ }
+
+ @Override
+ public boolean stop() {
+ try {
+ close();
+ } catch (IOException e) {
+ // Ignore
+ }
+ return false;
+ }
+
+ private Storage buildClient(Map<String, String> configuration) throws HyracksDataException {
+ try {
+ return ExternalDataUtils.GCS.buildClient(configuration);
+ } catch (CompilationException ex) {
+ throw HyracksDataException.create(ex);
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
new file mode 100644
index 0000000..0e7ea90
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSInputStreamFactory.java
@@ -0,0 +1,146 @@
+/*
+ * 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.external.input.record.reader.gcs;
+
+import static org.apache.asterix.external.util.ExternalDataUtils.getIncludeExcludeMatchers;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.api.AsterixInputStream;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+
+import com.google.api.gax.paging.Page;
+import com.google.cloud.BaseServiceException;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.Storage;
+
+public class GCSInputStreamFactory extends AbstractExternalInputStreamFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public AsterixInputStream createInputStream(IHyracksTaskContext ctx, int partition) throws HyracksDataException {
+ return new GCSInputStream(configuration, partitionWorkLoadsBasedOnSize.get(partition).getFilePaths());
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException {
+ super.configure(ctx, configuration, warningCollector);
+
+ // Ensure the validity of include/exclude
+ ExternalDataUtils.validateIncludeExclude(configuration);
+
+ // Prepare to retrieve the objects
+ List<Blob> filesOnly = new ArrayList<>();
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ Storage gcs = ExternalDataUtils.GCS.buildClient(configuration);
+ Storage.BlobListOption options = Storage.BlobListOption.prefix(ExternalDataUtils.getPrefix(configuration));
+ Page<Blob> items;
+
+ try {
+ items = gcs.list(container, options);
+ } catch (BaseServiceException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ // Collect the paths to files only
+ IncludeExcludeMatcher includeExcludeMatcher = getIncludeExcludeMatchers(configuration);
+ collectAndFilterFiles(items, includeExcludeMatcher.getPredicate(), includeExcludeMatcher.getMatchersList(),
+ filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+
+ // Distribute work load amongst the partitions
+ distributeWorkLoad(filesOnly, getPartitionsCount());
+ }
+
+ /**
+ * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
+ * a file if it does not end up with a "/" which is the separator in a folder structure.
+ *
+ * @param items List of returned objects
+ */
+ private void collectAndFilterFiles(Page<Blob> items, BiPredicate<List<Matcher>, String> predicate,
+ List<Matcher> matchers, List<Blob> filesOnly) {
+ for (Blob item : items.iterateAll()) {
+ // skip folders
+ if (item.getName().endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, item.getName())) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ /**
+ * To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
+ * size.
+ *
+ * Example:
+ * File1 1mb, File2 300kb, File3 300kb, File4 300kb
+ *
+ * Distribution:
+ * Partition1: [File1]
+ * Partition2: [File2, File3, File4]
+ *
+ * @param items items
+ * @param partitionsCount Partitions count
+ */
+ private void distributeWorkLoad(List<Blob> items, int partitionsCount) {
+ PriorityQueue<PartitionWorkLoadBasedOnSize> workloadQueue = new PriorityQueue<>(partitionsCount,
+ Comparator.comparingLong(PartitionWorkLoadBasedOnSize::getTotalSize));
+
+ // Prepare the workloads based on the number of partitions
+ for (int i = 0; i < partitionsCount; i++) {
+ workloadQueue.add(new PartitionWorkLoadBasedOnSize());
+ }
+
+ for (Blob item : items) {
+ PartitionWorkLoadBasedOnSize workload = workloadQueue.poll();
+ workload.addFilePath(item.getName(), item.getSize());
+ workloadQueue.add(workload);
+ }
+ partitionWorkLoadsBasedOnSize.addAll(workloadQueue);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSReaderFactory.java
new file mode 100644
index 0000000..ca42892
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/gcs/GCSReaderFactory.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.gcs;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.input.record.reader.stream.StreamRecordReaderFactory;
+import org.apache.asterix.external.provider.StreamRecordReaderProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+public class GCSReaderFactory extends StreamRecordReaderFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_GCS);
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public DataSourceType getDataSourceType() {
+ return DataSourceType.RECORDS;
+ }
+
+ @Override
+ public Class<?> getRecordClass() {
+ return char[].class;
+ }
+
+ @Override
+ public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException {
+ return streamFactory.getPartitionConstraint();
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException, HyracksDataException {
+ this.configuration = configuration;
+
+ // Stream factory
+ streamFactory = new GCSInputStreamFactory();
+ streamFactory.configure(ctx, configuration, warningCollector);
+
+ // record reader
+ recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
index 8b258e4..75d431d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
@@ -96,8 +96,6 @@
return (ILookupRecordReader<? extends T>) new TextLookupReader(snapshotAccessor, fs, conf);
case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
return (ILookupRecordReader<? extends T>) new SequenceLookupReader(snapshotAccessor, fs, conf);
- case ExternalDataConstants.INPUT_FORMAT_RC:
- return (ILookupRecordReader<? extends T>) new RCLookupReader(snapshotAccessor, fs, conf);
default:
throw new HyracksDataException("Unrecognised input format: " + inputFormatParameter);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/RCLookupReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/RCLookupReader.java
deleted file mode 100644
index c878e03..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/RCLookupReader.java
+++ /dev/null
@@ -1,93 +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.
- */
-package org.apache.asterix.external.input.record.reader.hdfs;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.indexing.ExternalFileIndexAccessor;
-import org.apache.asterix.external.indexing.RecordId;
-import org.apache.asterix.external.input.record.GenericRecord;
-import org.apache.asterix.hivecompat.io.RCFile.Reader;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class RCLookupReader extends AbstractHDFSLookupRecordReader<BytesRefArrayWritable> {
- public RCLookupReader(ExternalFileIndexAccessor snapshotAccessor, FileSystem fs, Configuration conf) {
- super(snapshotAccessor, fs, conf);
- }
-
- private static final Logger LOGGER = LogManager.getLogger();
- private Reader reader;
- private LongWritable key = new LongWritable();
- private BytesRefArrayWritable value = new BytesRefArrayWritable();
- private GenericRecord<BytesRefArrayWritable> record = new GenericRecord<BytesRefArrayWritable>();
- private long offset;
- private int row;
-
- @Override
- public Class<?> getRecordClass() throws IOException {
- return Writable.class;
- }
-
- @Override
- protected IRawRecord<BytesRefArrayWritable> lookup(RecordId rid) throws IOException {
- if (rid.getOffset() != offset) {
- offset = rid.getOffset();
- if (reader.getPosition() != offset)
- reader.seek(offset);
- reader.resetBuffer();
- row = -1;
- }
-
- // skip rows to the record row
- while (row < rid.getRow()) {
- reader.next(key);
- reader.getCurrentRow(value);
- row++;
- }
- record.set(value);
- return record;
- }
-
- @Override
- protected void closeFile() {
- if (reader == null) {
- return;
- }
- try {
- reader.close();
- } catch (Exception e) {
- LOGGER.warn("Error closing HDFS file", e);
- }
- }
-
- @Override
- protected void openFile() throws IllegalArgumentException, IOException {
- reader = new Reader(fs, new Path(file.getFileName()), conf);
- offset = -1;
- row = -1;
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AbstractComplexConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AbstractComplexConverter.java
index 81b61f8..363d2d2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AbstractComplexConverter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AbstractComplexConverter.java
@@ -26,11 +26,11 @@
import org.apache.parquet.io.api.Converter;
import org.apache.parquet.io.api.GroupConverter;
import org.apache.parquet.schema.GroupType;
-import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
import org.apache.parquet.schema.Type;
import org.apache.parquet.schema.Type.Repetition;
-abstract class AbstractComplexConverter extends GroupConverter implements IFieldValue {
+public abstract class AbstractComplexConverter extends GroupConverter implements IFieldValue {
protected final AbstractComplexConverter parent;
private final IValueReference fieldName;
private final int index;
@@ -38,12 +38,11 @@
protected final ParserContext context;
protected IMutableValueStorage tempStorage;
- public AbstractComplexConverter(AbstractComplexConverter parent, int index, GroupType parquetType,
- ParserContext context) {
+ AbstractComplexConverter(AbstractComplexConverter parent, int index, GroupType parquetType, ParserContext context) {
this(parent, null, index, parquetType, context);
}
- public AbstractComplexConverter(AbstractComplexConverter parent, IValueReference fieldName, int index,
+ AbstractComplexConverter(AbstractComplexConverter parent, IValueReference fieldName, int index,
GroupType parquetType, ParserContext context) {
this.parent = parent;
this.fieldName = fieldName;
@@ -52,9 +51,11 @@
converters = new Converter[parquetType.getFieldCount()];
for (int i = 0; i < parquetType.getFieldCount(); i++) {
final Type type = parquetType.getType(i);
- if (type.isPrimitive()) {
+ if (type == AsterixTypeToParquetTypeVisitor.MISSING) {
+ converters[i] = MissingConverter.INSTANCE;
+ } else if (type.isPrimitive()) {
converters[i] = createAtomicConverter(parquetType, i);
- } else if (type.getOriginalType() == OriginalType.LIST) {
+ } else if (LogicalTypeAnnotation.listType().equals(type.getLogicalTypeAnnotation())) {
converters[i] = createArrayConverter(parquetType, i);
} else if (type.getRepetition() == Repetition.REPEATED) {
converters[i] = createRepeatedConverter(parquetType, i);
@@ -86,7 +87,6 @@
*
* In Avro:
* optional group urls (LIST) {
- * // if number of fields > 1, then should be treated as array of objects
* repeated group array {
* optional binary display_url (UTF8);
* optional binary expanded_url (UTF8);
@@ -106,12 +106,13 @@
*
* @formatter:on
*/
- public AbstractComplexConverter createRepeatedConverter(GroupType type, int index) {
- final GroupType arrayType = type.getType(index).asGroupType();
- if (arrayType.getFieldCount() == 1) {
- return new RepeatedConverter(this, index, arrayType, context);
+ private AbstractComplexConverter createRepeatedConverter(GroupType type, int index) {
+ GroupType repeatedType = type.getType(index).asGroupType();
+ //The name "array" is used by Avro to represent group element (array of objects)
+ if (repeatedType.getFieldCount() > 1 || "array".equals(repeatedType.getName())) {
+ return new ObjectConverter(this, index, repeatedType, context);
}
- return new ObjectConverter(this, index, arrayType, context);
+ return new RepeatedConverter(this, index, repeatedType, context);
}
@Override
@@ -149,5 +150,4 @@
}
parent.addValue(this);
}
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
new file mode 100644
index 0000000..c0a47d5
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AsterixTypeToParquetTypeVisitor.java
@@ -0,0 +1,209 @@
+/*
+ * 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.external.input.record.reader.hdfs.parquet;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.IATypeVisitor;
+import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.LogicalTypeAnnotation;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Types;
+
+/**
+ * This visitor clips the filesSchema stored in Parquet using the expected type
+ */
+public class AsterixTypeToParquetTypeVisitor implements IATypeVisitor<Type, Type> {
+ public static final MessageType EMPTY_PARQUET_MESSAGE = Types.buildMessage().named("EMPTY");
+ public static final PrimitiveType MISSING =
+ Types.optional(PrimitiveType.PrimitiveTypeName.BOOLEAN).named("MISSING");
+
+ private final List<Warning> warnings;
+ private Map<String, FunctionCallInformation> funcInfo;
+
+ public AsterixTypeToParquetTypeVisitor() {
+ warnings = new ArrayList<>();
+ }
+
+ public MessageType clipType(ARecordType rootType, MessageType fileSchema,
+ Map<String, FunctionCallInformation> funcInfo) {
+ if (rootType == DataProjectionInfo.EMPTY_TYPE) {
+ return EMPTY_PARQUET_MESSAGE;
+ } else if (rootType == DataProjectionInfo.ALL_FIELDS_TYPE) {
+ return fileSchema;
+ }
+ Types.MessageTypeBuilder builder = Types.buildMessage();
+ this.funcInfo = funcInfo;
+ clipObjectChildren(builder, rootType, fileSchema);
+ return builder.named(fileSchema.getName());
+ }
+
+ public List<Warning> getWarnings() {
+ return warnings;
+ }
+
+ @Override
+ public Type visit(ARecordType recordType, Type arg) {
+ //No LogicalTypeAnnotation for Object types
+ if (isNotCompatibleType(arg, recordType)) {
+ return MISSING;
+ }
+ Types.GroupBuilder<GroupType> builder = Types.buildGroup(arg.getRepetition());
+ if (clipObjectChildren(builder, recordType, arg) == 0) {
+ //If no fields where added, add MISSING as parquet does not allow empty objects.
+ builder.addField(MISSING);
+ }
+ return builder.named(arg.getName());
+ }
+
+ /**
+ * There are two ways for representing arrays using ProtoBuf schema see the example in
+ * {@link org.apache.asterix.external.input.record.reader.hdfs.parquet.AbstractComplexConverter} for more
+ * information.
+ */
+ @Override
+ public Type visit(AbstractCollectionType collectionType, Type arg) {
+ if (isNotCompatibleType(arg, collectionType)) {
+ return MISSING;
+ }
+ GroupType arrayType = arg.asGroupType();
+ //There is only one child
+ Type childType = arrayType.getType(0);
+ if ("array".equals(childType.getName()) || childType.asGroupType().getFieldCount() > 1) {
+ //Handle Avro-like schema
+ return handleHandleAvroArray(collectionType, arrayType);
+ }
+ //Handling spark-like schema
+ Types.ListBuilder<GroupType> builder = Types.list(arg.getRepetition());
+ //LIST items always wrapped in a group called list
+ childType = childType.asGroupType().getType(0);
+ Type requestedChildType = collectionType.getItemType().accept(this, childType);
+ //setElementType will wrap childType again in a group type
+ builder.setElementType(requestedChildType);
+ return builder.named(arg.getName());
+ }
+
+ private int clipObjectChildren(Types.GroupBuilder<?> builder, ARecordType recordType, Type arg) {
+ GroupType groupType = arg.asGroupType();
+ String[] fieldNames = recordType.getFieldNames();
+ IAType[] fieldTypes = recordType.getFieldTypes();
+ int numberOfAddedFields = 0;
+ for (int i = 0; i < fieldNames.length; i++) {
+ Type type = getType(groupType, fieldNames[i]);
+ Type childType = fieldTypes[i].accept(this, type);
+ if (childType != MISSING) {
+ //We only add non-MISSING children
+ builder.addField(childType);
+ numberOfAddedFields++;
+ }
+ }
+ return numberOfAddedFields;
+ }
+
+ private Type handleHandleAvroArray(AbstractCollectionType collectionType, GroupType groupType) {
+ Types.GroupBuilder<GroupType> builder =
+ Types.buildGroup(groupType.getRepetition()).as(groupType.getLogicalTypeAnnotation());
+ //There is only one child
+ Type type = groupType.getType(0);
+ Type childType = collectionType.getItemType().accept(this, type);
+ builder.addField(childType);
+ return builder.named(groupType.getName());
+ }
+
+ @Override
+ public Type visit(AUnionType unionType, Type arg) {
+ if (arg.getLogicalTypeAnnotation() == LogicalTypeAnnotation.listType()) {
+ //Currently, only array type is supported
+ return unionType.getType(ATypeTag.ARRAY).accept(this, arg);
+ } else {
+ return unionType.getType(ATypeTag.OBJECT).accept(this, arg);
+ }
+ }
+
+ @Override
+ public Type visitFlat(IAType node, Type arg) {
+ return arg;
+ }
+
+ private boolean isNotCompatibleType(Type type, IAType node) {
+ if (type == MISSING) {
+ return true;
+ }
+ ATypeTag actualType = mapType(type);
+ ATypeTag expectedType = node.getTypeTag();
+
+ boolean isNotExpected = actualType != expectedType;
+ if (isNotExpected) {
+ //typeName is unique
+ FunctionCallInformation info = funcInfo.get(node.getTypeName());
+ //If no warning is created, then it means it has been reported
+ Warning warning = info.createTypeMismatchWarning(expectedType, actualType);
+ if (warning != null) {
+ //New warning that we saw for the first time. We should report it.
+ warnings.add(warning);
+ }
+ }
+ return isNotExpected;
+ }
+
+ private static ATypeTag mapType(Type parquetType) {
+ LogicalTypeAnnotation typeAnnotation = parquetType.getLogicalTypeAnnotation();
+ if (!parquetType.isPrimitive() && typeAnnotation == null) {
+ return ATypeTag.OBJECT;
+ } else if (typeAnnotation == LogicalTypeAnnotation.listType()) {
+ return ATypeTag.ARRAY;
+ } else if (typeAnnotation == LogicalTypeAnnotation.stringType()) {
+ return ATypeTag.STRING;
+ } else {
+ //Check other primitive types
+ PrimitiveType.PrimitiveTypeName primitiveTypeName = parquetType.asPrimitiveType().getPrimitiveTypeName();
+ switch (primitiveTypeName) {
+ case BOOLEAN:
+ return ATypeTag.BOOLEAN;
+ case INT32:
+ case INT64:
+ return ATypeTag.BIGINT;
+ case FLOAT:
+ case DOUBLE:
+ return ATypeTag.DOUBLE;
+ default:
+ throw new IllegalStateException("Unsupported type " + parquetType);
+ }
+ }
+ }
+
+ private static Type getType(GroupType groupType, String fieldName) {
+ if (groupType.containsField(fieldName)) {
+ return groupType.getType(fieldName);
+ }
+ return MISSING;
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MissingConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MissingConverter.java
new file mode 100644
index 0000000..e38056b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MissingConverter.java
@@ -0,0 +1,60 @@
+/*
+ * 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.external.input.record.reader.hdfs.parquet;
+
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.PrimitiveConverter;
+
+class MissingConverter extends PrimitiveConverter {
+ protected static final MissingConverter INSTANCE = new MissingConverter();
+
+ private MissingConverter() {
+ }
+
+ @Override
+ public void addBinary(Binary value) {
+ //NoOp
+ }
+
+ @Override
+ public void addBoolean(boolean value) {
+ //NoOp
+ }
+
+ @Override
+ public void addFloat(float value) {
+ //NoOp
+ }
+
+ @Override
+ public void addDouble(double value) {
+ //NoOp
+ }
+
+ @Override
+ public void addInt(int value) {
+ //NoOp
+ }
+
+ @Override
+ public void addLong(long value) {
+ //NoOp
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
index e208240..cc9b34c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
@@ -22,10 +22,12 @@
import org.apache.asterix.external.input.record.ValueReferenceRecord;
import org.apache.asterix.external.input.record.reader.hdfs.AbstractHDFSRecordReader;
+import org.apache.asterix.external.util.HDFSUtils;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.data.std.api.IValueReference;
/**
@@ -33,10 +35,12 @@
* The reader returns records in ADM format.
*/
public class ParquetFileRecordReader<V extends IValueReference> extends AbstractHDFSRecordReader<Void, V> {
+ private final IWarningCollector warningCollector;
public ParquetFileRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
- JobConf conf) {
+ JobConf conf, IWarningCollector warningCollector) {
super(read, inputSplits, readSchedule, nodeName, new ValueReferenceRecord<>(), conf);
+ this.warningCollector = warningCollector;
}
@Override
@@ -44,6 +48,13 @@
return false;
}
+ @Override
+ public void close() throws IOException {
+ super.close();
+ //Issue warning if any was reported
+ HDFSUtils.issueWarnings(warningCollector, conf);
+ }
+
@SuppressWarnings("unchecked")
@Override
protected RecordReader<Void, V> getRecordReader(int splitIndex) throws IOException {
@@ -53,5 +64,4 @@
}
return reader;
}
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetReadSupport.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetReadSupport.java
index 814da3c..aac293d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetReadSupport.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetReadSupport.java
@@ -18,33 +18,27 @@
*/
package org.apache.asterix.external.input.record.reader.hdfs.parquet;
+import java.io.IOException;
import java.util.Collections;
+import java.util.List;
import java.util.Map;
-import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.HDFSUtils;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.parquet.hadoop.api.InitContext;
import org.apache.parquet.hadoop.api.ReadSupport;
import org.apache.parquet.io.api.GroupConverter;
import org.apache.parquet.io.api.RecordMaterializer;
-import org.apache.parquet.schema.GroupType;
import org.apache.parquet.schema.MessageType;
-import org.apache.parquet.schema.PrimitiveType;
-import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
-import org.apache.parquet.schema.Type;
-import org.apache.parquet.schema.Type.Repetition;
-import org.apache.parquet.schema.Types;
-import org.apache.parquet.schema.Types.GroupBuilder;
-import org.apache.parquet.schema.Types.MessageTypeBuilder;
public class ParquetReadSupport extends ReadSupport<IValueReference> {
- private static final PrimitiveType NULL = Types.optional(PrimitiveTypeName.BOOLEAN).named("NULL");
-
@Override
public ReadContext init(InitContext context) {
- final String requestedSchemaString = context.getConfiguration().get(ExternalDataConstants.KEY_REQUESTED_FIELDS);
- final MessageType requestedSchema = getRequestedSchema(requestedSchemaString, context.getFileSchema());
+ MessageType requestedSchema = getRequestedSchema(context);
return new ReadContext(requestedSchema, Collections.emptyMap());
}
@@ -54,6 +48,29 @@
return new ADMRecordMaterializer(readContext);
}
+ private static MessageType getRequestedSchema(InitContext context) {
+ Configuration configuration = context.getConfiguration();
+ MessageType fileSchema = context.getFileSchema();
+ AsterixTypeToParquetTypeVisitor visitor = new AsterixTypeToParquetTypeVisitor();
+ try {
+ ARecordType expectedType = HDFSUtils.getExpectedType(configuration);
+ Map<String, FunctionCallInformation> functionCallInformationMap =
+ HDFSUtils.getFunctionCallInformationMap(configuration);
+ MessageType requestedType = visitor.clipType(expectedType, fileSchema, functionCallInformationMap);
+ List<Warning> warnings = visitor.getWarnings();
+
+ if (!warnings.isEmpty()) {
+ //New warnings were created, set the warnings in hadoop configuration to be reported
+ HDFSUtils.setWarnings(warnings, configuration);
+ //Update the reported warnings so that we do not report the same warning again
+ HDFSUtils.setFunctionCallInformationMap(functionCallInformationMap, configuration);
+ }
+ return requestedType;
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
private static class ADMRecordMaterializer extends RecordMaterializer<IValueReference> {
private final RootConverter rootConverter;
@@ -70,43 +87,5 @@
public GroupConverter getRootConverter() {
return rootConverter;
}
-
}
-
- private static MessageType getRequestedSchema(String requestedSchemaString, MessageType fileSchema) {
- if ("*".equals(requestedSchemaString)) {
- return fileSchema;
- }
-
- final MessageTypeBuilder builder = Types.buildMessage();
- final String[] paths = requestedSchemaString.split(",");
- for (int i = 0; i < paths.length; i++) {
- buildRequestedType(paths[i].trim().split("[.]"), builder, fileSchema, 0);
- }
-
- return builder.named("asterix");
-
- }
-
- private static void buildRequestedType(String[] fieldNames, GroupBuilder<?> builder, GroupType groupType,
- int start) {
- final String fieldName = fieldNames[start].trim();
-
- Type type = getType(groupType, fieldName);
- if (type != NULL && start < fieldNames.length - 1) {
- final GroupBuilder<GroupType> innerFieldBuilder = Types.buildGroup(Repetition.OPTIONAL);
- buildRequestedType(fieldNames, innerFieldBuilder, type.asGroupType(), start + 1);
- builder.addField(innerFieldBuilder.named(fieldName));
- } else {
- builder.addField(type);
- }
- }
-
- private static Type getType(GroupType groupType, String fieldName) {
- if (groupType.containsField(fieldName)) {
- return groupType.getType(fieldName);
- }
- return NULL;
- }
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
index 4b86142..db20d31 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.external.input.record.reader.stream;
+import static org.apache.asterix.external.util.ExternalDataConstants.BYTE_ORDER_MARK;
+
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
@@ -121,6 +123,10 @@
}
}
for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
+ if (inputBuffer[bufferPosn] == BYTE_ORDER_MARK) {
+ startPosn++;
+ continue;
+ }
if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
newlineLength = (prevCharCR) ? 2 : 1;
++bufferPosn; // at next invocation proceed from following byte
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
index 4c253bc..4433b49 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.stream;
+import static org.apache.asterix.external.util.ExternalDataConstants.BYTE_ORDER_MARK;
import static org.apache.asterix.external.util.ExternalDataConstants.REC_ENDED_AT_EOF;
import java.io.IOException;
@@ -119,6 +120,10 @@
boolean maybeInQuote = false;
for (; bufferPosn < bufferLength; ++bufferPosn) {
char ch = inputBuffer[bufferPosn];
+ if (ch == BYTE_ORDER_MARK) {
+ startPosn++;
+ continue;
+ }
// count lines here since we need to also count the lines inside quotes
if (ch == ExternalDataConstants.LF || prevCharCR) {
lineNumber++;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
index 0e23e46..2c31a0a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.external.input.record.reader.stream;
+import static org.apache.asterix.external.util.ExternalDataConstants.BYTE_ORDER_MARK;
import static org.apache.asterix.external.util.ExternalDataConstants.CLOSING_BRACKET;
import static org.apache.asterix.external.util.ExternalDataConstants.COMMA;
import static org.apache.asterix.external.util.ExternalDataConstants.CR;
@@ -134,7 +135,7 @@
lineNumber++;
}
isLastCharCR = c == CR;
- if (c == SPACE || c == TAB || c == LF || c == CR) {
+ if (c == SPACE || c == TAB || c == LF || c == CR || c == BYTE_ORDER_MARK) {
continue;
}
if (c == recordStart && state != State.NESTED_OBJECT) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/builders/ListLikeNumericArrayFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/builders/ListLikeNumericArrayFactory.java
new file mode 100644
index 0000000..e53f157
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/builders/ListLikeNumericArrayFactory.java
@@ -0,0 +1,36 @@
+/*
+ * 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.external.input.stream.builders;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.om.util.container.IObjectFactory;
+
+public class ListLikeNumericArrayFactory<T extends Number> implements IObjectFactory<List<T>, T> {
+
+ @Override
+ public List<T> create(T arg) {
+ List<T> list = new ArrayList<>(arg.intValue());
+ list.addAll(Collections.nCopies(arg.intValue(), arg));
+ return list;
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/builders/StandardToModifiedUTF8DataOutputFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/builders/StandardToModifiedUTF8DataOutputFactory.java
new file mode 100644
index 0000000..606aee2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/builders/StandardToModifiedUTF8DataOutputFactory.java
@@ -0,0 +1,37 @@
+/*
+ * 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.external.input.stream.builders;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.external.input.stream.StandardUTF8ToModifiedUTF8DataOutput;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+
+public class StandardToModifiedUTF8DataOutputFactory
+ implements IObjectFactory<StandardUTF8ToModifiedUTF8DataOutput, ATypeTag> {
+
+ @Override
+ public StandardUTF8ToModifiedUTF8DataOutput create(ATypeTag type) {
+ return new StandardUTF8ToModifiedUTF8DataOutput(
+ new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader()));
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
index cd7ec18..c803517 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
@@ -18,14 +18,28 @@
import static org.apache.hyracks.ipc.impl.Message.HEADER_SIZE;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
+import org.apache.asterix.om.pointables.AFlatValuePointable;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.ipc.impl.Message;
import org.msgpack.core.MessagePack;
import org.msgpack.core.MessageUnpacker;
@@ -33,26 +47,32 @@
public class PythonIPCProto {
- private PythonMessageBuilder messageBuilder;
- private OutputStream sockOut;
- private ByteBuffer headerBuffer = ByteBuffer.allocate(21);
+ private final PythonMessageBuilder messageBuilder;
+ private final DataOutputStream sockOut;
+ private final ByteBuffer headerBuffer = ByteBuffer.allocate(21);
private ByteBuffer recvBuffer = ByteBuffer.allocate(32768);
- private ExternalFunctionResultRouter router;
+ private final ExternalFunctionResultRouter router;
private long routeId;
private Pair<ByteBuffer, Exception> bufferBox;
- private Process pythonProc;
+ private final Process pythonProc;
private long maxFunctionId;
- private ArrayBufferInput unpackerInput;
- private MessageUnpacker unpacker;
+ private final ArrayBufferInput unpackerInput;
+ private final MessageUnpacker unpacker;
+ private final ArrayBackedValueStorage argsStorage;
+ private final PointableAllocator pointableAllocator;
+ private final MsgPackPointableVisitor pointableVisitor;
public PythonIPCProto(OutputStream sockOut, ExternalFunctionResultRouter router, Process pythonProc) {
- this.sockOut = sockOut;
+ this.sockOut = new DataOutputStream(sockOut);
messageBuilder = new PythonMessageBuilder();
this.router = router;
this.pythonProc = pythonProc;
- this.maxFunctionId = 0l;
+ this.maxFunctionId = 0L;
unpackerInput = new ArrayBufferInput(new byte[0]);
unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
+ this.argsStorage = new ArrayBackedValueStorage();
+ this.pointableAllocator = new PointableAllocator();
+ this.pointableVisitor = new MsgPackPointableVisitor();
}
public void start() {
@@ -65,10 +85,10 @@
recvBuffer.clear();
recvBuffer.position(0);
recvBuffer.limit(0);
- messageBuilder.buf.clear();
- messageBuilder.buf.position(0);
+ messageBuilder.reset();
messageBuilder.hello();
- sendMsg(routeId);
+ sendHeader(routeId, messageBuilder.getLength());
+ sendMsg();
receiveMsg();
if (getResponseType() != MessageType.HELO) {
throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
@@ -81,10 +101,10 @@
recvBuffer.clear();
recvBuffer.position(0);
recvBuffer.limit(0);
- messageBuilder.buf.clear();
- messageBuilder.buf.position(0);
+ messageBuilder.reset();
messageBuilder.init(module, clazz, fn);
- sendMsg(functionId);
+ sendHeader(functionId, messageBuilder.getLength());
+ sendMsg();
receiveMsg();
if (getResponseType() != MessageType.INIT_RSP) {
throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
@@ -93,14 +113,21 @@
return functionId;
}
- public ByteBuffer call(long functionId, ByteBuffer args, int numArgs) throws IOException, AsterixException {
+ public ByteBuffer call(long functionId, IAType[] argTypes, IValueReference[] argValues, boolean nullCall)
+ throws IOException, AsterixException {
recvBuffer.clear();
recvBuffer.position(0);
recvBuffer.limit(0);
- messageBuilder.buf.clear();
- messageBuilder.buf.position(0);
- messageBuilder.call(args.array(), args.position(), numArgs);
- sendMsg(functionId);
+ messageBuilder.reset();
+ argsStorage.reset();
+ for (int i = 0; i < argTypes.length; i++) {
+ visitValueRef(argTypes[i], argsStorage.getDataOutput(), argValues[i], pointableAllocator, pointableVisitor,
+ nullCall);
+ }
+ int len = argsStorage.getLength() + 5;
+ sendHeader(functionId, len);
+ messageBuilder.call(argValues.length, len);
+ sendMsg(argsStorage);
receiveMsg();
if (getResponseType() != MessageType.CALL_RSP) {
throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
@@ -109,14 +136,16 @@
return recvBuffer;
}
- public ByteBuffer callMulti(long key, ByteBuffer args, int numTuples) throws IOException, AsterixException {
+ public ByteBuffer callMulti(long key, ArrayBackedValueStorage args, int numTuples)
+ throws IOException, AsterixException {
recvBuffer.clear();
recvBuffer.position(0);
recvBuffer.limit(0);
- messageBuilder.buf.clear();
- messageBuilder.buf.position(0);
- messageBuilder.callMulti(args.array(), args.position(), numTuples);
- sendMsg(key);
+ messageBuilder.reset();
+ int len = args.getLength() + 4;
+ sendHeader(key, len);
+ messageBuilder.callMulti(0, numTuples);
+ sendMsg(args);
receiveMsg();
if (getResponseType() != MessageType.CALL_RSP) {
throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
@@ -132,7 +161,7 @@
}
public void receiveMsg() throws IOException, AsterixException {
- Exception except = null;
+ Exception except;
try {
synchronized (bufferBox) {
while ((bufferBox.getFirst().limit() == 0 || bufferBox.getSecond() != null) && pythonProc.isAlive()) {
@@ -162,15 +191,27 @@
}
}
- public void sendMsg(long key) throws IOException {
+ public void sendHeader(long key, int msgLen) throws IOException {
headerBuffer.clear();
headerBuffer.position(0);
- headerBuffer.putInt(HEADER_SIZE + Integer.BYTES + messageBuilder.buf.position());
+ headerBuffer.putInt(HEADER_SIZE + Integer.BYTES + msgLen);
headerBuffer.putLong(key);
headerBuffer.putLong(routeId);
headerBuffer.put(Message.NORMAL);
sockOut.write(headerBuffer.array(), 0, HEADER_SIZE + Integer.BYTES);
- sockOut.write(messageBuilder.buf.array(), 0, messageBuilder.buf.position());
+ sockOut.flush();
+ }
+
+ public void sendMsg(ArrayBackedValueStorage content) throws IOException {
+ sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
+ messageBuilder.getBuf().position());
+ sockOut.write(content.getByteArray(), content.getStartOffset(), content.getLength());
+ sockOut.flush();
+ }
+
+ public void sendMsg() throws IOException {
+ sockOut.write(messageBuilder.getBuf().array(), messageBuilder.getBuf().arrayOffset(),
+ messageBuilder.getBuf().position());
sockOut.flush();
}
@@ -182,4 +223,68 @@
return routeId;
}
+ public DataOutputStream getSockOut() {
+ return sockOut;
+ }
+
+ public static void visitValueRef(IAType type, DataOutput out, IValueReference valueReference,
+ PointableAllocator pointableAllocator, MsgPackPointableVisitor pointableVisitor, boolean visitNull)
+ throws IOException {
+ IVisitablePointable pointable;
+ switch (type.getTypeTag()) {
+ case OBJECT:
+ pointable = pointableAllocator.allocateRecordValue(type);
+ pointable.set(valueReference);
+ pointableVisitor.visit((ARecordVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+ break;
+ case ARRAY:
+ case MULTISET:
+ pointable = pointableAllocator.allocateListValue(type);
+ pointable.set(valueReference);
+ pointableVisitor.visit((AListVisitablePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+ break;
+ case ANY:
+ ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+ .deserialize(valueReference.getByteArray()[valueReference.getStartOffset()]);
+ IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
+ switch (rtTypeTag) {
+ case OBJECT:
+ pointable = pointableAllocator.allocateRecordValue(rtType);
+ pointable.set(valueReference);
+ pointableVisitor.visit((ARecordVisitablePointable) pointable,
+ pointableVisitor.getTypeInfo(rtType, out));
+ break;
+ case ARRAY:
+ case MULTISET:
+ pointable = pointableAllocator.allocateListValue(rtType);
+ pointable.set(valueReference);
+ pointableVisitor.visit((AListVisitablePointable) pointable,
+ pointableVisitor.getTypeInfo(rtType, out));
+ break;
+ case MISSING:
+ case NULL:
+ if (!visitNull) {
+ return;
+ }
+ default:
+ pointable = pointableAllocator.allocateFieldValue(rtType);
+ pointable.set(valueReference);
+ pointableVisitor.visit((AFlatValuePointable) pointable,
+ pointableVisitor.getTypeInfo(rtType, out));
+ break;
+ }
+ break;
+ case MISSING:
+ case NULL:
+ if (!visitNull) {
+ return;
+ }
+ default:
+ pointable = pointableAllocator.allocateFieldValue(type);
+ pointable.set(valueReference);
+ pointableVisitor.visit((AFlatValuePointable) pointable, pointableVisitor.getTypeInfo(type, out));
+ break;
+ }
+ }
+
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
index 5052eb4..5429657 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
@@ -26,15 +26,13 @@
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
-import org.apache.asterix.external.library.msgpack.MessagePackerFromADM;
-import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.asterix.external.library.msgpack.MessagePackUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class PythonMessageBuilder {
- private static final int MAX_BUF_SIZE = 64 * 1024 * 1024; //64MB.
MessageType type;
long dataLength;
- ByteBuffer buf;
+ private final ByteBuffer buf;
public PythonMessageBuilder() {
this.type = null;
@@ -42,12 +40,25 @@
this.buf = ByteBuffer.allocate(4096);
}
+ public void reset() {
+ //TODO: should be able to get away w/o clearing buf?
+ buf.clear();
+ }
+
+ public ByteBuffer getBuf() {
+ return buf;
+ }
+
+ public int getLength() {
+ return buf.position() - buf.arrayOffset();
+ }
+
public void setType(MessageType type) {
this.type = type;
}
public void packHeader() throws HyracksDataException {
- MessagePackerFromADM.packFixPos(buf, (byte) type.ordinal());
+ MessagePackUtils.packFixPos(buf, (byte) type.ordinal());
}
//TODO: this is wrong for any multibyte chars
@@ -75,7 +86,7 @@
this.type = MessageType.QUIT;
dataLength = getStringLength("QUIT");
packHeader();
- MessagePackerFromADM.packFixStr(buf, "QUIT");
+ MessagePackUtils.packFixStr(buf, "QUIT");
}
public void init(final String module, final String clazz, final String fn) throws HyracksDataException {
@@ -89,46 +100,27 @@
}
packHeader();
int numArgs = clazz == null ? 2 : 3;
- MessagePackerFromADM.packFixArrayHeader(buf, (byte) numArgs);
- MessagePackerFromADM.packStr(buf, module);
+ MessagePackUtils.packFixArrayHeader(buf, (byte) numArgs);
+ MessagePackUtils.packStr(buf, module);
if (clazz != null) {
- MessagePackerFromADM.packStr(buf, clazz);
+ MessagePackUtils.packStr(buf, clazz);
}
- MessagePackerFromADM.packStr(buf, fn);
+ MessagePackUtils.packStr(buf, fn);
}
- public void call(byte[] args, int lim, int numArgs) throws HyracksDataException {
- if (args.length > buf.capacity()) {
- int growTo = ExternalFunctionResultRouter.closestPow2(args.length);
- if (growTo > MAX_BUF_SIZE) {
- throw HyracksDataException.create(ErrorCode.ILLEGAL_STATE,
- "Unable to allocate message buffer larger than:" + MAX_BUF_SIZE + " bytes");
- }
- buf = ByteBuffer.allocate(growTo);
- }
+ public void call(int numArgs, int len) throws HyracksDataException {
buf.clear();
buf.position(0);
this.type = MessageType.CALL;
- dataLength = 5 + 1 + lim;
+ dataLength = 5 + 1 + len;
packHeader();
//TODO: make this switch between fixarray/array16/array32
buf.put((byte) (FIXARRAY_PREFIX + 1));
- buf.put(ARRAY32);
- buf.putInt(numArgs);
- if (numArgs > 0) {
- buf.put(args, 0, lim);
- }
+ buf.put(ARRAY16);
+ buf.putShort((short) numArgs);
}
- public void callMulti(byte[] args, int lim, int numArgs) throws HyracksDataException {
- if (args.length > buf.capacity()) {
- int growTo = ExternalFunctionResultRouter.closestPow2(args.length);
- if (growTo > MAX_BUF_SIZE) {
- throw HyracksDataException.create(ErrorCode.ILLEGAL_STATE,
- "Unable to allocate message buffer larger than:" + MAX_BUF_SIZE + " bytes");
- }
- buf = ByteBuffer.allocate(growTo);
- }
+ public void callMulti(int lim, int numArgs) throws HyracksDataException {
buf.clear();
buf.position(0);
this.type = MessageType.CALL;
@@ -137,9 +129,6 @@
//TODO: make this switch between fixarray/array16/array32
buf.put(ARRAY16);
buf.putShort((short) numArgs);
- if (numArgs > 0) {
- buf.put(args, 0, lim);
- }
}
//this is used to send a serialized java inetaddress to the entrypoint so it can send it back
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
index e54b729..f71150a 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
@@ -639,7 +639,9 @@
final INetworkSecurityConfig configuration = networkSecurityManager.getConfiguration();
KeyStore trustStore = KeyStore.getInstance(KeyStore.getDefaultType());
try (FileInputStream trustStoreFile = new FileInputStream(configuration.getTrustStoreFile())) {
- trustStore.load(trustStoreFile, configuration.getKeyStorePassword().toCharArray());
+ String ksPassword = configuration.getKeyStorePassword();
+ trustStore.load(trustStoreFile,
+ ksPassword == null || ksPassword.isEmpty() ? null : ksPassword.toCharArray());
}
SSLContext sslcontext = SSLContexts.custom().loadTrustMaterial(trustStore, null).build();
SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslcontext,
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
index 7c860a2..94a4dd2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
@@ -61,6 +61,7 @@
private MessageUnpacker unpacker;
private ArrayBufferInput unpackerInput;
+ private MessageUnpackerToADM unpackerToADM;
private long fnId;
@@ -87,6 +88,7 @@
this.sourceLocation = sourceLoc;
this.unpackerInput = new ArrayBufferInput(new byte[0]);
this.unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
+ this.unpackerToADM = new MessageUnpackerToADM();
}
@Override
@@ -107,18 +109,13 @@
hasNullArg = true;
}
}
- try {
- PythonLibraryEvaluator.setArgument(argTypes[i], argValues[i], argHolder, nullCall);
- } catch (IOException e) {
- throw new HyracksDataException("Error evaluating Python UDF", e);
- }
}
if (!nullCall && hasNullArg) {
PointableHelper.setNull(result);
return;
}
try {
- ByteBuffer res = libraryEvaluator.callPython(fnId, argHolder, argTypes.length);
+ ByteBuffer res = libraryEvaluator.callPython(fnId, argTypes, argValues, nullCall);
resultBuffer.reset();
wrap(res, resultBuffer.getDataOutput());
} catch (Exception e) {
@@ -133,30 +130,28 @@
outputWrapper.position(0);
try {
if (resultWrapper == null) {
- outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
- out.write(outputWrapper.array(), 0, outputWrapper.position() + outputWrapper.arrayOffset());
+ out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
return;
}
if ((resultWrapper.get() ^ FIXARRAY_PREFIX) != (byte) 2) {
- throw HyracksDataException.create(AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
- "Returned result missing outer wrapper"));
+ throw HyracksDataException
+ .create(AsterixException.create(ErrorCode.EXTERNAL_UDF_PROTO_RETURN_EXCEPTION));
}
int numresults = resultWrapper.get() ^ FIXARRAY_PREFIX;
if (numresults > 0) {
- MessageUnpackerToADM.unpack(resultWrapper, outputWrapper, true);
+ unpackerToADM.unpack(resultWrapper, out, true);
}
unpackerInput.reset(resultWrapper.array(), resultWrapper.position() + resultWrapper.arrayOffset(),
resultWrapper.remaining());
unpacker.reset(unpackerInput);
- int numEntries = unpacker.unpackArrayHeader();
- for (int j = 0; j < numEntries; j++) {
- outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ int numErrors = unpacker.unpackArrayHeader();
+ for (int j = 0; j < numErrors; j++) {
+ out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
if (evaluatorContext.getWarningCollector().shouldWarn()) {
evaluatorContext.getWarningCollector().warn(
Warning.of(sourceLocation, ErrorCode.EXTERNAL_UDF_EXCEPTION, unpacker.unpackString()));
}
}
- out.write(outputWrapper.array(), 0, outputWrapper.position() + outputWrapper.arrayOffset());
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
index d1b2685..98755fc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaFunctionHelper.java
@@ -56,7 +56,7 @@
private final IObjectPool<IJObject, IAType> objectPool = new ListObjectPool<>(JTypeObjectFactory.INSTANCE);
private final JObjectPointableVisitor pointableVisitor;
private final PointableAllocator pointableAllocator;
- private final Map<Integer, TypeInfo> poolTypeInfo;
+ private final Map<Integer, JavaTypeInfo> poolTypeInfo;
private final Map<String, String> parameters;
private final IAType[] argTypes;
@@ -164,10 +164,10 @@
arguments[index] = jObject;
}
- private TypeInfo getTypeInfo(int index, IAType type) {
- TypeInfo typeInfo = poolTypeInfo.get(index);
+ private JavaTypeInfo getTypeInfo(int index, IAType type) {
+ JavaTypeInfo typeInfo = poolTypeInfo.get(index);
if (typeInfo == null) {
- typeInfo = new TypeInfo(objectPool, type, type.getTypeTag());
+ typeInfo = new JavaTypeInfo(objectPool, type, type.getTypeTag());
poolTypeInfo.put(index, typeInfo);
}
return typeInfo;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaTypeInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaTypeInfo.java
new file mode 100644
index 0000000..e60cc5d
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/JavaTypeInfo.java
@@ -0,0 +1,67 @@
+/*
+ * 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.external.library;
+
+import org.apache.asterix.external.api.IJObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.util.container.IObjectPool;
+
+public class JavaTypeInfo {
+
+ private IObjectPool<IJObject, IAType> objectPool;
+ private IAType atype;
+ private ATypeTag typeTag;
+
+ public JavaTypeInfo(IObjectPool<IJObject, IAType> objectPool, IAType atype, ATypeTag typeTag) {
+ this.objectPool = objectPool;
+ this.atype = atype;
+ this.typeTag = typeTag;
+ }
+
+ public void reset(IAType atype, ATypeTag typeTag) {
+ this.atype = atype;
+ this.typeTag = typeTag;
+ }
+
+ public IObjectPool<IJObject, IAType> getObjectPool() {
+ return objectPool;
+ }
+
+ public void setObjectPool(IObjectPool<IJObject, IAType> objectPool) {
+ this.objectPool = objectPool;
+ }
+
+ public IAType getAtype() {
+ return atype;
+ }
+
+ public void setAtype(IAType atype) {
+ this.atype = atype;
+ }
+
+ public ATypeTag getTypeTag() {
+ return typeTag;
+ }
+
+ public void setTypeTag(ATypeTag typeTag) {
+ this.typeTag = typeTag;
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PyTypeInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PyTypeInfo.java
new file mode 100644
index 0000000..e2b7ad4
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PyTypeInfo.java
@@ -0,0 +1,54 @@
+/*
+ * 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.external.library;
+
+import java.io.DataOutput;
+import java.util.Objects;
+
+import org.apache.asterix.om.types.IAType;
+
+public class PyTypeInfo {
+
+ private final IAType type;
+ private final DataOutput out;
+
+ public PyTypeInfo(IAType type, DataOutput out) {
+ this.type = type;
+ this.out = out;
+ }
+
+ public DataOutput getDataOutput() {
+ return out;
+ }
+
+ public IAType getType() {
+ return type;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(out, type);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ return out.equals(out) && type.equals(type);
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
index 457b86a..f82b30d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
@@ -34,7 +34,7 @@
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
import org.apache.asterix.external.ipc.PythonIPCProto;
-import org.apache.asterix.external.library.msgpack.MessagePackerFromADM;
+import org.apache.asterix.external.library.msgpack.MessagePackUtils;
import org.apache.asterix.om.functions.IExternalFunctionInfo;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -50,6 +50,7 @@
import org.apache.hyracks.api.resources.IDeallocatable;
import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
import org.apache.hyracks.ipc.impl.IPCSystem;
@@ -86,7 +87,6 @@
this.ipcSys = ipcSys;
this.warningCollector = warningCollector;
this.sourceLoc = sourceLoc;
-
}
private void initialize() throws IOException, AsterixException {
@@ -128,10 +128,11 @@
return proto.init(packageModule, clazz, fn);
}
- public ByteBuffer callPython(long id, ByteBuffer arguments, int numArgs) throws IOException {
+ public ByteBuffer callPython(long id, IAType[] argTypes, IValueReference[] valueReferences, boolean nullCall)
+ throws IOException {
ByteBuffer ret = null;
try {
- ret = proto.call(id, arguments, numArgs);
+ ret = proto.call(id, argTypes, valueReferences, nullCall);
} catch (AsterixException e) {
if (warningCollector.shouldWarn()) {
warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
@@ -140,7 +141,7 @@
return ret;
}
- public ByteBuffer callPythonMulti(long id, ByteBuffer arguments, int numTuples) throws IOException {
+ public ByteBuffer callPythonMulti(long id, ArrayBackedValueStorage arguments, int numTuples) throws IOException {
ByteBuffer ret = null;
try {
ret = proto.callMulti(id, arguments, numTuples);
@@ -169,20 +170,6 @@
router.removeRoute(proto.getRouteId());
}
- public static ATypeTag setArgument(IAType type, IValueReference valueReference, ByteBuffer argHolder,
- boolean nullCall) throws IOException {
- ATypeTag tag = type.getTypeTag();
- if (tag == ATypeTag.ANY) {
- TaggedValuePointable pointy = TaggedValuePointable.FACTORY.createPointable();
- pointy.set(valueReference);
- ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
- IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
- return MessagePackerFromADM.pack(valueReference, rtType, argHolder, nullCall);
- } else {
- return MessagePackerFromADM.pack(valueReference, type, argHolder, nullCall);
- }
- }
-
public static ATypeTag peekArgument(IAType type, IValueReference valueReference) throws HyracksDataException {
ATypeTag tag = type.getTypeTag();
if (tag == ATypeTag.ANY) {
@@ -190,15 +177,15 @@
pointy.set(valueReference);
ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
- return MessagePackerFromADM.peekUnknown(rtType);
+ return MessagePackUtils.peekUnknown(rtType);
} else {
- return MessagePackerFromADM.peekUnknown(type);
+ return MessagePackUtils.peekUnknown(type);
}
}
- public static void setVoidArgument(ByteBuffer argHolder) {
- argHolder.put(ARRAY16);
- argHolder.putShort((short) 0);
+ public static void setVoidArgument(ArrayBackedValueStorage argHolder) throws IOException {
+ argHolder.getDataOutput().writeByte(ARRAY16);
+ argHolder.getDataOutput().writeShort((short) 0);
}
public static PythonLibraryEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
@@ -218,4 +205,5 @@
}
return evaluator;
}
+
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/TypeInfo.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/TypeInfo.java
deleted file mode 100644
index 453cf39..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/TypeInfo.java
+++ /dev/null
@@ -1,67 +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.
- */
-package org.apache.asterix.external.library;
-
-import org.apache.asterix.external.api.IJObject;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.container.IObjectPool;
-
-public class TypeInfo {
-
- private IObjectPool<IJObject, IAType> objectPool;
- private IAType atype;
- private ATypeTag typeTag;
-
- public TypeInfo(IObjectPool<IJObject, IAType> objectPool, IAType atype, ATypeTag typeTag) {
- this.objectPool = objectPool;
- this.atype = atype;
- this.typeTag = typeTag;
- }
-
- public void reset(IAType atype, ATypeTag typeTag) {
- this.atype = atype;
- this.typeTag = typeTag;
- }
-
- public IObjectPool<IJObject, IAType> getObjectPool() {
- return objectPool;
- }
-
- public void setObjectPool(IObjectPool<IJObject, IAType> objectPool) {
- this.objectPool = objectPool;
- }
-
- public IAType getAtype() {
- return atype;
- }
-
- public void setAtype(IAType atype) {
- this.atype = atype;
- }
-
- public ATypeTag getTypeTag() {
- return typeTag;
- }
-
- public void setTypeTag(ATypeTag typeTag) {
- this.typeTag = typeTag;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
index 230627f..5479b56a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectAccessors.java
@@ -42,7 +42,7 @@
import org.apache.asterix.external.api.IJObject;
import org.apache.asterix.external.api.IJObjectAccessor;
import org.apache.asterix.external.api.IJRecordAccessor;
-import org.apache.asterix.external.library.TypeInfo;
+import org.apache.asterix.external.library.JavaTypeInfo;
import org.apache.asterix.external.library.java.base.JBoolean;
import org.apache.asterix.external.library.java.base.JByte;
import org.apache.asterix.external.library.java.base.JDate;
@@ -356,14 +356,14 @@
public static class JRecordAccessor implements IJRecordAccessor {
- private final TypeInfo typeInfo;
+ private final JavaTypeInfo typeInfo;
private final JRecord jRecord;
private final IJObject[] jObjects;
private final LinkedHashMap<String, IJObject> openFields;
private final UTF8StringReader reader = new UTF8StringReader();
public JRecordAccessor(ARecordType recordType, IObjectPool<IJObject, IAType> objectPool) {
- this.typeInfo = new TypeInfo(objectPool, null, null);
+ this.typeInfo = new JavaTypeInfo(objectPool, null, null);
this.jObjects = new IJObject[recordType.getFieldNames().length];
this.openFields = new LinkedHashMap<>();
this.jRecord = new JRecord(recordType, jObjects, openFields);
@@ -439,10 +439,10 @@
public static class JListAccessor implements IJListAccessor {
- private final TypeInfo typeInfo;
+ private final JavaTypeInfo typeInfo;
public JListAccessor(IObjectPool<IJObject, IAType> objectPool) {
- this.typeInfo = new TypeInfo(objectPool, null, null);
+ this.typeInfo = new JavaTypeInfo(objectPool, null, null);
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java
index 9edc569..bd64922 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/JObjectPointableVisitor.java
@@ -25,7 +25,7 @@
import org.apache.asterix.external.api.IJObject;
import org.apache.asterix.external.api.IJObjectAccessor;
import org.apache.asterix.external.api.IJRecordAccessor;
-import org.apache.asterix.external.library.TypeInfo;
+import org.apache.asterix.external.library.JavaTypeInfo;
import org.apache.asterix.external.library.java.JObjectAccessors.JListAccessor;
import org.apache.asterix.external.library.java.JObjectAccessors.JRecordAccessor;
import org.apache.asterix.om.pointables.AFlatValuePointable;
@@ -37,7 +37,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-public class JObjectPointableVisitor implements IVisitablePointableVisitor<IJObject, TypeInfo> {
+public class JObjectPointableVisitor implements IVisitablePointableVisitor<IJObject, JavaTypeInfo> {
private final Map<ATypeTag, IJObjectAccessor> flatJObjectAccessors = new HashMap<ATypeTag, IJObjectAccessor>();
private final Map<IVisitablePointable, IJRecordAccessor> raccessorToJObject =
@@ -46,7 +46,7 @@
new HashMap<IVisitablePointable, IJListAccessor>();
@Override
- public IJObject visit(AListVisitablePointable accessor, TypeInfo arg) throws HyracksDataException {
+ public IJObject visit(AListVisitablePointable accessor, JavaTypeInfo arg) throws HyracksDataException {
IJObject result = null;
IJListAccessor jListAccessor = laccessorToPrinter.get(accessor);
if (jListAccessor == null) {
@@ -58,7 +58,7 @@
}
@Override
- public IJObject visit(ARecordVisitablePointable accessor, TypeInfo arg) throws HyracksDataException {
+ public IJObject visit(ARecordVisitablePointable accessor, JavaTypeInfo arg) throws HyracksDataException {
IJObject result = null;
IJRecordAccessor jRecordAccessor = raccessorToJObject.get(accessor);
if (jRecordAccessor == null) {
@@ -70,7 +70,7 @@
}
@Override
- public IJObject visit(AFlatValuePointable accessor, TypeInfo arg) throws HyracksDataException {
+ public IJObject visit(AFlatValuePointable accessor, JavaTypeInfo arg) throws HyracksDataException {
ATypeTag typeTag = arg.getTypeTag();
IJObject result = null;
IJObjectAccessor jObjectAccessor = flatJObjectAccessors.get(typeTag);
@@ -83,7 +83,8 @@
return result;
}
- public IJObject visit(AFlatValuePointable accessor, ATypeTag typeTag, TypeInfo arg) throws HyracksDataException {
+ public IJObject visit(AFlatValuePointable accessor, ATypeTag typeTag, JavaTypeInfo arg)
+ throws HyracksDataException {
IJObject result = null;
IJObjectAccessor jObjectAccessor = flatJObjectAccessors.get(typeTag);
if (jObjectAccessor == null) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/JInterval.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/JInterval.java
index 14caa1e..39a8909 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/JInterval.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/JInterval.java
@@ -27,7 +27,7 @@
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang3.ArrayUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public final class JInterval extends JObject<List<Long>> {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackAccessor.java
new file mode 100644
index 0000000..6b2f0f8
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackAccessor.java
@@ -0,0 +1,26 @@
+/*
+ * 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.external.library.msgpack;
+
+import java.io.IOException;
+
+@FunctionalInterface
+public interface IMsgPackAccessor<T, U, R> {
+ R apply(T t, U u) throws IOException;
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackListAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackListAccessor.java
new file mode 100644
index 0000000..a061a06
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackListAccessor.java
@@ -0,0 +1,28 @@
+/*
+ * 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.external.library.msgpack;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@FunctionalInterface
+public interface IMsgPackListAccessor<P, D, T, V, R> {
+
+ R access(P p, D d, T t, V v) throws HyracksDataException;
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackRecordAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackRecordAccessor.java
new file mode 100644
index 0000000..0ce417d
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/IMsgPackRecordAccessor.java
@@ -0,0 +1,28 @@
+/*
+ * 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.external.library.msgpack;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@FunctionalInterface
+public interface IMsgPackRecordAccessor<P, D, T, V, R> {
+
+ R access(P p, D d, T t, V v) throws HyracksDataException;
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackUtils.java
new file mode 100644
index 0000000..2377f9a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackUtils.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.library.msgpack;
+
+import static org.msgpack.core.MessagePack.Code.FIXARRAY_PREFIX;
+import static org.msgpack.core.MessagePack.Code.FIXSTR_PREFIX;
+import static org.msgpack.core.MessagePack.Code.STR32;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class MessagePackUtils {
+
+ public static ATypeTag peekUnknown(IAType type) {
+ switch (type.getTypeTag()) {
+ case MISSING:
+ case NULL:
+ return type.getTypeTag();
+ default:
+ return ATypeTag.TYPE;
+ }
+ }
+
+ public static void packFixPos(ByteBuffer buf, byte in) throws HyracksDataException {
+ byte mask = (byte) (1 << 7);
+ if ((in & mask) != 0) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+ "fixint7 must be positive");
+ }
+ buf.put(in);
+ }
+
+ public static void packFixStr(ByteBuffer buf, String in) throws HyracksDataException {
+ byte[] strBytes = in.getBytes(StandardCharsets.UTF_8);
+ if (strBytes.length > 31) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+ "fixint7 must be positive");
+ }
+ buf.put((byte) (FIXSTR_PREFIX + strBytes.length));
+ buf.put(strBytes);
+ }
+
+ public static void packStr(ByteBuffer out, String in) {
+ out.put(STR32);
+ byte[] strBytes = in.getBytes(StandardCharsets.UTF_8);
+ out.putInt(strBytes.length);
+ out.put(strBytes);
+ }
+
+ public static void packFixArrayHeader(ByteBuffer buf, byte numObj) {
+ buf.put((byte) (FIXARRAY_PREFIX + (0x0F & numObj)));
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackerFromADM.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackerFromADM.java
deleted file mode 100644
index f0ac56e..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackerFromADM.java
+++ /dev/null
@@ -1,362 +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.
- */
-package org.apache.asterix.external.library.msgpack;
-
-import static org.msgpack.core.MessagePack.Code.ARRAY32;
-import static org.msgpack.core.MessagePack.Code.FALSE;
-import static org.msgpack.core.MessagePack.Code.FIXARRAY_PREFIX;
-import static org.msgpack.core.MessagePack.Code.FIXSTR_PREFIX;
-import static org.msgpack.core.MessagePack.Code.FLOAT32;
-import static org.msgpack.core.MessagePack.Code.FLOAT64;
-import static org.msgpack.core.MessagePack.Code.INT16;
-import static org.msgpack.core.MessagePack.Code.INT32;
-import static org.msgpack.core.MessagePack.Code.INT64;
-import static org.msgpack.core.MessagePack.Code.INT8;
-import static org.msgpack.core.MessagePack.Code.MAP32;
-import static org.msgpack.core.MessagePack.Code.NIL;
-import static org.msgpack.core.MessagePack.Code.STR32;
-import static org.msgpack.core.MessagePack.Code.TRUE;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
-import org.apache.asterix.om.types.AbstractCollectionType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.types.TypeTagUtil;
-import org.apache.asterix.om.utils.NonTaggedFormatUtil;
-import org.apache.asterix.om.utils.RecordUtil;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.primitive.BooleanPointable;
-import org.apache.hyracks.data.std.primitive.BytePointable;
-import org.apache.hyracks.data.std.primitive.DoublePointable;
-import org.apache.hyracks.data.std.primitive.FloatPointable;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.data.std.primitive.LongPointable;
-import org.apache.hyracks.data.std.primitive.ShortPointable;
-import org.apache.hyracks.util.string.UTF8StringUtil;
-
-public class MessagePackerFromADM {
-
- private static final int TYPE_TAG_SIZE = 1;
- private static final int TYPE_SIZE = 1;
- private static final int LENGTH_SIZE = 4;
- private static final int ITEM_COUNT_SIZE = 4;
- private static final int ITEM_OFFSET_SIZE = 4;
-
- public static ATypeTag pack(IValueReference ptr, IAType type, ByteBuffer out, boolean packUnknown)
- throws HyracksDataException {
- return pack(ptr.getByteArray(), ptr.getStartOffset(), type, true, packUnknown, out);
- }
-
- public static ATypeTag pack(byte[] ptr, int offs, IAType type, boolean tagged, boolean packUnknown, ByteBuffer out)
- throws HyracksDataException {
- int relOffs = tagged ? offs + 1 : offs;
- ATypeTag tag = type.getTypeTag();
- switch (tag) {
- case STRING:
- packStr(ptr, relOffs, out);
- break;
- case BOOLEAN:
- if (BooleanPointable.getBoolean(ptr, relOffs)) {
- out.put(TRUE);
- } else {
- out.put(FALSE);
- }
- break;
- case TINYINT:
- packByte(out, BytePointable.getByte(ptr, relOffs));
- break;
- case SMALLINT:
- packShort(out, ShortPointable.getShort(ptr, relOffs));
- break;
- case INTEGER:
- packInt(out, IntegerPointable.getInteger(ptr, relOffs));
- break;
- case BIGINT:
- packLong(out, LongPointable.getLong(ptr, relOffs));
- break;
- case FLOAT:
- packFloat(out, FloatPointable.getFloat(ptr, relOffs));
- break;
- case DOUBLE:
- packDouble(out, DoublePointable.getDouble(ptr, relOffs));
- break;
- case ARRAY:
- case MULTISET:
- packArray(ptr, offs, type, out);
- break;
- case OBJECT:
- packObject(ptr, offs, type, out);
- break;
- case MISSING:
- case NULL:
- if (packUnknown) {
- packNull(out);
- break;
- } else {
- return tag;
- }
- default:
- throw HyracksDataException.create(AsterixException.create(ErrorCode.PARSER_ADM_DATA_PARSER_CAST_ERROR,
- tag.name(), "to a msgpack"));
- }
- return ATypeTag.TYPE;
- }
-
- public static ATypeTag peekUnknown(IAType type) {
- switch (type.getTypeTag()) {
- case MISSING:
- case NULL:
- return type.getTypeTag();
- default:
- return ATypeTag.TYPE;
- }
- }
-
- public static void packNull(ByteBuffer out) {
- out.put(NIL);
- }
-
- public static void packByte(ByteBuffer out, byte in) {
- out.put(INT8);
- out.put(in);
- }
-
- public static void packShort(ByteBuffer out, short in) {
- out.put(INT16);
- out.putShort(in);
- }
-
- public static void packInt(ByteBuffer out, int in) {
- out.put(INT32);
- out.putInt(in);
-
- }
-
- public static void packLong(ByteBuffer out, long in) {
- out.put(INT64);
- out.putLong(in);
- }
-
- public static void packFloat(ByteBuffer out, float in) {
- out.put(FLOAT32);
- out.putFloat(in);
- }
-
- public static void packDouble(ByteBuffer out, double in) {
- out.put(FLOAT64);
- out.putDouble(in);
- }
-
- public static void packFixPos(ByteBuffer out, byte in) throws HyracksDataException {
- byte mask = (byte) (1 << 7);
- if ((in & mask) != 0) {
- throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
- "fixint7 must be positive");
- }
- out.put(in);
- }
-
- public static void packFixStr(ByteBuffer buf, String in) throws HyracksDataException {
- byte[] strBytes = in.getBytes(StandardCharsets.UTF_8);
- if (strBytes.length > 31) {
- throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
- "fixint7 must be positive");
- }
- buf.put((byte) (FIXSTR_PREFIX + strBytes.length));
- buf.put(strBytes);
- }
-
- public static void packStr(ByteBuffer out, String in) {
- out.put(STR32);
- byte[] strBytes = in.getBytes(StandardCharsets.UTF_8);
- out.putInt(strBytes.length);
- out.put(strBytes);
- }
-
- private static void packStr(byte[] in, int offs, ByteBuffer out) {
- out.put(STR32);
- //TODO: tagged/untagged. closed support is borked so always tagged rn
- String str = UTF8StringUtil.toString(in, offs);
- byte[] strBytes = str.getBytes(StandardCharsets.UTF_8);
- out.putInt(strBytes.length);
- out.put(strBytes);
- }
-
- public static void packStr(String str, ByteBuffer out) {
- out.put(STR32);
- byte[] strBytes = str.getBytes(StandardCharsets.UTF_8);
- out.putInt(strBytes.length);
- out.put(strBytes);
- }
-
- private static void packArray(byte[] in, int offs, IAType type, ByteBuffer out) throws HyracksDataException {
- //TODO: - could optimize to pack fixarray/array16 for small arrays
- // - this code is basically a static version of AListPointable, could be deduped
- AbstractCollectionType collType = (AbstractCollectionType) type;
- out.put(ARRAY32);
- int lenOffs = offs + TYPE_TAG_SIZE + TYPE_SIZE;
- int itemCtOffs = LENGTH_SIZE + lenOffs;
- int itemCt = IntegerPointable.getInteger(in, itemCtOffs);
- boolean fixType = NonTaggedFormatUtil.isFixedSizedCollection(type);
- out.putInt(itemCt);
- for (int i = 0; i < itemCt; i++) {
- if (fixType) {
- int itemOffs = itemCtOffs + ITEM_COUNT_SIZE + (i
- * NonTaggedFormatUtil.getFieldValueLength(in, 0, collType.getItemType().getTypeTag(), false));
- pack(in, itemOffs, collType.getItemType(), false, true, out);
- } else {
- int itemOffs =
- offs + IntegerPointable.getInteger(in, itemCtOffs + ITEM_COUNT_SIZE + (i * ITEM_OFFSET_SIZE));
- ATypeTag tag = ATypeTag.VALUE_TYPE_MAPPING[BytePointable.getByte(in, itemOffs)];
- pack(in, itemOffs, TypeTagUtil.getBuiltinTypeByTag(tag), true, true, out);
- }
- }
- }
-
- private static void packObject(byte[] in, int offs, IAType type, ByteBuffer out) throws HyracksDataException {
- ARecordType recType = (ARecordType) type;
- out.put(MAP32);
- int fieldCt = recType.getFieldNames().length + RecordUtils.getOpenFieldCount(in, offs, recType);
- out.putInt(fieldCt);
- for (int i = 0; i < recType.getFieldNames().length; i++) {
- String field = recType.getFieldNames()[i];
- IAType fieldType = RecordUtils.getClosedFieldType(recType, i);
- packStr(field, out);
- pack(in, RecordUtils.getClosedFieldOffset(in, offs, recType, i), fieldType, false, true, out);
- }
- if (RecordUtils.isExpanded(in, offs, recType)) {
- for (int i = 0; i < RecordUtils.getOpenFieldCount(in, offs, recType); i++) {
- packStr(in, RecordUtils.getOpenFieldNameOffset(in, offs, recType, i), out);
- ATypeTag tag = ATypeTag.VALUE_TYPE_MAPPING[RecordUtils.getOpenFieldTag(in, offs, recType, i)];
- pack(in, RecordUtils.getOpenFieldValueOffset(in, offs, recType, i),
- TypeTagUtil.getBuiltinTypeByTag(tag), true, true, out);
- }
- }
-
- }
-
- public static void packFixArrayHeader(ByteBuffer buf, byte numObj) {
- buf.put((byte) (FIXARRAY_PREFIX + (0x0F & numObj)));
- }
-
- private static class RecordUtils {
-
- static final int TAG_SIZE = 1;
- static final int RECORD_LENGTH_SIZE = 4;
- static final int EXPANDED_SIZE = 1;
- static final int OPEN_OFFSET_SIZE = 4;
- static final int CLOSED_COUNT_SIZE = 4;
- static final int FIELD_OFFSET_SIZE = 4;
- static final int OPEN_COUNT_SIZE = 4;
- private static final int OPEN_FIELD_HASH_SIZE = 4;
- private static final int OPEN_FIELD_OFFSET_SIZE = 4;
- private static final int OPEN_FIELD_HEADER = OPEN_FIELD_HASH_SIZE + OPEN_FIELD_OFFSET_SIZE;
-
- private static boolean isOpen(ARecordType recordType) {
- return recordType == null || recordType.isOpen();
- }
-
- public static int getLength(byte[] bytes, int start) {
- return IntegerPointable.getInteger(bytes, start + TAG_SIZE);
- }
-
- public static boolean isExpanded(byte[] bytes, int start, ARecordType recordType) {
- return isOpen(recordType) && BooleanPointable.getBoolean(bytes, start + TAG_SIZE + RECORD_LENGTH_SIZE);
- }
-
- public static int getOpenPartOffset(int start, ARecordType recordType) {
- return start + TAG_SIZE + RECORD_LENGTH_SIZE + (isOpen(recordType) ? EXPANDED_SIZE : 0);
- }
-
- public static int getNullBitmapOffset(byte[] bytes, int start, ARecordType recordType) {
- return getOpenPartOffset(start, recordType) + (isExpanded(bytes, start, recordType) ? OPEN_OFFSET_SIZE : 0)
- + CLOSED_COUNT_SIZE;
- }
-
- public static int getNullBitmapSize(ARecordType recordType) {
- return RecordUtil.computeNullBitmapSize(recordType);
- }
-
- public static final IAType getClosedFieldType(ARecordType recordType, int fieldId) {
- IAType aType = recordType.getFieldTypes()[fieldId];
- if (NonTaggedFormatUtil.isOptional(aType)) {
- // optional field: add the embedded non-null type tag
- aType = ((AUnionType) aType).getActualType();
- }
- return aType;
- }
-
- public static final int getClosedFieldOffset(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- int offset = getNullBitmapOffset(bytes, start, recordType) + getNullBitmapSize(recordType)
- + fieldId * FIELD_OFFSET_SIZE;
- return start + IntegerPointable.getInteger(bytes, offset);
- }
-
- public static final int getOpenFieldCount(byte[] bytes, int start, ARecordType recordType) {
- return isExpanded(bytes, start, recordType)
- ? IntegerPointable.getInteger(bytes, getOpenFieldCountOffset(bytes, start, recordType)) : 0;
- }
-
- public static int getOpenFieldCountSize(byte[] bytes, int start, ARecordType recordType) {
- return isExpanded(bytes, start, recordType) ? OPEN_COUNT_SIZE : 0;
- }
-
- public static int getOpenFieldCountOffset(byte[] bytes, int start, ARecordType recordType) {
- return start + IntegerPointable.getInteger(bytes, getOpenPartOffset(start, recordType));
- }
-
- public static final int getOpenFieldValueOffset(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- return getOpenFieldNameOffset(bytes, start, recordType, fieldId)
- + getOpenFieldNameSize(bytes, start, recordType, fieldId);
- }
-
- public static int getOpenFieldNameSize(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- int utfleng = UTF8StringUtil.getUTFLength(bytes, getOpenFieldNameOffset(bytes, start, recordType, fieldId));
- return utfleng + UTF8StringUtil.getNumBytesToStoreLength(utfleng);
- }
-
- public static int getOpenFieldNameOffset(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- return getOpenFieldOffset(bytes, start, recordType, fieldId);
- }
-
- public static final byte getOpenFieldTag(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- return bytes[getOpenFieldValueOffset(bytes, start, recordType, fieldId)];
- }
-
- public static int getOpenFieldHashOffset(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- return getOpenFieldCountOffset(bytes, start, recordType) + getOpenFieldCountSize(bytes, start, recordType)
- + fieldId * OPEN_FIELD_HEADER;
- }
-
- public static int getOpenFieldOffset(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- return start
- + IntegerPointable.getInteger(bytes, getOpenFieldOffsetOffset(bytes, start, recordType, fieldId));
- }
-
- public static int getOpenFieldOffsetOffset(byte[] bytes, int start, ARecordType recordType, int fieldId) {
- return getOpenFieldHashOffset(bytes, start, recordType, fieldId) + OPEN_FIELD_HASH_SIZE;
- }
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java
index 4af1121..d6121e8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java
@@ -18,18 +18,39 @@
import static org.msgpack.core.MessagePack.Code.*;
+import java.io.DataOutput;
+import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import org.apache.asterix.builders.AbvsBuilderFactory;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.input.stream.StandardUTF8ToModifiedUTF8DataOutput;
+import org.apache.asterix.external.input.stream.builders.ListLikeNumericArrayFactory;
+import org.apache.asterix.external.input.stream.builders.StandardToModifiedUTF8DataOutputFactory;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.util.container.IObjectPool;
+import org.apache.asterix.om.util.container.ListObjectPool;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
+import org.apache.hyracks.data.std.api.IMutableValueStorage;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.util.string.UTF8StringUtil;
public class MessageUnpackerToADM {
- public static void unpack(ByteBuffer in, ByteBuffer out, boolean tagged) throws HyracksDataException {
+ private final IObjectPool<IMutableValueStorage, ATypeTag> abvsBuilderPool =
+ new ListObjectPool<>(new AbvsBuilderFactory());
+ private final IObjectPool<StandardUTF8ToModifiedUTF8DataOutput, ATypeTag> utfPool =
+ new ListObjectPool<>(new StandardToModifiedUTF8DataOutputFactory());
+ private final IObjectPool<List<Long>, Long> listPool = new ListObjectPool<>(new ListLikeNumericArrayFactory<>());
+
+ public MessageUnpackerToADM() {
+ }
+
+ public void unpack(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
byte tag = NIL;
if (in != null) {
tag = in.get();
@@ -38,12 +59,12 @@
unpackStr(in, out, (tag ^ FIXSTR_PREFIX), tagged);
} else if (isFixInt(tag)) {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT8_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT8_TYPE_TAG);
}
if (isPosFixInt(tag)) {
- out.put(tag);
+ out.writeByte(tag);
} else if (isNegFixInt(tag)) {
- out.put(tag);
+ out.writeByte(tag);
}
} else if (isFixedArray(tag)) {
unpackArray(in, out, (tag ^ FIXARRAY_PREFIX));
@@ -52,15 +73,15 @@
} else {
switch (tag) {
case TRUE:
- out.put(ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
- out.put((byte) 1);
+ out.writeByte(ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+ out.writeByte((byte) 1);
break;
case FALSE:
- out.put(ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
- out.put((byte) 0);
+ out.writeByte(ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
+ out.writeByte((byte) 0);
break;
case NIL:
- out.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
break;
case UINT8:
unpackUByte(in, out, tagged);
@@ -113,7 +134,6 @@
case MAP32:
unpackMap(in, out, (int) Integer.toUnsignedLong(in.getInt()));
break;
-
default:
throw HyracksDataException.create(AsterixException.create(
ErrorCode.PARSER_ADM_DATA_PARSER_CAST_ERROR, "msgpack tag " + tag + " ", "to an ADM type"));
@@ -121,154 +141,159 @@
}
}
- public static void unpackByte(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackByte(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT8_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT8_TYPE_TAG);
}
- out.put(in.get());
+ out.writeByte(in.get());
}
- public static void unpackShort(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackShort(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT16_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT16_TYPE_TAG);
}
- out.putShort(in.getShort());
+ out.writeShort(in.getShort());
}
- public static void unpackInt(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackInt(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT32_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
- out.putInt(in.getInt());
+ out.writeInt(in.getInt());
}
- public static void unpackLong(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackLong(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
- out.putLong(in.getLong());
+ out.writeLong(in.getLong());
}
- public static void unpackUByte(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackUByte(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT16_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT16_TYPE_TAG);
}
- out.putShort((short) (in.get() & ((short) 0x00FF)));
+ out.writeShort((short) (in.get() & ((short) 0x00FF)));
}
- public static void unpackUShort(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackUShort(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT32_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT32_TYPE_TAG);
}
- out.putInt(in.getShort() & 0x0000FFFF);
+ out.writeInt(in.getShort() & 0x0000FFFF);
}
- public static void unpackUInt(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackUInt(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
- out.putLong(in.getInt() & 0x00000000FFFFFFFFl);
+ out.writeLong(in.getInt() & 0x00000000FFFFFFFFl);
}
- public static void unpackULong(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackULong(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_INT64_TYPE_TAG);
}
long val = in.getLong();
if (val < 0) {
throw new IllegalArgumentException("Integer overflow");
}
- out.putLong(val);
+ out.writeLong(val);
}
- public static void unpackFloat(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackFloat(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
}
- out.putFloat(in.getFloat());
+ out.writeFloat(in.getFloat());
}
- public static void unpackDouble(ByteBuffer in, ByteBuffer out, boolean tagged) {
+ public static void unpackDouble(ByteBuffer in, DataOutput out, boolean tagged) throws IOException {
if (tagged) {
- out.put(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
}
- out.putDouble(in.getDouble());
+ out.writeDouble(in.getDouble());
}
- public static void unpackArray(ByteBuffer in, ByteBuffer out, long uLen) throws HyracksDataException {
+ public void unpackArray(ByteBuffer in, DataOutput out, long uLen) throws IOException {
if (uLen > Integer.MAX_VALUE) {
throw new UnsupportedOperationException("Array is too long");
}
+ ArrayBackedValueStorage buildBuf = (ArrayBackedValueStorage) abvsBuilderPool.allocate(ATypeTag.ARRAY);
+ buildBuf.reset();
+ DataOutput bufOut = buildBuf.getDataOutput();
int count = (int) uLen;
- int offs = out.position();
- out.put(ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
- out.put(ATypeTag.ANY.serialize());
- int asxLenPos = out.position();
+ bufOut.writeByte(ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG);
+ bufOut.writeByte(ATypeTag.ANY.serialize());
+ int asxLenPos = buildBuf.getLength();
//reserve space
- out.putInt(-1);
- out.putInt(count);
- int slotStartOffs = out.position() + out.arrayOffset();
+ bufOut.writeInt(-1);
+ bufOut.writeInt(count);
+ int slotStartOffs = buildBuf.getLength();
for (int i = 0; i < count; i++) {
- out.putInt(0xFFFF);
+ bufOut.writeInt(0xDEADBEEF);
}
for (int i = 0; i < count; i++) {
- out.putInt(slotStartOffs + (i * 4), (out.position() - offs));
- unpack(in, out, true);
+ IntegerPointable.setInteger(buildBuf.getByteArray(), ((slotStartOffs) + (i * 4)), buildBuf.getLength());
+ //tagged b/c any
+ unpack(in, bufOut, true);
}
- int totalLen = out.position() - offs;
- out.putInt(asxLenPos, totalLen);
+ IntegerPointable.setInteger(buildBuf.getByteArray(), asxLenPos, buildBuf.getLength());
+ out.write(buildBuf.getByteArray(), buildBuf.getStartOffset(), buildBuf.getLength());
}
- public static void unpackMap(ByteBuffer in, ByteBuffer out, int count) throws HyracksDataException {
+ public void unpackMap(ByteBuffer in, DataOutput out, int count) throws IOException {
//TODO: need to handle typed records. this only produces a completely open record.
+ ArrayBackedValueStorage buildBuf = (ArrayBackedValueStorage) abvsBuilderPool.allocate(ATypeTag.OBJECT);
+ List<Long> offsets = listPool.allocate((long) count);
+ DataOutput bufOut = buildBuf.getDataOutput();
//hdr size = 6?
- int startOffs = out.position();
- out.put(ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
- int totalSizeOffs = out.position();
- out.putInt(-1);
+ bufOut.writeByte(ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
+ int totalSizeOffs = buildBuf.getLength();
+ bufOut.writeInt(-1);
//isExpanded
- out.put((byte) 1);
- int openPartOffs = out.position();
- out.putInt(-1);
+ bufOut.writeByte((byte) 1);
+ int openPartOffs = buildBuf.getLength();
+ bufOut.writeInt(-1);
//isExpanded, so num of open fields
- out.putInt(openPartOffs, out.position() - startOffs);
- out.putInt(count);
- int offsetAryPos = out.position();
+ IntegerPointable.setInteger(buildBuf.getByteArray(), openPartOffs, buildBuf.getLength());
+ bufOut.writeInt(count);
+ int offsetAryPos = buildBuf.getLength();
int offsetArySz = count * 2;
//allocate space for open field offsets
for (int i = 0; i < offsetArySz; i++) {
- out.putInt(0xDEADBEEF);
+ bufOut.writeInt(0xDEADBEEF);
}
for (int i = 0; i < count; i++) {
- int offs = out.position() + out.arrayOffset();
- int relOffs = offs - startOffs;
- unpack(in, out, false);
- int hash = UTF8StringUtil.hash(out.array(), offs);
- out.putInt(offsetAryPos, hash);
- offsetAryPos += 4;
- out.putInt(offsetAryPos, relOffs);
- offsetAryPos += 4;
- unpack(in, out, true);
+ int offs = buildBuf.getLength();
+ unpack(in, bufOut, false);
+ long hash = UTF8StringUtil.hash(buildBuf.getByteArray(), offs);
+ offsets.set(i, (hash << 32) + offs);
+ unpack(in, bufOut, true);
}
- out.putInt(totalSizeOffs, out.position() - startOffs);
+ Collections.sort(offsets);
+ for (Long l : offsets) {
+ IntegerPointable.setInteger(buildBuf.getByteArray(), offsetAryPos, (int) (l >> 32));
+ offsetAryPos += 4;
+ IntegerPointable.setInteger(buildBuf.getByteArray(), offsetAryPos, (int) ((l << 32) >> 32));
+ offsetAryPos += 4;
+ }
+ IntegerPointable.setInteger(buildBuf.getByteArray(), totalSizeOffs, buildBuf.getLength());
+ out.write(buildBuf.getByteArray(), buildBuf.getStartOffset(), buildBuf.getLength());
}
- public static void unpackStr(ByteBuffer in, ByteBuffer out, long uLen, boolean tag) {
- //TODO: this probably breaks for 3 and 4 byte UTF-8
+ public void unpackStr(ByteBuffer in, DataOutput out, long uLen, boolean tag) throws IOException {
if (tag) {
- out.put(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
}
if (Long.compareUnsigned(uLen, Integer.MAX_VALUE) > 0) {
throw new UnsupportedOperationException("String is too long");
}
int len = (int) uLen;
- int strLen = UTF8StringUtil.getStringLength(in.array(), in.position() + in.arrayOffset(), len);
- int adv = VarLenIntEncoderDecoder.encode(strLen, out.array(), out.position() + out.arrayOffset());
- out.position(out.position() + adv);
- System.arraycopy(in.array(), in.arrayOffset() + in.position(), out.array(), out.arrayOffset() + out.position(),
- len);
- out.position(out.position() + len);
+ StandardUTF8ToModifiedUTF8DataOutput conv = utfPool.allocate(ATypeTag.STRING);
+ conv.setDataOutput(out);
+ conv.write(in.array(), in.arrayOffset() + in.position(), len);
in.position(in.position() + len);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
new file mode 100644
index 0000000..a90a183
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackAccessors.java
@@ -0,0 +1,275 @@
+/*
+ * 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.external.library.msgpack;
+
+import static org.apache.hyracks.util.string.UTF8StringUtil.getUTFLength;
+import static org.msgpack.core.MessagePack.Code.ARRAY32;
+import static org.msgpack.core.MessagePack.Code.FALSE;
+import static org.msgpack.core.MessagePack.Code.FLOAT32;
+import static org.msgpack.core.MessagePack.Code.FLOAT64;
+import static org.msgpack.core.MessagePack.Code.INT16;
+import static org.msgpack.core.MessagePack.Code.INT32;
+import static org.msgpack.core.MessagePack.Code.INT64;
+import static org.msgpack.core.MessagePack.Code.INT8;
+import static org.msgpack.core.MessagePack.Code.MAP32;
+import static org.msgpack.core.MessagePack.Code.NIL;
+import static org.msgpack.core.MessagePack.Code.STR32;
+import static org.msgpack.core.MessagePack.Code.TRUE;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.asterix.external.library.PyTypeInfo;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
+
+public class MsgPackAccessors {
+
+ private MsgPackAccessors() {
+ }
+
+ public static IMsgPackAccessor<IPointable, DataOutput, Void> createFlatMsgPackAccessor(ATypeTag aTypeTag)
+ throws HyracksDataException {
+ switch (aTypeTag) {
+ case BOOLEAN:
+ return MsgPackBooleanAccessor::apply;
+ case TINYINT:
+ return MsgPackInt8Accessor::apply;
+ case SMALLINT:
+ return MsgPackInt16Accessor::apply;
+ case INTEGER:
+ return MsgPackInt32Accessor::apply;
+ case BIGINT:
+ return MsgPackInt64Accessor::apply;
+ case FLOAT:
+ return MsgPackFloatAccessor::apply;
+ case DOUBLE:
+ return MsgPackDoubleAccessor::apply;
+ case STRING:
+ return MsgPackStringAccessor::apply;
+ case MISSING:
+ case NULL:
+ return MsgPackNullAccessor::apply;
+ default:
+ throw HyracksDataException
+ .create(AsterixException.create(ErrorCode.TYPE_UNSUPPORTED, "msgpack", aTypeTag.name()));
+ }
+ }
+
+ public static class MsgPackInt8Accessor {
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ byte o = AInt8SerializerDeserializer.getByte(b, s + 1);
+ out.writeByte(INT8);
+ out.writeByte(o);
+ return null;
+ }
+ }
+
+ public static class MsgPackInt16Accessor {
+
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ short i = AInt16SerializerDeserializer.getShort(b, s + 1);
+ out.writeByte(INT16);
+ out.writeShort(i);
+ return null;
+ }
+ }
+
+ public static class MsgPackInt32Accessor {
+
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int i = AInt32SerializerDeserializer.getInt(b, s + 1);
+ out.writeByte(INT32);
+ out.writeByte(i);
+ return null;
+ }
+ }
+
+ public static class MsgPackNullAccessor {
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ out.writeByte(NIL);
+ return null;
+ }
+ }
+
+ public static class MsgPackInt64Accessor {
+
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ long v = AInt64SerializerDeserializer.getLong(b, s + 1);
+ out.writeByte(INT64);
+ out.writeLong(v);
+ return null;
+ }
+ }
+
+ public static class MsgPackFloatAccessor {
+
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ float v = AFloatSerializerDeserializer.getFloat(b, s + 1);
+ out.writeByte(FLOAT32);
+ out.writeFloat(v);
+ return null;
+ }
+ }
+
+ public static class MsgPackDoubleAccessor {
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ double v = ADoubleSerializerDeserializer.getDouble(b, s + 1);
+ out.writeByte(FLOAT64);
+ out.writeDouble(v);
+ return null;
+ }
+ }
+
+ public static class MsgPackStringAccessor {
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ out.writeByte(STR32);
+ final int calculatedLength = getUTFLength(b, s + 1);
+ out.writeInt(calculatedLength);
+ PrintTools.writeUTF8StringRaw(b, s + 1, calculatedLength, out);
+ return null;
+ }
+
+ }
+
+ public static class MsgPackBooleanAccessor {
+ public static Void apply(IPointable pointable, DataOutput out) throws IOException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ boolean v = ABooleanSerializerDeserializer.getBoolean(b, s + 1);
+ if (v) {
+ out.writeByte(TRUE);
+ } else {
+ out.writeByte(FALSE);
+ }
+ return null;
+ }
+ }
+
+ public static class MsgPackRecordAccessor {
+
+ public static int getUTFLength(byte[] b, int s) {
+ return VarLenIntEncoderDecoder.decode(b, s);
+ }
+
+ public static Void access(ARecordVisitablePointable pointable, PyTypeInfo arg,
+ MsgPackPointableVisitor pointableVisitor) throws HyracksDataException {
+ List<IVisitablePointable> fieldPointables = pointable.getFieldValues();
+ List<IVisitablePointable> fieldTypeTags = pointable.getFieldTypeTags();
+ List<IVisitablePointable> fieldNames = pointable.getFieldNames();
+ boolean closedPart;
+ int index = 0;
+ DataOutput out = arg.getDataOutput();
+ ARecordType recordType = ((ARecordType) arg.getType());
+ try {
+ out.writeByte(MAP32);
+ out.writeInt(fieldNames.size());
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ try {
+ for (IVisitablePointable fieldPointable : fieldPointables) {
+ closedPart = index < recordType.getFieldTypes().length;
+ IVisitablePointable tt = fieldTypeTags.get(index);
+ ATypeTag typeTag =
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(tt.getByteArray()[tt.getStartOffset()]);
+ IAType fieldType;
+ fieldType =
+ closedPart ? recordType.getFieldTypes()[index] : TypeTagUtil.getBuiltinTypeByTag(typeTag);
+ IPointable fieldName = fieldNames.get(index);
+ MsgPackAccessors.createFlatMsgPackAccessor(BuiltinType.ASTRING.getTypeTag()).apply(fieldName,
+ arg.getDataOutput());
+ PyTypeInfo fieldTypeInfo = pointableVisitor.getTypeInfo(fieldType, out);
+ fieldPointable.accept(pointableVisitor, fieldTypeInfo);
+ index++;
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ return null;
+ }
+ }
+
+ public static class MsgPackListAccessor {
+
+ public static Void access(AListVisitablePointable pointable, PyTypeInfo arg,
+ MsgPackPointableVisitor pointableVisitor) throws HyracksDataException {
+ List<IVisitablePointable> items = pointable.getItems();
+ List<IVisitablePointable> itemTags = pointable.getItemTags();
+ DataOutput out = arg.getDataOutput();
+ try {
+ out.writeByte(ARRAY32);
+ out.writeInt(items.size());
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ for (int iter1 = 0; iter1 < items.size(); iter1++) {
+ IVisitablePointable itemPointable = items.get(iter1);
+ // First, try to get defined type.
+ IAType fieldType = ((AbstractCollectionType) arg.getType()).getItemType();
+ if (fieldType.getTypeTag() == ATypeTag.ANY) {
+ // Second, if defined type is not available, try to infer it from data
+ IVisitablePointable itemTagPointable = itemTags.get(iter1);
+ ATypeTag itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+ .deserialize(itemTagPointable.getByteArray()[itemTagPointable.getStartOffset()]);
+ fieldType = TypeTagUtil.getBuiltinTypeByTag(itemTypeTag);
+ }
+ PyTypeInfo fieldTypeInfo = pointableVisitor.getTypeInfo(fieldType, out);
+ itemPointable.accept(pointableVisitor, fieldTypeInfo);
+ }
+ return null;
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackPointableVisitor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackPointableVisitor.java
new file mode 100644
index 0000000..08be6ac
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MsgPackPointableVisitor.java
@@ -0,0 +1,77 @@
+/*
+ * 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.external.library.msgpack;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.external.library.PyTypeInfo;
+import org.apache.asterix.om.pointables.AFlatValuePointable;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.visitor.IVisitablePointableVisitor;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class MsgPackPointableVisitor implements IVisitablePointableVisitor<Void, PyTypeInfo> {
+
+ Map<DataOutput, Map<IAType, PyTypeInfo>> typeInfoMap = new HashMap<>();
+
+ public Void visit(AListVisitablePointable accessor, PyTypeInfo arg) throws HyracksDataException {
+ MsgPackAccessors.MsgPackListAccessor.access(accessor, arg, this);
+ return null;
+ }
+
+ @Override
+ public Void visit(ARecordVisitablePointable accessor, PyTypeInfo arg) throws HyracksDataException {
+ MsgPackAccessors.MsgPackRecordAccessor.access(accessor, arg, this);
+ return null;
+ }
+
+ @Override
+ public Void visit(AFlatValuePointable accessor, PyTypeInfo arg) throws HyracksDataException {
+ try {
+ MsgPackAccessors.createFlatMsgPackAccessor(arg.getType().getTypeTag()).apply(accessor, arg.getDataOutput());
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ return null;
+ }
+
+ public PyTypeInfo getTypeInfo(IAType type, DataOutput out) {
+ PyTypeInfo tInfo = null;
+ Map<IAType, PyTypeInfo> type2TypeInfo = typeInfoMap.get(out);
+ if (type2TypeInfo == null) {
+ type2TypeInfo = new HashMap<>();
+ tInfo = new PyTypeInfo(type, out);
+ type2TypeInfo.put(type, tInfo);
+ typeInfoMap.put(out, type2TypeInfo);
+ }
+ tInfo = tInfo == null ? type2TypeInfo.get(type) : tInfo;
+ if (tInfo == null) {
+ tInfo = new PyTypeInfo(type, out);
+ type2TypeInfo.put(type, tInfo);
+ }
+ return tInfo;
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
index 593bac6..741dad2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
@@ -33,11 +33,14 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.ipc.PythonIPCProto;
import org.apache.asterix.external.library.PythonLibraryEvaluator;
import org.apache.asterix.external.library.PythonLibraryEvaluatorFactory;
import org.apache.asterix.external.library.msgpack.MessageUnpackerToADM;
-import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.library.msgpack.MsgPackPointableVisitor;
+import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.om.functions.IExternalFunctionDescriptor;
+import org.apache.asterix.om.pointables.PointableAllocator;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.Counter;
@@ -48,6 +51,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import org.msgpack.core.MessagePack;
@@ -58,12 +62,10 @@
public final class ExternalAssignBatchRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
private static final long serialVersionUID = 1L;
- private int[] outColumns;
+ private final int[] outColumns;
private final IExternalFunctionDescriptor[] fnDescs;
private final int[][] fnArgColumns;
- private int rpcBufferSize;
-
public ExternalAssignBatchRuntimeFactory(int[] outColumns, IExternalFunctionDescriptor[] fnDescs,
int[][] fnArgColumns, int[] projectionList) {
super(projectionList);
@@ -76,17 +78,14 @@
public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx) {
final int[] projectionToOutColumns = new int[projectionList.length];
- //this is a temporary bodge. these buffers need to work like vsize frames, or be absent entirely
- int maxArgSz = ExternalDataUtils.getArgBufferSize();
- rpcBufferSize = ExternalDataUtils.roundUpToNearestFrameSize(maxArgSz, ctx.getInitialFrameSize());
for (int j = 0; j < projectionList.length; j++) {
projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
}
return new AbstractOneInputOneOutputOneFramePushRuntime() {
- private ByteBuffer outputWrapper;
- private List<ByteBuffer> argHolders;
+ private ArrayBackedValueStorage outputWrapper;
+ private List<ArrayBackedValueStorage> argHolders;
ArrayTupleBuilder tupleBuilder;
private List<Pair<Long, PythonLibraryEvaluator>> libraryEvaluators;
private ATypeTag[][] nullCalls;
@@ -95,6 +94,9 @@
private MessageUnpacker unpacker;
private ArrayBufferInput unpackerInput;
private List<Pair<ByteBuffer, Counter>> batchResults;
+ private MessageUnpackerToADM unpackerToADM;
+ private PointableAllocator pointableAllocator;
+ private MsgPackPointableVisitor pointableVisitor;
@Override
public void open() throws HyracksDataException {
@@ -116,23 +118,26 @@
}
argHolders = new ArrayList<>(fnArgColumns.length);
for (int i = 0; i < fnArgColumns.length; i++) {
- argHolders.add(ctx.allocateFrame(rpcBufferSize));
+ argHolders.add(new ArrayBackedValueStorage());
}
- outputWrapper = ctx.allocateFrame();
+ outputWrapper = new ArrayBackedValueStorage();
nullCalls = new ATypeTag[argHolders.size()][0];
numCalls = new int[fnArgColumns.length];
batchResults = new ArrayList<>(argHolders.size());
for (int i = 0; i < argHolders.size(); i++) {
- batchResults.add(new Pair<>(ctx.allocateFrame(rpcBufferSize), new Counter(-1)));
+ batchResults.add(new Pair<>(ByteBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE),
+ new Counter(-1)));
}
unpackerInput = new ArrayBufferInput(new byte[0]);
unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
+ unpackerToADM = new MessageUnpackerToADM();
+ pointableAllocator = new PointableAllocator();
+ pointableVisitor = new MsgPackPointableVisitor();
}
private void resetBuffers(int numTuples, int[] numCalls) {
for (int func = 0; func < fnArgColumns.length; func++) {
- argHolders.get(func).clear();
- argHolders.get(func).position(0);
+ argHolders.get(func).reset();
if (nullCalls[func].length < numTuples) {
nullCalls[func] = new ATypeTag[numTuples];
}
@@ -188,6 +193,8 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ /*TODO: this whole transposition stuff is not necessary
+ the evaulator should accept a format that is a collection of rows, logically*/
tAccess.reset(buffer);
tupleBuilder.reset();
try {
@@ -211,14 +218,15 @@
}
if (argumentStatus == ATypeTag.TYPE) {
if (cols.length > 0) {
- argHolders.get(func).put(ARRAY16);
- argHolders.get(func).putShort((short) cols.length);
+ argHolders.get(func).getDataOutput().writeByte(ARRAY16);
+ argHolders.get(func).getDataOutput().writeShort((short) cols.length);
}
for (int colIdx = 0; colIdx < cols.length; colIdx++) {
ref.set(buffer.array(), tRef.getFieldStart(cols[colIdx]),
tRef.getFieldLength(cols[colIdx]));
- PythonLibraryEvaluator.setArgument(fnDescs[func].getArgumentTypes()[colIdx], ref,
- argHolders.get(func), fnDescs[func].getFunctionInfo().getNullCall());
+ PythonIPCProto.visitValueRef(fnDescs[func].getArgumentTypes()[colIdx],
+ argHolders.get(func).getDataOutput(), ref, pointableAllocator,
+ pointableVisitor, fnDescs[func].getFunctionInfo().getNullCall());
}
} else {
numCalls[func]--;
@@ -228,6 +236,7 @@
}
}
}
+
//TODO: maybe this could be done in parallel for each unique library evaluator?
for (int argHolderIdx = 0; argHolderIdx < argHolders.size(); argHolderIdx++) {
Pair<Long, PythonLibraryEvaluator> fnEval = libraryEvaluators.get(argHolderIdx);
@@ -236,15 +245,14 @@
if (columnResult != null) {
Pair<ByteBuffer, Counter> resultholder = batchResults.get(argHolderIdx);
if (resultholder.getFirst().capacity() < columnResult.capacity()) {
- resultholder.setFirst(ctx.allocateFrame(ExternalDataUtils.roundUpToNearestFrameSize(
- columnResult.capacity(), ctx.getInitialFrameSize())));
+ ByteBuffer realloc = ctx.reallocateFrame(resultholder.getFirst(),
+ columnResult.capacity() * 2, false);
+ resultholder.setFirst(realloc);
}
ByteBuffer resultBuf = resultholder.getFirst();
- resultBuf.clear();
- resultBuf.position(0);
//offset 1 to skip message type
- System.arraycopy(columnResult.array(), columnResult.arrayOffset() + 1, resultBuf.array(),
- resultBuf.arrayOffset(), columnResult.capacity() - 1);
+ System.arraycopy(columnResult.array(), 1, resultBuf.array(), 0,
+ columnResult.remaining() - 1);
//wrapper for results and warnings arrays. always length 2
consumeAndGetBatchLength(resultBuf);
int numResults = (int) consumeAndGetBatchLength(resultBuf);
@@ -260,37 +268,30 @@
}
}
}
+
//decompose returned function columns into frame tuple format
for (int i = 0; i < numTuples; i++) {
tupleBuilder.reset();
for (int f = 0; f < projectionList.length; f++) {
int k = projectionToOutColumns[f];
if (k >= 0) {
- outputWrapper.clear();
- outputWrapper.position(0);
+ outputWrapper.reset();
Pair<ByteBuffer, Counter> result = batchResults.get(k);
- if (result.getFirst() != null) {
- if (result.getFirst().capacity() > outputWrapper.capacity()) {
- outputWrapper = ctx.allocateFrame(ExternalDataUtils.roundUpToNearestFrameSize(
- outputWrapper.capacity(), ctx.getInitialFrameSize()));
- }
- }
- int start = outputWrapper.arrayOffset();
ATypeTag functionCalled = nullCalls[k][i];
if (functionCalled == ATypeTag.TYPE) {
if (result.getSecond().get() > 0) {
- MessageUnpackerToADM.unpack(result.getFirst(), outputWrapper, true);
+ unpackerToADM.unpack(result.getFirst(), outputWrapper.getDataOutput(), true);
result.getSecond().set(result.getSecond().get() - 1);
} else {
//emit NULL for functions which failed with a warning
- outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ outputWrapper.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
}
} else if (functionCalled == ATypeTag.NULL) {
- outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ outputWrapper.getDataOutput().writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
} else {
- outputWrapper.put(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ outputWrapper.getDataOutput().writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
}
- tupleBuilder.addField(outputWrapper.array(), start, start + outputWrapper.position());
+ tupleBuilder.addField(outputWrapper.getByteArray(), 0, outputWrapper.getLength());
} else {
tupleBuilder.addField(tAccess, i, projectionList[f]);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
index 5ecd283..56bedea 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
@@ -40,7 +40,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false);
+ maxFilterFieldIndexes, false, null);
this.version = version;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
index 80ebb4c..4e669c3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
@@ -46,7 +46,8 @@
boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, int version) throws HyracksDataException {
super(ctx, partition, intputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, null, null,
- indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false);
+ indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false,
+ null);
this.version = version;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
index 493d3ba..bd6c922 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
@@ -41,7 +41,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
super(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false);
+ maxFilterFieldIndexes, false, null);
this.version = version;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
index 0b294de..508cff7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
@@ -46,7 +46,7 @@
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int version) throws HyracksDataException {
super(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false);
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false, null);
this.version = version;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
index 37e589a..7bc3b4f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
@@ -162,7 +162,7 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- long tid = tracer.durationB("Ingestion-Store", traceCategory, null);
+ long tid = tracer.durationB("Ingestion-Store", traceCategory);
try {
if (hasMessage) {
FeedUtils.processFeedMessage(buffer, message, fta);
@@ -172,7 +172,7 @@
LOGGER.log(Level.WARN, "Failure Processing a frame at store side", e);
throw HyracksDataException.create(e);
} finally {
- tracer.durationE(tid, traceCategory, null);
+ tracer.durationE(traceCategory, tid);
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
index cec4b2e..400ddf8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
@@ -41,6 +41,7 @@
import org.apache.asterix.om.base.AInt8;
import org.apache.asterix.om.base.AInterval;
import org.apache.asterix.om.base.ALine;
+import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.AMutableBinary;
import org.apache.asterix.om.base.AMutableCircle;
import org.apache.asterix.om.base.AMutableDate;
@@ -58,6 +59,7 @@
import org.apache.asterix.om.base.AMutableLine;
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.AMutablePoint3D;
+import org.apache.asterix.om.base.AMutablePolygon;
import org.apache.asterix.om.base.AMutableRectangle;
import org.apache.asterix.om.base.AMutableString;
import org.apache.asterix.om.base.AMutableTime;
@@ -66,6 +68,7 @@
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.APoint;
import org.apache.asterix.om.base.APoint3D;
+import org.apache.asterix.om.base.APolygon;
import org.apache.asterix.om.base.ARectangle;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.ATime;
@@ -78,8 +81,10 @@
import org.apache.asterix.om.base.temporal.ATimeParserFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.common.NumberUtils;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.StringUtil;
import org.apache.hyracks.util.bytes.Base64Parser;
import org.apache.hyracks.util.bytes.HexParser;
import org.apache.hyracks.util.string.UTF8StringReader;
@@ -114,6 +119,7 @@
protected AMutableRectangle aRectangle = new AMutableRectangle(null, null);
protected AMutablePoint aPoint2 = new AMutablePoint(0, 0);
protected AMutableLine aLine = new AMutableLine(null, null);
+ protected AMutablePolygon aPolygon = new AMutablePolygon(null);
protected AMutableDate aDate = new AMutableDate(0);
protected AMutableInterval aInterval = new AMutableInterval(0L, 0L, (byte) 0);
@@ -146,6 +152,9 @@
protected ISerializerDeserializer<ABoolean> booleanSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
@SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<AMissing> missingSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+ @SuppressWarnings("unchecked")
protected ISerializerDeserializer<ANull> nullSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
@@ -201,6 +210,9 @@
protected final static ISerializerDeserializer<ALine> lineSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ALINE);
@SuppressWarnings("unchecked")
+ protected final static ISerializerDeserializer<APolygon> polygonSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.APOLYGON);
+ @SuppressWarnings("unchecked")
protected static final ISerializerDeserializer<AInterval> intervalSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINTERVAL);
@@ -235,13 +247,13 @@
protected void parseDateTimeDuration(char[] buffer, int begin, int len, DataOutput out)
throws HyracksDataException {
- ADurationParserFactory.parseDuration(buffer, begin, len, aDayTimeDuration, ADurationParseOption.All);
+ ADurationParserFactory.parseDuration(buffer, begin, len, aDayTimeDuration, ADurationParseOption.DAY_TIME);
dayTimeDurationSerde.serialize(aDayTimeDuration, out);
}
protected void parseYearMonthDuration(char[] buffer, int begin, int len, DataOutput out)
throws HyracksDataException {
- ADurationParserFactory.parseDuration(buffer, begin, len, aYearMonthDuration, ADurationParseOption.All);
+ ADurationParserFactory.parseDuration(buffer, begin, len, aYearMonthDuration, ADurationParseOption.YEAR_MONTH);
yearMonthDurationSerde.serialize(aYearMonthDuration, out);
}
@@ -342,6 +354,11 @@
binarySerde.serialize(aBinary, out);
}
+ protected void parseUUID(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ aUUID.parseUUIDString(buffer, start, length);
+ uuidSerde.serialize(aUUID, out);
+ }
+
protected long parseDatePart(String interval, int startOffset, int endOffset) throws HyracksDataException {
while (interval.charAt(endOffset) == '"' || interval.charAt(endOffset) == ' ') {
@@ -368,16 +385,33 @@
return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
}
- protected double parseDouble(char[] buffer, int begin, int len) {
+ protected double parseDouble(char[] buffer, int begin, int len) throws ParseException {
// TODO: parse double directly from char[]
String str = new String(buffer, begin, len);
- return Double.valueOf(str);
+ try {
+ return Double.parseDouble(str);
+ } catch (NumberFormatException e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, str,
+ BuiltinType.ADOUBLE.getTypeName());
+ }
}
- protected float parseFloat(char[] buffer, int begin, int len) {
- //TODO: pares float directly from char[]
+ protected float parseFloat(char[] buffer, int begin, int len) throws ParseException {
+ // TODO: parse float directly from char[]
String str = new String(buffer, begin, len);
- return Float.valueOf(str);
+ try {
+ return Float.parseFloat(str);
+ } catch (NumberFormatException e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, str,
+ BuiltinType.AFLOAT.getTypeName());
+ }
+ }
+
+ protected void parseInt64(char[] buffer, int begin, int len, AMutableInt64 result) throws ParseException {
+ if (!NumberUtils.parseInt64(buffer, begin, begin + len, StringUtil.getCharArrayAccessor(), result, null)) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+ BuiltinType.AINT64.getTypeName());
+ }
}
protected int indexOf(char[] buffer, int begin, int len, char target) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractJsonDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractJsonDataParser.java
index 2d20cc2..9a602de 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractJsonDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractJsonDataParser.java
@@ -73,10 +73,8 @@
/**
* Initialize JSONDataParser with GeometryCoParser
*
- * @param recordType
- * defined type.
- * @param jsonFactory
- * Jackson JSON parser factory.
+ * @param recordType defined type.
+ * @param jsonFactory Jackson JSON parser factory.
*/
public AbstractJsonDataParser(ARecordType recordType, JsonFactory jsonFactory) {
// recordType currently cannot be null, however this is to guarantee for any future changes.
@@ -101,7 +99,7 @@
parseValue(BuiltinType.ANY, out);
return true;
} catch (IOException e) {
- throw new RuntimeDataException(ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM, e);
+ throw createException(e);
}
}
@@ -238,8 +236,7 @@
/**
* Geometry in GeoJSON is an object
*
- * @param typeTag
- * geometry typeTag
+ * @param typeTag geometry typeTag
* @param out
* @throws IOException
*/
@@ -329,7 +326,7 @@
break;
case INT:
case DOUBLE:
- serailizeNumeric(actualType.getTypeTag(), out);
+ serializeNumeric(actualType.getTypeTag(), out);
break;
case STRING:
serializeString(actualType.getTypeTag(), out);
@@ -352,7 +349,7 @@
* @param out
* @throws IOException
*/
- private void serailizeNumeric(ATypeTag numericType, DataOutput out) throws IOException {
+ protected void serializeNumeric(ATypeTag numericType, DataOutput out) throws IOException {
final ATypeTag typeToUse = numericType == ATypeTag.ANY ? currentToken().getTypeTag() : numericType;
switch (typeToUse) {
@@ -393,7 +390,7 @@
* @param out
* @throws IOException
*/
- private void serializeString(ATypeTag stringVariantType, DataOutput out) throws IOException {
+ protected void serializeString(ATypeTag stringVariantType, DataOutput out) throws IOException {
char[] buffer = jsonParser.getTextCharacters();
int begin = jsonParser.getTextOffset();
int len = jsonParser.getTextLength();
@@ -412,6 +409,18 @@
case TIME:
parseTime(buffer, begin, len, out);
break;
+ case YEARMONTHDURATION:
+ parseYearMonthDuration(buffer, begin, len, out);
+ break;
+ case DAYTIMEDURATION:
+ parseDateTimeDuration(buffer, begin, len, out);
+ break;
+ case DURATION:
+ parseDuration(buffer, begin, len, out);
+ break;
+ case UUID:
+ parseUUID(buffer, begin, len, out);
+ break;
default:
throw new RuntimeDataException(ErrorCode.TYPE_UNSUPPORTED, jsonParser.currentToken().toString());
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractNestedDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractNestedDataParser.java
index eecbb19..bc15f1e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractNestedDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractNestedDataParser.java
@@ -183,8 +183,10 @@
protected boolean isConvertable(ATypeTag parsedTypeTag, ATypeTag definedTypeTag) {
boolean convertable = parsedTypeTag == ATypeTag.STRING;
- convertable &= definedTypeTag == ATypeTag.UUID || definedTypeTag == ATypeTag.DATE
- || definedTypeTag == ATypeTag.TIME || definedTypeTag == ATypeTag.DATETIME;
+ convertable &=
+ definedTypeTag == ATypeTag.UUID || definedTypeTag == ATypeTag.DATE || definedTypeTag == ATypeTag.TIME
+ || definedTypeTag == ATypeTag.DATETIME || definedTypeTag == ATypeTag.YEARMONTHDURATION
+ || definedTypeTag == ATypeTag.DAYTIMEDURATION || definedTypeTag == ATypeTag.DURATION;
return convertable || ATypeHierarchy.canPromote(parsedTypeTag, definedTypeTag)
|| ATypeHierarchy.canDemote(parsedTypeTag, definedTypeTag);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
deleted file mode 100644
index 9ab6c7d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
+++ /dev/null
@@ -1,379 +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.
- */
-package org.apache.asterix.external.parser;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.asterix.builders.IARecordBuilder;
-import org.apache.asterix.builders.OrderedListBuilder;
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.builders.UnorderedListBuilder;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.api.IRecordDataParser;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
-import org.apache.asterix.om.types.AOrderedListType;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.AUnionType;
-import org.apache.asterix.om.types.AUnorderedListType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.NonTaggedFormatUtil;
-import org.apache.hadoop.hive.serde.Constants;
-import org.apache.hadoop.hive.serde2.SerDe;
-import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.StructField;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.util.string.UTF8StringWriter;
-
-@SuppressWarnings("deprecation")
-public class HiveRecordParser implements IRecordDataParser<Writable> {
-
- private ARecordType recordType;
- private SerDe hiveSerde;
- private StructObjectInspector oi;
- private IARecordBuilder recBuilder;
- private ArrayBackedValueStorage fieldValueBuffer;
- private ArrayBackedValueStorage listItemBuffer;
- private byte[] fieldTypeTags;
- private IAType[] fieldTypes;
- private OrderedListBuilder orderedListBuilder;
- private UnorderedListBuilder unorderedListBuilder;
- private List<? extends StructField> fieldRefs;
- private UTF8StringWriter utf8Writer = new UTF8StringWriter();
-
- public HiveRecordParser(ARecordType recordType, JobConf hadoopConfiguration, String hiveSerdeClassName)
- throws HyracksDataException {
- try {
- this.recordType = recordType;
- int n = recordType.getFieldNames().length;
- fieldTypes = recordType.getFieldTypes();
- //create the hive table schema.
- Properties tbl = new Properties();
- tbl.put(Constants.LIST_COLUMNS, getCommaDelimitedColNames(this.recordType));
- tbl.put(Constants.LIST_COLUMN_TYPES, getColTypes(this.recordType));
- hiveSerde = (SerDe) Class.forName(hiveSerdeClassName).newInstance();
- hiveSerde.initialize(hadoopConfiguration, tbl);
- oi = (StructObjectInspector) hiveSerde.getObjectInspector();
- fieldValueBuffer = new ArrayBackedValueStorage();
- recBuilder = new RecordBuilder();
- recBuilder.reset(recordType);
- recBuilder.init();
- fieldTypeTags = new byte[n];
- for (int i = 0; i < n; i++) {
- ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
- fieldTypeTags[i] = tag.serialize();
- }
- fieldRefs = oi.getAllStructFieldRefs();
- } catch (
-
- Exception e)
-
- {
- throw HyracksDataException.create(e);
- }
-
- }
-
- @Override
- public boolean parse(IRawRecord<? extends Writable> record, DataOutput out) throws HyracksDataException {
- try {
- Writable hiveRawRecord = record.get();
- Object hiveObject = hiveSerde.deserialize(hiveRawRecord);
- int n = recordType.getFieldNames().length;
- List<Object> attributesValues = oi.getStructFieldsDataAsList(hiveObject);
- recBuilder.reset(recordType);
- recBuilder.init();
- for (int i = 0; i < n; i++) {
- final Object value = attributesValues.get(i);
- final ObjectInspector foi = fieldRefs.get(i).getFieldObjectInspector();
- fieldValueBuffer.reset();
- final DataOutput dataOutput = fieldValueBuffer.getDataOutput();
- dataOutput.writeByte(fieldTypeTags[i]);
- //get field type
- parseItem(fieldTypes[i], value, foi, dataOutput, false);
- recBuilder.addField(i, fieldValueBuffer);
- }
- recBuilder.write(out, true);
- return true;
- } catch (Exception e) {
- throw HyracksDataException.create(e);
- }
- }
-
- private void parseItem(IAType itemType, Object value, ObjectInspector foi, DataOutput dataOutput,
- boolean primitiveOnly) throws HyracksDataException {
- try {
- switch (itemType.getTypeTag()) {
- case BOOLEAN:
- parseBoolean(value, (BooleanObjectInspector) foi, dataOutput);
- break;
- case TIME:
- parseTime(value, (TimestampObjectInspector) foi, dataOutput);
- break;
- case DATE:
- parseDate(value, (TimestampObjectInspector) foi, dataOutput);
- break;
- case DATETIME:
- parseDateTime(value, (TimestampObjectInspector) foi, dataOutput);
- break;
- case DOUBLE:
- parseDouble(value, (DoubleObjectInspector) foi, dataOutput);
- break;
- case FLOAT:
- parseFloat(value, (FloatObjectInspector) foi, dataOutput);
- break;
- case TINYINT:
- parseInt8(value, (ByteObjectInspector) foi, dataOutput);
- break;
- case SMALLINT:
- parseInt16(value, (ShortObjectInspector) foi, dataOutput);
- break;
- case INTEGER:
- parseInt32(value, (IntObjectInspector) foi, dataOutput);
- break;
- case BIGINT:
- parseInt64(value, (LongObjectInspector) foi, dataOutput);
- break;
- case STRING:
- parseString(value, (StringObjectInspector) foi, dataOutput);
- break;
- case ARRAY:
- if (primitiveOnly) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_NON_PRIMITIVE_LIST_NOT_SUPPORT);
- }
- parseOrderedList((AOrderedListType) itemType, value, (ListObjectInspector) foi);
- break;
- case MULTISET:
- if (primitiveOnly) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_NON_PRIMITIVE_LIST_NOT_SUPPORT);
- }
- parseUnorderedList((AUnorderedListType) itemType, value, (ListObjectInspector) foi);
- break;
- default:
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_FIELD_TYPE, itemType.getTypeTag());
- }
- } catch (IOException e) {
- throw HyracksDataException.create(e);
- }
- }
-
- private Object getColTypes(ARecordType record) throws HyracksDataException {
- int n = record.getFieldTypes().length;
- if (n < 1) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_GET_COLUMNS);
- }
- //First Column
- String cols = getHiveTypeString(record.getFieldTypes(), 0);
- for (int i = 1; i < n; i++) {
- cols = cols + "," + getHiveTypeString(record.getFieldTypes(), i);
- }
- return cols;
- }
-
- private String getCommaDelimitedColNames(ARecordType record) throws HyracksDataException {
- if (record.getFieldNames().length < 1) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_NO_CLOSED_COLUMNS);
- }
-
- String cols = record.getFieldNames()[0];
- for (int i = 1; i < record.getFieldNames().length; i++) {
- cols = cols + "," + record.getFieldNames()[i];
- }
- return cols;
- }
-
- private String getHiveTypeString(IAType[] types, int i) throws HyracksDataException {
- final IAType type = types[i];
- ATypeTag tag = type.getTypeTag();
- if (tag == ATypeTag.UNION) {
- if (NonTaggedFormatUtil.isOptional(type)) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_NOT_SUPPORT_NON_OP_UNION);
- }
- tag = ((AUnionType) type).getActualType().getTypeTag();
- }
- if (tag == null) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_MISSING_FIELD_TYPE_INFO, i);
- }
- switch (tag) {
- case BOOLEAN:
- return Constants.BOOLEAN_TYPE_NAME;
- case DATE:
- return Constants.DATE_TYPE_NAME;
- case DATETIME:
- return Constants.DATETIME_TYPE_NAME;
- case DOUBLE:
- return Constants.DOUBLE_TYPE_NAME;
- case FLOAT:
- return Constants.FLOAT_TYPE_NAME;
- case SMALLINT:
- return Constants.SMALLINT_TYPE_NAME;
- case INTEGER:
- return Constants.INT_TYPE_NAME;
- case BIGINT:
- return Constants.BIGINT_TYPE_NAME;
- case TINYINT:
- return Constants.TINYINT_TYPE_NAME;
- case ARRAY:
- return Constants.LIST_TYPE_NAME;
- case STRING:
- return Constants.STRING_TYPE_NAME;
- case TIME:
- return Constants.DATETIME_TYPE_NAME;
- case MULTISET:
- return Constants.LIST_TYPE_NAME;
- default:
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_FIELD_TYPE, tag);
- }
- }
-
- private void parseInt64(Object obj, LongObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeLong(foi.get(obj));
- }
-
- private void parseInt32(Object obj, IntObjectInspector foi, DataOutput dataOutput) throws IOException {
- if (obj == null) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_NULL_FIELD);
- }
- dataOutput.writeInt(foi.get(obj));
- }
-
- private void parseInt16(Object obj, ShortObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeShort(foi.get(obj));
- }
-
- private void parseFloat(Object obj, FloatObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeFloat(foi.get(obj));
- }
-
- private void parseDouble(Object obj, DoubleObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeDouble(foi.get(obj));
- }
-
- private void parseDateTime(Object obj, TimestampObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeLong(foi.getPrimitiveJavaObject(obj).getTime());
- }
-
- private void parseDate(Object obj, TimestampObjectInspector foi, DataOutput dataOutput) throws IOException {
- long chrononTimeInMs = foi.getPrimitiveJavaObject(obj).getTime();
- short temp = 0;
- if (chrononTimeInMs < 0 && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
- temp = 1;
- }
- dataOutput.writeInt((int) (chrononTimeInMs / GregorianCalendarSystem.CHRONON_OF_DAY) - temp);
- }
-
- private void parseBoolean(Object obj, BooleanObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeBoolean(foi.get(obj));
- }
-
- private void parseInt8(Object obj, ByteObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeByte(foi.get(obj));
- }
-
- private void parseString(Object obj, StringObjectInspector foi, DataOutput dataOutput) throws IOException {
- utf8Writer.writeUTF8(foi.getPrimitiveJavaObject(obj), dataOutput);
- }
-
- private void parseTime(Object obj, TimestampObjectInspector foi, DataOutput dataOutput) throws IOException {
- dataOutput.writeInt((int) (foi.getPrimitiveJavaObject(obj).getTime() % 86400000));
- }
-
- private void parseOrderedList(AOrderedListType aOrderedListType, Object obj, ListObjectInspector foi)
- throws HyracksDataException {
- OrderedListBuilder orderedListBuilder = getOrderedListBuilder();
- IAType itemType = null;
- if (aOrderedListType != null)
- itemType = aOrderedListType.getItemType();
- orderedListBuilder.reset(aOrderedListType);
-
- int n = foi.getListLength(obj);
- for (int i = 0; i < n; i++) {
- Object element = foi.getListElement(obj, i);
- ObjectInspector eoi = foi.getListElementObjectInspector();
- if (element == null) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_NULL_VALUE_IN_LIST);
- }
- parseItem(itemType, element, eoi, listItemBuffer.getDataOutput(), true);
- orderedListBuilder.addItem(listItemBuffer);
- }
- orderedListBuilder.write(fieldValueBuffer.getDataOutput(), true);
- }
-
- private void parseUnorderedList(AUnorderedListType uoltype, Object obj, ListObjectInspector oi) throws IOException {
- UnorderedListBuilder unorderedListBuilder = getUnorderedListBuilder();
- IAType itemType = null;
- if (uoltype != null)
- itemType = uoltype.getItemType();
- byte tagByte = itemType.getTypeTag().serialize();
- unorderedListBuilder.reset(uoltype);
-
- int n = oi.getListLength(obj);
- for (int i = 0; i < n; i++) {
- Object element = oi.getListElement(obj, i);
- ObjectInspector eoi = oi.getListElementObjectInspector();
- if (element == null) {
- throw new RuntimeDataException(ErrorCode.PARSER_HIVE_NULL_VALUE_IN_LIST);
- }
- listItemBuffer.reset();
- final DataOutput dataOutput = listItemBuffer.getDataOutput();
- dataOutput.writeByte(tagByte);
- parseItem(itemType, element, eoi, dataOutput, true);
- unorderedListBuilder.addItem(listItemBuffer);
- }
- unorderedListBuilder.write(fieldValueBuffer.getDataOutput(), true);
- }
-
- private OrderedListBuilder getOrderedListBuilder() {
- if (orderedListBuilder != null)
- return orderedListBuilder;
- else {
- orderedListBuilder = new OrderedListBuilder();
- return orderedListBuilder;
- }
- }
-
- private UnorderedListBuilder getUnorderedListBuilder() {
- if (unorderedListBuilder != null)
- return unorderedListBuilder;
- else {
- unorderedListBuilder = new UnorderedListBuilder();
- return unorderedListBuilder;
- }
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/LosslessADMJSONDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/LosslessADMJSONDataParser.java
new file mode 100644
index 0000000..79bca9c
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/LosslessADMJSONDataParser.java
@@ -0,0 +1,302 @@
+/*
+ * 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.external.parser;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ATaggedValuePrinter;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.util.bytes.HexParser;
+
+import com.fasterxml.jackson.core.JsonFactory;
+
+public final class LosslessADMJSONDataParser extends JSONDataParser {
+
+ private AMutablePoint[] polygonPoints;
+
+ public LosslessADMJSONDataParser(JsonFactory jsonFactory) {
+ super(RecordUtil.FULLY_OPEN_RECORD_TYPE, jsonFactory);
+ }
+
+ @Override
+ protected void serializeNumeric(ATypeTag numericType, DataOutput out) throws IOException {
+ super.serializeNumeric(ATypeTag.BIGINT, out);
+ }
+
+ @Override
+ protected void serializeString(ATypeTag stringVariantType, DataOutput out) throws IOException {
+ char[] textChars = jsonParser.getTextCharacters();
+ int textOffset = jsonParser.getTextOffset();
+ int textLength = jsonParser.getTextLength();
+
+ ATypeTag typeToUse = parseTypeTag(textChars, textOffset, textLength, aInt32);
+ if (typeToUse == null) {
+ throw new RuntimeDataException(ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM);
+ }
+ int parsedLength = aInt32.getIntegerValue();
+ int nonTaggedTextOffset = textOffset + parsedLength;
+ int nonTaggedTextLength = textLength - parsedLength;
+ switch (typeToUse) {
+ case MISSING:
+ missingSerde.serialize(AMissing.MISSING, out);
+ break;
+ case NULL:
+ nullSerde.serialize(ANull.NULL, out);
+ break;
+ case BOOLEAN:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ booleanSerde.serialize(ABoolean.valueOf(aInt64.getLongValue() != 0), out);
+ break;
+ case TINYINT:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aInt8.setValue((byte) aInt64.getLongValue());
+ int8Serde.serialize(aInt8, out);
+ break;
+ case SMALLINT:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aInt16.setValue((short) aInt64.getLongValue());
+ int16Serde.serialize(aInt16, out);
+ break;
+ case INTEGER:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aInt32.setValue((int) aInt64.getLongValue());
+ int32Serde.serialize(aInt32, out);
+ break;
+ case BIGINT:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ int64Serde.serialize(aInt64, out);
+ break;
+ case FLOAT:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aFloat.setValue(Float.intBitsToFloat((int) aInt64.getLongValue()));
+ floatSerde.serialize(aFloat, out);
+ break;
+ case DOUBLE:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aDouble.setValue(Double.longBitsToDouble(aInt64.getLongValue()));
+ doubleSerde.serialize(aDouble, out);
+ break;
+ case TIME:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aTime.setValue((int) aInt64.getLongValue());
+ timeSerde.serialize(aTime, out);
+ break;
+ case DATE:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aDate.setValue((int) aInt64.getLongValue());
+ dateSerde.serialize(aDate, out);
+ break;
+ case DATETIME:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aDateTime.setValue(aInt64.getLongValue());
+ datetimeSerde.serialize(aDateTime, out);
+ break;
+ case YEARMONTHDURATION:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aYearMonthDuration.setMonths((int) aInt64.getLongValue());
+ yearMonthDurationSerde.serialize(aYearMonthDuration, out);
+ break;
+ case DAYTIMEDURATION:
+ parseInt64(textChars, nonTaggedTextOffset, nonTaggedTextLength, aInt64);
+ aDayTimeDuration.setMilliseconds(aInt64.getLongValue());
+ dayTimeDurationSerde.serialize(aDayTimeDuration, out);
+ break;
+ case DURATION:
+ int delimIdx = findDelim(textChars, nonTaggedTextOffset, nonTaggedTextLength, BuiltinType.ADURATION);
+ parseInt64(textChars, nonTaggedTextOffset, delimIdx - nonTaggedTextOffset, aInt64);
+ int months = (int) aInt64.getLongValue();
+ parseInt64(textChars, delimIdx + 1, textLength - delimIdx - 1, aInt64);
+ long millis = aInt64.getLongValue();
+ aDuration.setValue(months, millis);
+ durationSerde.serialize(aDuration, out);
+ break;
+ case UUID:
+ aUUID.parseUUIDString(textChars, nonTaggedTextOffset, nonTaggedTextLength);
+ uuidSerde.serialize(aUUID, out);
+ break;
+ case STRING:
+ parseString(textChars, nonTaggedTextOffset, nonTaggedTextLength, out);
+ break;
+ case BINARY:
+ parseBase64BinaryString(textChars, nonTaggedTextOffset, nonTaggedTextLength, out);
+ break;
+ case POINT:
+ delimIdx = findDelim(textChars, nonTaggedTextOffset, nonTaggedTextLength, BuiltinType.APOINT);
+ parseInt64(textChars, nonTaggedTextOffset, delimIdx - nonTaggedTextOffset, aInt64);
+ double x = Double.longBitsToDouble(aInt64.getLongValue());
+ parseInt64(textChars, delimIdx + 1, textLength - delimIdx - 1, aInt64);
+ double y = Double.longBitsToDouble(aInt64.getLongValue());
+ aPoint.setValue(x, y);
+ pointSerde.serialize(aPoint, out);
+ break;
+ case POINT3D:
+ delimIdx = findDelim(textChars, nonTaggedTextOffset, nonTaggedTextLength, BuiltinType.APOINT3D);
+ parseInt64(textChars, nonTaggedTextOffset, delimIdx - nonTaggedTextOffset, aInt64);
+ x = Double.longBitsToDouble(aInt64.getLongValue());
+ int delimIdx2 = findDelim(textChars, delimIdx + 1, textLength - delimIdx - 1, BuiltinType.APOINT3D);
+ parseInt64(textChars, delimIdx + 1, delimIdx2 - delimIdx - 1, aInt64);
+ y = Double.longBitsToDouble(aInt64.getLongValue());
+ parseInt64(textChars, delimIdx2 + 1, textLength - delimIdx2 - 1, aInt64);
+ double z = Double.longBitsToDouble(aInt64.getLongValue());
+ aPoint3D.setValue(x, y, z);
+ point3DSerde.serialize(aPoint3D, out);
+ break;
+ case CIRCLE:
+ delimIdx = findDelim(textChars, nonTaggedTextOffset, nonTaggedTextLength, BuiltinType.ACIRCLE);
+ parseInt64(textChars, nonTaggedTextOffset, delimIdx - nonTaggedTextOffset, aInt64);
+ x = Double.longBitsToDouble(aInt64.getLongValue());
+ delimIdx2 = findDelim(textChars, delimIdx + 1, textLength - delimIdx - 1, BuiltinType.ACIRCLE);
+ parseInt64(textChars, delimIdx + 1, delimIdx2 - delimIdx - 1, aInt64);
+ y = Double.longBitsToDouble(aInt64.getLongValue());
+ parseInt64(textChars, delimIdx2 + 1, textLength - delimIdx2 - 1, aInt64);
+ z = Double.longBitsToDouble(aInt64.getLongValue());
+ aPoint.setValue(x, y);
+ aCircle.setValue(aPoint, z);
+ circleSerde.serialize(aCircle, out);
+ break;
+ case LINE:
+ delimIdx = findDelim(textChars, nonTaggedTextOffset, nonTaggedTextLength, BuiltinType.ALINE);
+ parseInt64(textChars, nonTaggedTextOffset, delimIdx - nonTaggedTextOffset, aInt64);
+ x = Double.longBitsToDouble(aInt64.getLongValue());
+ delimIdx2 = findDelim(textChars, delimIdx + 1, textLength - delimIdx - 1, BuiltinType.ALINE);
+ parseInt64(textChars, delimIdx + 1, delimIdx2 - delimIdx - 1, aInt64);
+ y = Double.longBitsToDouble(aInt64.getLongValue());
+ int delimIdx3 = findDelim(textChars, delimIdx2 + 1, textLength - delimIdx2 - 1, BuiltinType.ALINE);
+ parseInt64(textChars, delimIdx2 + 1, delimIdx3 - delimIdx2 - 1, aInt64);
+ double x2 = Double.longBitsToDouble(aInt64.getLongValue());
+ parseInt64(textChars, delimIdx3 + 1, textLength - delimIdx3 - 1, aInt64);
+ double y2 = Double.longBitsToDouble(aInt64.getLongValue());
+ aPoint.setValue(x, y);
+ aPoint2.setValue(x2, y2);
+ aLine.setValue(aPoint, aPoint2);
+ lineSerde.serialize(aLine, out);
+ break;
+ case RECTANGLE:
+ delimIdx = findDelim(textChars, nonTaggedTextOffset, nonTaggedTextLength, BuiltinType.ARECTANGLE);
+ parseInt64(textChars, nonTaggedTextOffset, delimIdx - nonTaggedTextOffset, aInt64);
+ x = Double.longBitsToDouble(aInt64.getLongValue());
+ delimIdx2 = findDelim(textChars, delimIdx + 1, textLength - delimIdx - 1, BuiltinType.ARECTANGLE);
+ parseInt64(textChars, delimIdx + 1, delimIdx2 - delimIdx - 1, aInt64);
+ y = Double.longBitsToDouble(aInt64.getLongValue());
+ delimIdx3 = findDelim(textChars, delimIdx2 + 1, textLength - delimIdx2 - 1, BuiltinType.ARECTANGLE);
+ parseInt64(textChars, delimIdx2 + 1, delimIdx3 - delimIdx2 - 1, aInt64);
+ x2 = Double.longBitsToDouble(aInt64.getLongValue());
+ parseInt64(textChars, delimIdx3 + 1, textLength - delimIdx3 - 1, aInt64);
+ y2 = Double.longBitsToDouble(aInt64.getLongValue());
+ aPoint.setValue(x, y);
+ aPoint2.setValue(x2, y2);
+ aRectangle.setValue(aPoint, aPoint2);
+ rectangleSerde.serialize(aRectangle, out);
+ break;
+ case POLYGON:
+ delimIdx = findDelim(textChars, nonTaggedTextOffset, nonTaggedTextLength, BuiltinType.APOLYGON);
+ parseInt64(textChars, nonTaggedTextOffset, delimIdx - nonTaggedTextOffset, aInt64);
+ int numPoints = (int) aInt64.getLongValue();
+ if (polygonPoints == null || polygonPoints.length != numPoints) {
+ polygonPoints = new AMutablePoint[numPoints];
+ for (int i = 0; i < numPoints; i++) {
+ polygonPoints[i] = new AMutablePoint(0, 0);
+ }
+ }
+ for (int i = 0; i < numPoints; i++) {
+ delimIdx2 = findDelim(textChars, delimIdx + 1, textLength - delimIdx - 1, BuiltinType.APOLYGON);
+ parseInt64(textChars, delimIdx + 1, delimIdx2 - delimIdx - 1, aInt64);
+ x = Double.longBitsToDouble(aInt64.getLongValue());
+ if (i < numPoints - 1) {
+ delimIdx3 =
+ findDelim(textChars, delimIdx2 + 1, textLength - delimIdx2 - 1, BuiltinType.APOLYGON);
+ parseInt64(textChars, delimIdx2 + 1, delimIdx3 - delimIdx2 - 1, aInt64);
+ delimIdx = delimIdx3;
+ } else {
+ parseInt64(textChars, delimIdx2 + 1, textLength - delimIdx2 - 1, aInt64);
+ }
+ y = Double.longBitsToDouble(aInt64.getLongValue());
+ polygonPoints[i].setValue(x, y);
+ }
+ aPolygon.setValue(polygonPoints);
+ polygonSerde.serialize(aPolygon, out);
+ break;
+ default:
+ throw new RuntimeDataException(ErrorCode.TYPE_UNSUPPORTED, "", typeToUse.toString());
+ }
+ }
+
+ private static ATypeTag parseTypeTag(char[] textChars, int textOffset, int textLength,
+ AMutableInt32 outParsedLength) {
+ if (textLength == 0) {
+ // empty string
+ outParsedLength.setValue(0);
+ return ATypeTag.STRING;
+ }
+ if (textChars[textOffset] == ATaggedValuePrinter.DELIMITER) {
+ // any string
+ outParsedLength.setValue(1);
+ return ATypeTag.STRING;
+ }
+ // any type
+ int typeTagLength = 2;
+ if (textLength < typeTagLength) {
+ return null;
+ }
+ byte typeTagByte;
+ try {
+ typeTagByte = HexParser.getByteFromValidHexChars(textChars[textOffset], textChars[textOffset + 1]);
+ } catch (IllegalArgumentException e) {
+ return null;
+ }
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagByte);
+ if (typeTag == null) {
+ return null;
+ }
+ if (typeTag == ATypeTag.MISSING || typeTag == ATypeTag.NULL) {
+ outParsedLength.setValue(typeTagLength);
+ return typeTag;
+ }
+ int delimiterLength = 1;
+ if (textLength < typeTagLength + delimiterLength) {
+ return null;
+ }
+ if (textChars[textOffset + typeTagLength] != ATaggedValuePrinter.DELIMITER) {
+ return null;
+ }
+ outParsedLength.setValue(typeTagLength + delimiterLength);
+ return typeTag;
+ }
+
+ private int findDelim(char[] text, int offset, int len, BuiltinType type) throws ParseException {
+ try {
+ return indexOf(text, offset, len, ATaggedValuePrinter.DELIMITER);
+ } catch (IllegalArgumentException e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(text, offset, len),
+ type.getTypeName());
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/RuntimeParserFunctionRegistrant.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/RuntimeParserFunctionRegistrant.java
new file mode 100644
index 0000000..a3fd0ff
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/RuntimeParserFunctionRegistrant.java
@@ -0,0 +1,31 @@
+/*
+ * 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.external.parser.evaluators;
+
+import org.apache.asterix.om.functions.IFunctionCollection;
+import org.apache.asterix.om.functions.IFunctionRegistrant;
+
+public class RuntimeParserFunctionRegistrant implements IFunctionRegistrant {
+ private static final long serialVersionUID = -2204964196487084743L;
+
+ @Override
+ public void register(IFunctionCollection collection) {
+ collection.add(StringJsonParseDescriptor.FACTORY);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/StringJsonParseDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/StringJsonParseDescriptor.java
new file mode 100644
index 0000000..4affafe
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/StringJsonParseDescriptor.java
@@ -0,0 +1,78 @@
+/*
+ * 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.external.parser.evaluators;
+
+import static org.apache.asterix.om.functions.BuiltinFunctions.STRING_PARSE_JSON;
+
+import java.io.IOException;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * the function parse-json() accepts a string and produces an ADM value
+ * Example:
+ * SELECT VALUE parse_json('[1,2]')
+ * <p>
+ * Output:
+ * [1, 2]
+ */
+@MissingNullInOutFunction
+public class StringJsonParseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1119187597484196172L;
+ public static final IFunctionDescriptorFactory FACTORY = StringJsonParseDescriptor::new;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return STRING_PARSE_JSON;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) throws AlgebricksException {
+ return new StringJsonParseEvalFactory(args[0], sourceLoc);
+ }
+
+ public static class StringJsonParseEvalFactory implements IScalarEvaluatorFactory {
+ private static final long serialVersionUID = 7976257476594454552L;
+ private final IScalarEvaluatorFactory stringEvalFactory;
+ private final SourceLocation sourceLocation;
+
+ public StringJsonParseEvalFactory(IScalarEvaluatorFactory stringEvalFactory, SourceLocation sourceLocation) {
+ this.stringEvalFactory = stringEvalFactory;
+ this.sourceLocation = sourceLocation;
+ }
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ try {
+ return new StringJsonParseEval(ctx, stringEvalFactory.createScalarEvaluator(ctx), sourceLocation);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/StringJsonParseEval.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/StringJsonParseEval.java
new file mode 100644
index 0000000..7589e33
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/evaluators/StringJsonParseEval.java
@@ -0,0 +1,114 @@
+/*
+ * 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.external.parser.evaluators;
+
+import static org.apache.asterix.om.functions.BuiltinFunctions.STRING_PARSE_JSON;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.external.parser.JSONDataParser;
+import org.apache.asterix.external.parser.factory.JSONDataParserFactory;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class StringJsonParseEval implements IScalarEvaluator {
+ private final IEvaluatorContext ctx;
+ private final IScalarEvaluator inputEval;
+ private final JSONDataParser parser;
+ private final SourceLocation sourceLocation;
+ private final IPointable inputVal;
+ private final UTF8StringPointable utf8Val;
+ private final ByteArrayAccessibleInputStream inputStream;
+ private final ArrayBackedValueStorage resultStorage;
+ private final DataOutput out;
+
+ public StringJsonParseEval(IEvaluatorContext ctx, IScalarEvaluator inputEval, SourceLocation sourceLocation)
+ throws IOException {
+ this.ctx = ctx;
+ this.inputEval = inputEval;
+ this.parser = (JSONDataParser) new JSONDataParserFactory().createInputStreamParser(ctx.getTaskContext(), 0);
+ this.sourceLocation = sourceLocation;
+ inputVal = new VoidPointable();
+ utf8Val = new UTF8StringPointable();
+ inputStream = new ByteArrayAccessibleInputStream(new byte[0], 0, 0);
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ parser.setInputStream(inputStream);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ inputEval.evaluate(tuple, inputVal);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, inputVal)) {
+ return;
+ }
+
+ byte[] bytes = inputVal.getByteArray();
+ int offset = inputVal.getStartOffset();
+ if (bytes[offset] == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ utf8Val.set(bytes, offset + 1, inputVal.getLength() - 1);
+ inputStream.setContent(bytes, utf8Val.getCharStartOffset(), utf8Val.getUTF8Length());
+ resultStorage.reset();
+ try {
+ if (parser.parseAnyValue(out)) {
+ result.set(resultStorage);
+ return;
+ } else {
+ //Reset the parser: EOF was encountered
+ resetParser();
+ }
+ } catch (HyracksDataException e) {
+ IWarningCollector warningCollector = ctx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(sourceLocation, ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM));
+ }
+ //Reset the parser: An error was encountered.
+ resetParser();
+ }
+ } else {
+ ExceptionUtil.warnTypeMismatch(ctx, sourceLocation, STRING_PARSE_JSON, bytes[offset], 0, ATypeTag.STRING);
+ }
+
+ PointableHelper.setNull(result);
+ }
+
+ private void resetParser() throws HyracksDataException {
+ try {
+ parser.reset(inputStream);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java
deleted file mode 100644
index d9cedea..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/HiveDataParserFactory.java
+++ /dev/null
@@ -1,83 +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.
- */
-package org.apache.asterix.external.parser.factory;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
-import org.apache.asterix.external.api.IRecordDataParser;
-import org.apache.asterix.external.api.IRecordDataParserFactory;
-import org.apache.asterix.external.parser.HiveRecordParser;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.external.util.HDFSUtils;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hadoop.io.Writable;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class HiveDataParserFactory implements IRecordDataParserFactory<Writable> {
-
- private static final long serialVersionUID = 1L;
- private static final List<String> parserFormats =
- Collections.unmodifiableList(Arrays.asList("hive", "hive-parser"));
- private Map<String, String> configuration;
- private ARecordType recordType;
- private String hiveSerdeClassName;
-
- @Override
- public DataSourceType getDataSourceType() {
- return DataSourceType.RECORDS;
- }
-
- @Override
- public void configure(Map<String, String> configuration) {
- this.configuration = configuration;
- hiveSerdeClassName = configuration.get(ExternalDataConstants.KEY_HIVE_SERDE);
- if (hiveSerdeClassName == null) {
- throw new IllegalArgumentException("no hive serde provided for hive deserialized records");
- }
- }
-
- @Override
- public void setRecordType(ARecordType recordType) {
- this.recordType = recordType;
- }
-
- @Override
- public IRecordDataParser<Writable> createRecordParser(IHyracksTaskContext ctx) throws HyracksDataException {
- return new HiveRecordParser(recordType, HDFSUtils.configureHDFSJobConf(configuration), hiveSerdeClassName);
- }
-
- @Override
- public Class<? extends Writable> getRecordClass() {
- return Writable.class;
- }
-
- @Override
- public void setMetaType(ARecordType metaType) {
- }
-
- @Override
- public List<String> getParserFormats() {
- return parserFormats;
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
index 8fa0aca..b0aded6 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
@@ -20,13 +20,16 @@
import java.io.IOException;
import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
import java.net.URL;
import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
@@ -46,7 +49,8 @@
public class DatasourceFactoryProvider {
private static final String RESOURCE = "META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory";
- private static Map<String, Class> factories = null;
+ private static final String DEFAULT_FORMAT = "DEFAULT_FORMAT";
+ private static Map<String, Map<String, Class<?>>> factories = null;
private DatasourceFactoryProvider() {
}
@@ -96,15 +100,16 @@
return streamSourceFactory;
}
- protected static IRecordReaderFactory getInstance(Class clazz) throws AsterixException {
+ protected static IRecordReaderFactory<?> getInstance(Class<?> clazz) throws AsterixException {
try {
- return (IRecordReaderFactory) clazz.newInstance();
- } catch (InstantiationException | IllegalAccessException | ClassCastException e) {
+ return (IRecordReaderFactory<?>) clazz.getDeclaredConstructor().newInstance();
+ } catch (InstantiationException | IllegalAccessException | ClassCastException | NoSuchMethodException
+ | InvocationTargetException e) {
throw new AsterixException("Cannot create: " + clazz.getSimpleName(), e);
}
}
- public static IRecordReaderFactory getRecordReaderFactory(String adaptorName, Map<String, String> configuration)
+ public static IRecordReaderFactory<?> getRecordReaderFactory(String adaptorName, Map<String, String> configuration)
throws HyracksDataException, AsterixException {
if (adaptorName.equals(ExternalDataConstants.EXTERNAL)) {
//return ExternalDataUtils.createExternalRecordReaderFactory(libraryManager, configuration);
@@ -112,24 +117,27 @@
}
if (factories == null) {
- factories = initFactories();
+ initFactories();
}
if (factories.containsKey(adaptorName)) {
- return getInstance(factories.get(adaptorName));
+ Map<String, Class<?>> formatClassMap = factories.get(adaptorName);
+ String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
+ return getInstance(formatClassMap.getOrDefault(format, formatClassMap.get(DEFAULT_FORMAT)));
}
try {
- return (IRecordReaderFactory) Class.forName(adaptorName).newInstance();
- } catch (IllegalAccessException | ClassNotFoundException | InstantiationException | ClassCastException e) {
+ return (IRecordReaderFactory<?>) Class.forName(adaptorName).getDeclaredConstructor().newInstance();
+ } catch (IllegalAccessException | ClassNotFoundException | InstantiationException | ClassCastException
+ | NoSuchMethodException | InvocationTargetException e) {
throw new RuntimeDataException(ErrorCode.UNKNOWN_RECORD_READER_FACTORY, e, adaptorName);
}
}
- protected static Map<String, Class> initFactories() throws AsterixException {
- Map<String, Class> factories = new HashMap<>();
+ protected static void initFactories() throws AsterixException {
+ factories = new HashMap<>();
ClassLoader cl = ParserFactoryProvider.class.getClassLoader();
- final Charset encoding = Charset.forName("UTF-8");
+ final Charset encoding = StandardCharsets.UTF_8;
try {
Enumeration<URL> urls = cl.getResources(RESOURCE);
for (URL url : Collections.list(urls)) {
@@ -142,19 +150,44 @@
continue;
}
final Class<?> clazz = Class.forName(className);
- List<String> formats = ((IRecordReaderFactory) clazz.newInstance()).getRecordReaderNames();
- for (String format : formats) {
- if (factories.containsKey(format)) {
- throw new AsterixException(ErrorCode.PROVIDER_DATASOURCE_FACTORY_DUPLICATE_FORMAT_MAPPING,
- format);
- }
- factories.put(format, clazz);
- }
+ final IRecordReaderFactory<?> readerFactory =
+ (IRecordReaderFactory<?>) clazz.getDeclaredConstructor().newInstance();
+ List<String> readerNames = readerFactory.getRecordReaderNames();
+ Set<String> supportedFormats = readerFactory.getReaderSupportedFormats();
+ putFactory(readerNames, supportedFormats, clazz);
}
}
- } catch (IOException | ClassNotFoundException | InstantiationException | IllegalAccessException e) {
+ } catch (IOException | ClassNotFoundException | InstantiationException | IllegalAccessException
+ | NoSuchMethodException | InvocationTargetException e) {
throw new AsterixException(e);
}
- return factories;
+ }
+
+ private static void putFactory(List<String> readerNames, Set<String> supportedFormats, Class<?> clazz)
+ throws AsterixException {
+ for (String reader : readerNames) {
+ Map<String, Class<?>> formatClassMap = factories.computeIfAbsent(reader, k -> new HashMap<>());
+ if (isDefaultFormat(supportedFormats)) {
+ //Ensure that only one reader is the default reader
+ checkDuplicates(formatClassMap, DEFAULT_FORMAT);
+ formatClassMap.put(DEFAULT_FORMAT, clazz);
+ } else {
+ //Specialized formats for the same reader name
+ for (String format : supportedFormats) {
+ checkDuplicates(formatClassMap, format);
+ formatClassMap.put(format, clazz);
+ }
+ }
+ }
+ }
+
+ private static boolean isDefaultFormat(Set<String> supportedFormats) {
+ return supportedFormats.equals(ExternalDataConstants.ALL_FORMATS);
+ }
+
+ private static void checkDuplicates(Map<String, Class<?>> factories, String key) throws AsterixException {
+ if (factories.containsKey(key)) {
+ throw new AsterixException(ErrorCode.PROVIDER_DATASOURCE_FACTORY_DUPLICATE_FORMAT_MAPPING, key);
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java
index 3c090a6..e24f433 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ExternalIndexerProvider.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.external.api.IExternalIndexer;
import org.apache.asterix.external.indexing.FileOffsetIndexer;
-import org.apache.asterix.external.indexing.RecordColumnarIndexer;
import org.apache.asterix.external.util.ExternalDataConstants;
public class ExternalIndexerProvider {
@@ -35,9 +34,6 @@
|| inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.INPUT_FORMAT_SEQUENCE)
|| inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)) {
return new FileOffsetIndexer();
- } else if (inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.INPUT_FORMAT_RC)
- || inputFormatParameter.equalsIgnoreCase(ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT)) {
- return new RecordColumnarIndexer();
} else {
throw new AsterixException("Unable to create indexer for data with format: " + inputFormatParameter);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
index db80c2a..db04479 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
@@ -35,8 +35,7 @@
Map<String, String> configuration, IWarningCollector warningCollector) throws AsterixException {
String inputFormat = HDFSUtils.getInputFormatClassName(configuration);
if (inputFormat.equals(ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT)
- || inputFormat.equals(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)
- || inputFormat.equals(ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT)) {
+ || inputFormat.equals(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)) {
HDFSLookupReaderFactory<Object> readerFactory = new HDFSLookupReaderFactory<>();
readerFactory.configure(serviceCtx, configuration, warningCollector);
return readerFactory;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index a0cf387..89d1132 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -60,6 +60,12 @@
public static final String KEY_HADOOP_SHORT_CIRCUIT = "dfs.client.read.shortcircuit";
public static final String KEY_HADOOP_SOCKET_PATH = "dfs.domain.socket.path";
public static final String KEY_HADOOP_BUFFER_SIZE = "io.file.buffer.size";
+ //Base64 encoded warnings issued from Hadoop
+ public static final String KEY_HADOOP_ASTERIX_WARNINGS_LIST = "org.apache.asterix.warnings.list";
+ //Disable caching FileSystem for Hadoop
+ public static final String KEY_HADOOP_DISABLE_FS_CACHE_TEMPLATE = "fs.%s.impl.disable.cache";
+ //Base64 encoded function call information
+ public static final String KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION = "org.apache.asterix.function.info";
public static final String KEY_SOURCE_DATATYPE = "type-name";
public static final String KEY_DELIMITER = "delimiter";
public static final String KEY_PARSER_FACTORY = "parser-factory";
@@ -79,7 +85,6 @@
public static final String KEY_ESCAPE = "escape";
public static final String KEY_PARSER = "parser";
public static final String KEY_DATASET_RECORD = "dataset-record";
- public static final String KEY_HIVE_SERDE = "hive-serde";
public static final String KEY_RSS_URL = "url";
public static final String KEY_INTERVAL = "interval";
public static final String KEY_IS_FEED = "is-feed";
@@ -136,13 +141,14 @@
public static final String KEY_ADAPTER_NAME_HTTP = "http_adapter";
public static final String KEY_ADAPTER_NAME_AWS_S3 = "S3";
public static final String KEY_ADAPTER_NAME_AZURE_BLOB = "AZUREBLOB";
+ public static final String KEY_ADAPTER_NAME_AZURE_DATA_LAKE = "AZUREDATALAKE";
+ public static final String KEY_ADAPTER_NAME_GCS = "GCS";
/**
* HDFS class names
*/
public static final String CLASS_NAME_TEXT_INPUT_FORMAT = "org.apache.hadoop.mapred.TextInputFormat";
public static final String CLASS_NAME_SEQUENCE_INPUT_FORMAT = "org.apache.hadoop.mapred.SequenceFileInputFormat";
- public static final String CLASS_NAME_RC_INPUT_FORMAT = "org.apache.asterix.hivecompat.io.RCFileInputFormat";
public static final String CLASS_NAME_PARQUET_INPUT_FORMAT =
"org.apache.asterix.external.input.record.reader.hdfs.parquet.MapredParquetInputFormat";
public static final String CLASS_NAME_HDFS_FILESYSTEM = "org.apache.hadoop.hdfs.DistributedFileSystem";
@@ -151,7 +157,6 @@
*/
public static final String INPUT_FORMAT_TEXT = "text-input-format";
public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
- public static final String INPUT_FORMAT_RC = "rc-input-format";
public static final String INPUT_FORMAT_PARQUET = "parquet-input-format";
/**
* Builtin streams
@@ -164,7 +169,6 @@
public static final String CLUSTER_LOCATIONS = "cluster-locations";
public static final String SCHEDULER = "hdfs-scheduler";
- public static final String PARSER_HIVE = "hive-parser";
public static final String HAS_HEADER = "has.header";
public static final String TIME_TRACKING = "time.tracking";
public static final String DEFAULT_QUOTE = "\"";
@@ -175,7 +179,6 @@
/**
* supported builtin record formats
*/
- public static final String FORMAT_HIVE = "hive";
public static final String FORMAT_BINARY = "binary";
public static final String FORMAT_ADM = "adm";
public static final String FORMAT_JSON_LOWER_CASE = "json";
@@ -190,11 +193,11 @@
public static final String FORMAT_KV = "kv";
public static final String FORMAT_CSV = "csv";
public static final String FORMAT_TSV = "tsv";
+ public static final String FORMAT_PARQUET = "parquet";
public static final Set<String> ALL_FORMATS;
static {
- Set<String> formats = new HashSet<>(13);
- formats.add(FORMAT_HIVE);
+ Set<String> formats = new HashSet<>(14);
formats.add(FORMAT_BINARY);
formats.add(FORMAT_ADM);
formats.add(FORMAT_JSON_LOWER_CASE);
@@ -207,6 +210,7 @@
formats.add(FORMAT_KV);
formats.add(FORMAT_CSV);
formats.add(FORMAT_TSV);
+ formats.add(FORMAT_PARQUET);
ALL_FORMATS = Collections.unmodifiableSet(formats);
}
@@ -251,6 +255,7 @@
public static final char OPEN_BRACKET = '[';
public static final char CLOSING_BRACKET = ']';
public static final char COMMA = ',';
+ public static final char BYTE_ORDER_MARK = '\uFEFF';
/**
* Constant byte characters
@@ -310,26 +315,87 @@
public static boolean isRetryableError(String errorCode) {
return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
}
+
+ /*
+ * Hadoop-AWS
+ * AWS connectors for s3 and s3n are deprecated.
+ */
+ public static final String HADOOP_ACCESS_KEY_ID = "fs.s3a.access.key";
+ public static final String HADOOP_SECRET_ACCESS_KEY = "fs.s3a.secret.key";
+ public static final String HADOOP_SESSION_TOKEN = "fs.s3a.session.token";
+ public static final String HADOOP_REGION = "fs.s3a.region";
+ public static final String HADOOP_SERVICE_END_POINT = "fs.s3a.endpoint";
+
+ /*
+ * Internal configurations
+ */
+ //Allows accessing directories as file system path
+ public static final String HADOOP_PATH_STYLE_ACCESS = "fs.s3a.path.style.access";
+ //The number of maximum HTTP connections in connection pool
+ public static final String HADOOP_S3_CONNECTION_POOL_SIZE = "fs.s3a.connection.maximum";
+ //S3 used protocol
+ public static final String HADOOP_S3_PROTOCOL = "s3a";
+
+ //Hadoop credentials provider key
+ public static final String HADOOP_CREDENTIAL_PROVIDER_KEY = "fs.s3a.aws.credentials.provider";
+ //Anonymous credential provider
+ public static final String HADOOP_ANONYMOUS_ACCESS = "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider";
+ //Temporary credential provider
+ public static final String HADOOP_TEMP_ACCESS = "org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider";
+
}
- public static class AzureBlob {
- private AzureBlob() {
+ /*
+ * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
+ * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
+ */
+ public static class Azure {
+ private Azure() {
throw new AssertionError("do not instantiate");
}
- public static final String CONNECTION_STRING_FIELD_NAME = "connectionString";
+ /*
+ * Asterix Configuration Keys
+ */
+ public static final String MANAGED_IDENTITY_ID_FIELD_NAME = "managedIdentityId";
public static final String ACCOUNT_NAME_FIELD_NAME = "accountName";
public static final String ACCOUNT_KEY_FIELD_NAME = "accountKey";
public static final String SHARED_ACCESS_SIGNATURE_FIELD_NAME = "sharedAccessSignature";
- public static final String BLOB_ENDPOINT_FIELD_NAME = "blobEndpoint";
- public static final String ENDPOINT_SUFFIX_FIELD_NAME = "endpointSuffix";
+ public static final String TENANT_ID_FIELD_NAME = "tenantId";
+ public static final String CLIENT_ID_FIELD_NAME = "clientId";
+ public static final String CLIENT_SECRET_FIELD_NAME = "clientSecret";
+ public static final String CLIENT_CERTIFICATE_FIELD_NAME = "clientCertificate";
+ public static final String CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME = "clientCertificatePassword";
+ public static final String ENDPOINT_FIELD_NAME = "endpoint";
- // Connection string requires PascalCase (MyFieldFormat)
- public static final String CONNECTION_STRING_CONNECTION_STRING = "ConnectionString";
- public static final String CONNECTION_STRING_ACCOUNT_NAME = "AccountName";
- public static final String CONNECTION_STRING_ACCOUNT_KEY = "AccountKey";
- public static final String CONNECTION_STRING_SHARED_ACCESS_SIGNATURE = "SharedAccessSignature";
- public static final String CONNECTION_STRING_BLOB_ENDPOINT = "BlobEndpoint";
- public static final String CONNECTION_STRING_ENDPOINT_SUFFIX = "EndpointSuffix";
+ // Specific Azure data lake property
+ /*
+ The behavior of Data Lake (true file system) is to read the files of the specified prefix only, example:
+ storage/myData/personal/file1.json
+ storage/myData/personal/file2.json
+ storage/myData/file3.json
+ If the prefix used is "myData", then only the file file3.json is read. However, if the property "recursive"
+ is set to "true" when creating the external dataset, then it goes recursively overall the paths, and the result
+ is file1.json, file2.json and file3.json.
+ */
+ public static final String RECURSIVE_FIELD_NAME = "recursive";
+
+ /*
+ * Hadoop-Azure
+ */
+ //Used when accountName and accessKey are provided
+ public static final String HADOOP_AZURE_FS_ACCOUNT_KEY = "fs.azure.account.key";
+ //Used when a connectionString is provided
+ public static final String HADOOP_AZURE_FS_SAS = "fs.azure.sas";
+ public static final String HADOOP_AZURE_BLOB_PROTOCOL = "wasbs";
+ public static final String HADOOP_AZURE_DATALAKE_PROTOCOL = "abfss";
+ }
+
+ public static class GCS {
+ private GCS() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ public static final String JSON_CREDENTIALS_FIELD_NAME = "jsonCredentials";
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 8e94263..b38f21d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -18,37 +18,77 @@
*/
package org.apache.asterix.external.util;
+import static com.google.cloud.storage.Storage.BlobListOption;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_SOURCE_ERROR;
+import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAMETERS_REQUIRED;
+import static org.apache.asterix.common.exceptions.ErrorCode.PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT;
import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.common.exceptions.ErrorCode.S3_REGION_NOT_SUPPORTED;
import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ACCESS_KEY_ID;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_ANONYMOUS_ACCESS;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_CREDENTIAL_PROVIDER_KEY;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_PATH_STYLE_ACCESS;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_CONNECTION_POOL_SIZE;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_S3_PROTOCOL;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SECRET_ACCESS_KEY;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_SESSION_TOKEN;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.HADOOP_TEMP_ACCESS;
import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.ACCOUNT_KEY_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.ACCOUNT_NAME_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.BLOB_ENDPOINT_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.CONNECTION_STRING_ACCOUNT_KEY;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.CONNECTION_STRING_ACCOUNT_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.CONNECTION_STRING_BLOB_ENDPOINT;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.CONNECTION_STRING_ENDPOINT_SUFFIX;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.CONNECTION_STRING_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.CONNECTION_STRING_SHARED_ACCESS_SIGNATURE;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.ENDPOINT_SUFFIX_FIELD_NAME;
-import static org.apache.asterix.external.util.ExternalDataConstants.AzureBlob.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ACCOUNT_NAME_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.CLIENT_SECRET_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.ENDPOINT_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_BLOB_PROTOCOL;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_ACCOUNT_KEY;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.HADOOP_AZURE_FS_SAS;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.MANAGED_IDENTITY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.RECURSIVE_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.SHARED_ACCESS_SIGNATURE_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.Azure.TENANT_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.GCS.JSON_CREDENTIALS_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ADAPTER_NAME_GCS;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXCLUDE;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_FORMAT;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_INCLUDE;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
+import static org.apache.hyracks.api.util.ExceptionUtils.getMessageOrToString;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
import java.util.EnumMap;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiPredicate;
import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.CompilationException;
@@ -62,11 +102,16 @@
import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
import org.apache.asterix.external.api.IInputStreamFactory;
import org.apache.asterix.external.api.IRecordReaderFactory;
+import org.apache.asterix.external.input.record.reader.abstracts.AbstractExternalInputStreamFactory.IncludeExcludeMatcher;
import org.apache.asterix.external.library.JavaLibrary;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
+import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hadoop.fs.s3a.Constants;
+import org.apache.hadoop.mapred.JobConf;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.IWarningCollector;
@@ -82,11 +127,27 @@
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import org.apache.hyracks.util.StorageUtil;
+import com.azure.core.credential.AzureSasCredential;
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.identity.ClientCertificateCredentialBuilder;
+import com.azure.identity.ClientSecretCredentialBuilder;
+import com.azure.identity.ManagedIdentityCredentialBuilder;
import com.azure.storage.blob.BlobContainerClient;
import com.azure.storage.blob.BlobServiceClient;
import com.azure.storage.blob.BlobServiceClientBuilder;
import com.azure.storage.blob.models.BlobItem;
import com.azure.storage.blob.models.ListBlobsOptions;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import com.azure.storage.file.datalake.models.PathItem;
+import com.google.api.gax.paging.Page;
+import com.google.auth.oauth2.ServiceAccountCredentials;
+import com.google.cloud.storage.Blob;
+import com.google.cloud.storage.Storage;
+import com.google.cloud.storage.StorageOptions;
import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
@@ -102,10 +163,10 @@
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Object;
import software.amazon.awssdk.services.s3.model.S3Response;
public class ExternalDataUtils {
-
private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
private static final int DEFAULT_MAX_ARGUMENT_SZ = 1024 * 1024;
private static final int HEADER_FUDGE = 64;
@@ -436,7 +497,8 @@
final String inputFormat = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT);
if (ExternalDataConstants.INPUT_FORMAT_PARQUET.equals(inputFormat)) {
//Parquet supports binary-to-binary conversion. No parsing is required
- configuration.put(ExternalDataConstants.KEY_FORMAT, ExternalDataConstants.FORMAT_NOOP);
+ configuration.put(ExternalDataConstants.KEY_PARSER, ExternalDataConstants.FORMAT_NOOP);
+ configuration.put(ExternalDataConstants.KEY_FORMAT, ExternalDataConstants.FORMAT_PARQUET);
}
if (!configuration.containsKey(ExternalDataConstants.KEY_PARSER)
&& configuration.containsKey(ExternalDataConstants.KEY_FORMAT)) {
@@ -542,7 +604,13 @@
AwsS3.validateProperties(configuration, srcLoc, collector);
break;
case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_BLOB:
- ExternalDataUtils.Azure.validateProperties(configuration, srcLoc, collector);
+ Azure.validateAzureBlobProperties(configuration, srcLoc, collector);
+ break;
+ case ExternalDataConstants.KEY_ADAPTER_NAME_AZURE_DATA_LAKE:
+ Azure.validateAzureDataLakeProperties(configuration, srcLoc, collector);
+ break;
+ case KEY_ADAPTER_NAME_GCS:
+ GCS.validateProperties(configuration, srcLoc, collector);
break;
default:
// Nothing needs to be done
@@ -660,9 +728,13 @@
* @param configuration configuration
*/
public static String getPrefix(Map<String, String> configuration) {
+ return getPrefix(configuration, true);
+ }
+
+ public static String getPrefix(Map<String, String> configuration, boolean appendSlash) {
String definition = configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
if (definition != null && !definition.isEmpty()) {
- return definition + (!definition.endsWith("/") ? "/" : "");
+ return appendSlash ? definition + (!definition.endsWith("/") ? "/" : "") : definition;
}
return "";
}
@@ -711,12 +783,105 @@
}
}
+ public static IncludeExcludeMatcher getIncludeExcludeMatchers(Map<String, String> configuration)
+ throws CompilationException {
+ // Get and compile the patterns for include/exclude if provided
+ List<Matcher> includeMatchers = new ArrayList<>();
+ List<Matcher> excludeMatchers = new ArrayList<>();
+ String pattern = null;
+ try {
+ for (Map.Entry<String, String> entry : configuration.entrySet()) {
+ if (entry.getKey().startsWith(KEY_INCLUDE)) {
+ pattern = entry.getValue();
+ includeMatchers.add(Pattern.compile(patternToRegex(pattern)).matcher(""));
+ } else if (entry.getKey().startsWith(KEY_EXCLUDE)) {
+ pattern = entry.getValue();
+ excludeMatchers.add(Pattern.compile(patternToRegex(pattern)).matcher(""));
+ }
+ }
+ } catch (PatternSyntaxException ex) {
+ throw new CompilationException(ErrorCode.INVALID_REGEX_PATTERN, pattern);
+ }
+
+ IncludeExcludeMatcher includeExcludeMatcher;
+ if (!includeMatchers.isEmpty()) {
+ includeExcludeMatcher =
+ new IncludeExcludeMatcher(includeMatchers, (matchers1, key) -> matchPatterns(matchers1, key));
+ } else if (!excludeMatchers.isEmpty()) {
+ includeExcludeMatcher =
+ new IncludeExcludeMatcher(excludeMatchers, (matchers1, key) -> !matchPatterns(matchers1, key));
+ } else {
+ includeExcludeMatcher = new IncludeExcludeMatcher(Collections.emptyList(), (matchers1, key) -> true);
+ }
+
+ return includeExcludeMatcher;
+ }
+
public static boolean supportsPushdown(Map<String, String> properties) {
//Currently, only Apache Parquet format is supported
- return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT
- .equals(properties.get(ExternalDataConstants.KEY_INPUT_FORMAT))
- || ExternalDataConstants.INPUT_FORMAT_PARQUET
- .equals(properties.get(ExternalDataConstants.KEY_INPUT_FORMAT));
+ return isParquetFormat(properties);
+ }
+
+ /**
+ * Validate the dataset type declared with a given type
+ *
+ * @param properties external dataset configuration
+ * @param datasetRecordType dataset declared type
+ */
+ public static void validateType(Map<String, String> properties, ARecordType datasetRecordType)
+ throws CompilationException {
+ if (isParquetFormat(properties) && datasetRecordType.getFieldTypes().length != 0) {
+ throw new CompilationException(ErrorCode.UNSUPPORTED_TYPE_FOR_PARQUET, datasetRecordType.getTypeName());
+ }
+ }
+
+ private static boolean isParquetFormat(Map<String, String> properties) {
+ String inputFormat = properties.get(ExternalDataConstants.KEY_INPUT_FORMAT);
+ return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(inputFormat)
+ || ExternalDataConstants.INPUT_FORMAT_PARQUET.equals(inputFormat)
+ || ExternalDataConstants.FORMAT_PARQUET.equals(properties.get(ExternalDataConstants.KEY_FORMAT));
+ }
+
+ public static void setExternalDataProjectionInfo(DataProjectionInfo projectionInfo, Map<String, String> properties)
+ throws IOException {
+ properties.put(ExternalDataConstants.KEY_REQUESTED_FIELDS,
+ serializeExpectedTypeToString(projectionInfo.getProjectionInfo()));
+ properties.put(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION,
+ serializeFunctionCallInfoToString(projectionInfo.getFunctionCallInfoMap()));
+ }
+
+ /**
+ * Serialize {@link ARecordType} as Base64 string to pass it to {@link org.apache.hadoop.conf.Configuration}
+ *
+ * @param expectedType expected type
+ * @return the expected type as Base64 string
+ */
+ private static String serializeExpectedTypeToString(ARecordType expectedType) throws IOException {
+ if (expectedType == DataProjectionInfo.EMPTY_TYPE || expectedType == DataProjectionInfo.ALL_FIELDS_TYPE) {
+ //Return the type name of EMPTY_TYPE and ALL_FIELDS_TYPE
+ return expectedType.getTypeName();
+ }
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+ Base64.Encoder encoder = Base64.getEncoder();
+ DataProjectionInfo.writeTypeField(expectedType, dataOutputStream);
+ return encoder.encodeToString(byteArrayOutputStream.toByteArray());
+ }
+
+ /**
+ * Serialize {@link FunctionCallInformation} map as Base64 string to pass it to
+ * {@link org.apache.hadoop.conf.Configuration}
+ *
+ * @param functionCallInfoMap function information map
+ * @return function information map as Base64 string
+ */
+ static String serializeFunctionCallInfoToString(Map<String, FunctionCallInformation> functionCallInfoMap)
+ throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+ Base64.Encoder encoder = Base64.getEncoder();
+ DataProjectionInfo.writeFunctionCallInformationMapField(functionCallInfoMap, dataOutputStream);
+ return encoder.encodeToString(byteArrayOutputStream.toByteArray());
}
public static class AwsS3 {
@@ -759,7 +924,16 @@
}
builder.credentialsProvider(credentialsProvider);
- builder.region(Region.of(regionId));
+
+ // Validate the region
+ List<Region> regions = S3Client.serviceMetadata().regions();
+ Optional<Region> selectedRegion =
+ regions.stream().filter(region -> region.id().equals(regionId)).findFirst();
+
+ if (selectedRegion.isEmpty()) {
+ throw new CompilationException(S3_REGION_NOT_SUPPORTED, regionId);
+ }
+ builder.region(selectedRegion.get());
// Validate the service endpoint if present
if (serviceEndpoint != null) {
@@ -768,7 +942,7 @@
try {
builder.endpointOverride(uri);
} catch (NullPointerException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
}
} catch (URISyntaxException ex) {
throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
@@ -780,6 +954,61 @@
}
/**
+ * Builds the S3 client using the provided configuration
+ *
+ * @param configuration properties
+ * @param numberOfPartitions number of partitions in the cluster
+ */
+ public static void configureAwsS3HdfsJobConf(JobConf conf, Map<String, String> configuration,
+ int numberOfPartitions) {
+ String accessKeyId = configuration.get(ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME);
+ String sessionToken = configuration.get(ExternalDataConstants.AwsS3.SESSION_TOKEN_FIELD_NAME);
+ String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
+
+ //Disable caching S3 FileSystem
+ HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_S3_PROTOCOL);
+
+ /*
+ * Authentication Methods:
+ * 1- Anonymous: no accessKeyId and no secretAccessKey
+ * 2- Temporary: has to provide accessKeyId, secretAccessKey and sessionToken
+ * 3- Private: has to provide accessKeyId and secretAccessKey
+ */
+ if (accessKeyId == null) {
+ //Tells hadoop-aws it is an anonymous access
+ conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_ANONYMOUS_ACCESS);
+ } else {
+ conf.set(HADOOP_ACCESS_KEY_ID, accessKeyId);
+ conf.set(HADOOP_SECRET_ACCESS_KEY, secretAccessKey);
+ if (sessionToken != null) {
+ conf.set(HADOOP_SESSION_TOKEN, sessionToken);
+ //Tells hadoop-aws it is a temporary access
+ conf.set(HADOOP_CREDENTIAL_PROVIDER_KEY, HADOOP_TEMP_ACCESS);
+ }
+ }
+
+ /*
+ * This is to allow S3 definition to have path-style form. Should always be true to match the current
+ * way we access files in S3
+ */
+ conf.set(HADOOP_PATH_STYLE_ACCESS, ExternalDataConstants.TRUE);
+
+ /*
+ * Set the size of S3 connection pool to be the number of partitions
+ */
+ conf.set(HADOOP_S3_CONNECTION_POOL_SIZE, String.valueOf(numberOfPartitions));
+
+ if (serviceEndpoint != null) {
+ // Validation of the URL should be done at hadoop-aws level
+ conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, serviceEndpoint);
+ } else {
+ //Region is ignored and buckets could be found by the central endpoint
+ conf.set(ExternalDataConstants.AwsS3.HADOOP_SERVICE_END_POINT, Constants.CENTRAL_ENDPOINT);
+ }
+ }
+
+ /**
* Validate external dataset properties
*
* @param configuration properties
@@ -829,10 +1058,10 @@
throw ex;
}
} catch (SdkException ex2) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex2.getMessage());
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
}
} catch (SdkException ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
} finally {
if (s3Client != null) {
CleanupUtils.close(s3Client, null);
@@ -856,11 +1085,10 @@
/**
* Checks for a single object in the specified bucket to determine if the bucket is empty or not.
*
- * @param s3Client s3 client
+ * @param s3Client s3 client
* @param container the container name
- * @param prefix Prefix to be used
+ * @param prefix Prefix to be used
* @param useOldApi flag whether to use the old API or not
- *
* @return returns the S3 response
*/
private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
@@ -876,8 +1104,161 @@
}
return response;
}
+
+ /**
+ * Returns the lists of S3 objects.
+ *
+ * @param configuration properties
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ public static List<S3Object> listS3Objects(Map<String, String> configuration,
+ IncludeExcludeMatcher includeExcludeMatcher, IWarningCollector warningCollector)
+ throws CompilationException {
+ // Prepare to retrieve the objects
+ List<S3Object> filesOnly;
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ S3Client s3Client = buildAwsS3Client(configuration);
+ String prefix = getPrefix(configuration);
+
+ try {
+ filesOnly = listS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+ } catch (S3Exception ex) {
+ // New API is not implemented, try falling back to old API
+ try {
+ // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+ if (ex.awsErrorDetails().errorCode()
+ .equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
+ filesOnly = oldApiListS3Objects(s3Client, container, prefix, includeExcludeMatcher);
+ } else {
+ throw ex;
+ }
+ } catch (SdkException ex2) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ } catch (SdkException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ } finally {
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Uses the latest API to retrieve the objects from the storage.
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param prefix definition prefix
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ private static List<S3Object> listS3Objects(S3Client s3Client, String container, String prefix,
+ IncludeExcludeMatcher includeExcludeMatcher) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsV2Response listObjectsResponse;
+ ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
+ listObjectsBuilder.prefix(prefix);
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse =
+ s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextContinuationToken();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param prefix definition prefix
+ * @param includeExcludeMatcher include/exclude matchers to apply
+ */
+ private static List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, String prefix,
+ IncludeExcludeMatcher includeExcludeMatcher) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsResponse listObjectsResponse;
+ ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
+ listObjectsBuilder.prefix(prefix);
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextMarker();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
+ * a file if it does not end up with a "/" which is the separator in a folder structure.
+ *
+ * @param s3Objects List of returned objects
+ */
+ private static void collectAndFilterFiles(List<S3Object> s3Objects,
+ BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<S3Object> filesOnly) {
+ for (S3Object object : s3Objects) {
+ // skip folders
+ if (object.key().endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, object.key())) {
+ filesOnly.add(object);
+ }
+ }
+ }
}
+ /*
+ * Note: Azure Blob and Azure Datalake use identical authentication, so they are using the same properties.
+ * If they end up diverging, then properties for AzureBlob and AzureDataLake need to be created.
+ */
public static class Azure {
private Azure() {
throw new AssertionError("do not instantiate");
@@ -889,79 +1270,575 @@
* @param configuration properties
* @return client
*/
- public static BlobServiceClient buildAzureClient(Map<String, String> configuration)
+ public static BlobServiceClient buildAzureBlobClient(Map<String, String> configuration)
throws CompilationException {
- // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
- String connectionString = configuration.get(CONNECTION_STRING_FIELD_NAME);
+ String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
- String blobEndpoint = configuration.get(BLOB_ENDPOINT_FIELD_NAME);
- String endpointSuffix = configuration.get(ENDPOINT_SUFFIX_FIELD_NAME);
+ String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+ String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+ String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+ String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+ String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
- // Constructor the connection string
- // Connection string format: name1=value1;name2=value2;....
- StringBuilder connectionStringBuilder = new StringBuilder();
+ // Client builder
BlobServiceClientBuilder builder = new BlobServiceClientBuilder();
- boolean authMethodFound = false;
-
- if (connectionString != null) {
- // connection string
- authMethodFound = true;
- connectionStringBuilder.append(connectionString).append(";");
+ // Endpoint is required
+ if (endpoint == null) {
+ throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
}
+ builder.endpoint(endpoint);
- if (accountName != null && accountKey != null) {
- if (authMethodFound) {
- throw new CompilationException(ErrorCode.ONLY_SINGLE_AUTHENTICATION_IS_ALLOWED);
+ // Shared Key
+ if (accountName != null || accountKey != null) {
+ if (accountName == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+ ACCOUNT_KEY_FIELD_NAME);
}
- authMethodFound = true;
- // account name + account key
- connectionStringBuilder.append(CONNECTION_STRING_ACCOUNT_NAME).append("=").append(accountName)
- .append(";").append(CONNECTION_STRING_ACCOUNT_KEY).append("=").append(accountKey).append(";");
- }
- if (accountName != null && sharedAccessSignature != null) {
- if (authMethodFound) {
- throw new CompilationException(ErrorCode.ONLY_SINGLE_AUTHENTICATION_IS_ALLOWED);
+ if (accountKey == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+ ACCOUNT_NAME_FIELD_NAME);
}
- authMethodFound = true;
- // account name + shared access token
- connectionStringBuilder.append(CONNECTION_STRING_ACCOUNT_NAME).append("=").append(accountName)
- .append(";").append(CONNECTION_STRING_SHARED_ACCESS_SIGNATURE).append("=")
- .append(sharedAccessSignature).append(";");
+
+ Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+ MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+ StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+ builder.credential(credential);
}
- if (!authMethodFound) {
- throw new CompilationException(ErrorCode.NO_AUTH_METHOD_PROVIDED);
+ // Shared access signature
+ if (sharedAccessSignature != null) {
+ Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+ builder.credential(credential);
}
- // Add blobEndpoint and endpointSuffix if present, adjust any '/' as needed
- if (blobEndpoint != null) {
- connectionStringBuilder.append(CONNECTION_STRING_BLOB_ENDPOINT).append("=").append(blobEndpoint)
- .append(";");
- if (endpointSuffix != null) {
- String endpointSuffixUpdated;
- if (blobEndpoint.endsWith("/")) {
- endpointSuffixUpdated =
- endpointSuffix.startsWith("/") ? endpointSuffix.substring(1) : endpointSuffix;
+ // Managed Identity auth
+ if (managedIdentityId != null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
+ TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ MANAGED_IDENTITY_ID_FIELD_NAME);
+ }
+ builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+ }
+
+ // Client secret & certificate auth
+ if (clientId != null) {
+ // Both (or neither) client secret and client secret were provided, only one is allowed
+ if ((clientSecret == null) == (clientCertificate == null)) {
+ if (clientSecret != null) {
+ throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME);
} else {
- endpointSuffixUpdated = endpointSuffix.startsWith("/") ? endpointSuffix : "/" + endpointSuffix;
+ throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
}
- connectionStringBuilder.append(CONNECTION_STRING_ENDPOINT_SUFFIX).append("=")
- .append(endpointSuffixUpdated).append(";");
+ }
+
+ // Tenant ID is required
+ if (tenantId == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME);
+ }
+
+ // Client certificate password is not allowed if client secret is used
+ if (clientCertificatePassword != null && clientSecret != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+ }
+
+ // Use AD authentication
+ if (clientSecret != null) {
+ ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+ secret.clientId(clientId);
+ secret.tenantId(tenantId);
+ secret.clientSecret(clientSecret);
+ builder.credential(secret.build());
+ } else {
+ // Certificate
+ ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+ certificate.clientId(clientId);
+ certificate.tenantId(tenantId);
+ try {
+ InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+ if (clientCertificatePassword == null) {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pemCertificate", InputStream.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent);
+ } else {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+ }
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ }
+ builder.credential(certificate.build());
+ }
+ }
+
+ // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+ // password are not present
+ if (clientId == null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
}
}
try {
- return builder.connectionString(connectionStringBuilder.toString()).buildClient();
+ return builder.buildClient();
} catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
}
}
/**
+ * Builds the Azure data lake storage account using the provided configuration
+ *
+ * @param configuration properties
+ * @return client
+ */
+ public static DataLakeServiceClient buildAzureDatalakeClient(Map<String, String> configuration)
+ throws CompilationException {
+ String managedIdentityId = configuration.get(MANAGED_IDENTITY_ID_FIELD_NAME);
+ String accountName = configuration.get(ACCOUNT_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ String tenantId = configuration.get(TENANT_ID_FIELD_NAME);
+ String clientId = configuration.get(CLIENT_ID_FIELD_NAME);
+ String clientSecret = configuration.get(CLIENT_SECRET_FIELD_NAME);
+ String clientCertificate = configuration.get(CLIENT_CERTIFICATE_FIELD_NAME);
+ String clientCertificatePassword = configuration.get(CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME);
+ String endpoint = configuration.get(ENDPOINT_FIELD_NAME);
+
+ // Client builder
+ DataLakeServiceClientBuilder builder = new DataLakeServiceClientBuilder();
+
+ // Endpoint is required
+ if (endpoint == null) {
+ throw new CompilationException(PARAMETERS_REQUIRED, ENDPOINT_FIELD_NAME);
+ }
+ builder.endpoint(endpoint);
+
+ // Shared Key
+ if (accountName != null || accountKey != null) {
+ if (accountName == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_NAME_FIELD_NAME,
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+
+ if (accountKey == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCOUNT_KEY_FIELD_NAME,
+ ACCOUNT_NAME_FIELD_NAME);
+ }
+
+ Optional<String> provided = getFirstNotNull(configuration, SHARED_ACCESS_SIGNATURE_FIELD_NAME,
+ MANAGED_IDENTITY_ID_FIELD_NAME, CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ ACCOUNT_KEY_FIELD_NAME);
+ }
+ StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey);
+ builder.credential(credential);
+ }
+
+ // Shared access signature
+ if (sharedAccessSignature != null) {
+ Optional<String> provided = getFirstNotNull(configuration, MANAGED_IDENTITY_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME, CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ AzureSasCredential credential = new AzureSasCredential(sharedAccessSignature);
+ builder.credential(credential);
+ }
+
+ // Managed Identity auth
+ if (managedIdentityId != null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_ID_FIELD_NAME,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME,
+ TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ MANAGED_IDENTITY_ID_FIELD_NAME);
+ }
+ builder.credential(new ManagedIdentityCredentialBuilder().clientId(managedIdentityId).build());
+ }
+
+ // Client secret & certificate auth
+ if (clientId != null) {
+ // Both (or neither) client secret and client secret were provided, only one is allowed
+ if ((clientSecret == null) == (clientCertificate == null)) {
+ if (clientSecret != null) {
+ throw new CompilationException(PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME);
+ } else {
+ throw new CompilationException(REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT,
+ CLIENT_SECRET_FIELD_NAME, CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_ID_FIELD_NAME);
+ }
+ }
+
+ // Tenant ID is required
+ if (tenantId == null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, TENANT_ID_FIELD_NAME,
+ CLIENT_ID_FIELD_NAME);
+ }
+
+ // Client certificate password is not allowed if client secret is used
+ if (clientCertificatePassword != null && clientSecret != null) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT,
+ CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, CLIENT_SECRET_FIELD_NAME);
+ }
+
+ // Use AD authentication
+ if (clientSecret != null) {
+ ClientSecretCredentialBuilder secret = new ClientSecretCredentialBuilder();
+ secret.clientId(clientId);
+ secret.tenantId(tenantId);
+ secret.clientSecret(clientSecret);
+ builder.credential(secret.build());
+ } else {
+ // Certificate
+ ClientCertificateCredentialBuilder certificate = new ClientCertificateCredentialBuilder();
+ certificate.clientId(clientId);
+ certificate.tenantId(tenantId);
+ try {
+ InputStream certificateContent = new ByteArrayInputStream(clientCertificate.getBytes(UTF_8));
+ if (clientCertificatePassword == null) {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pemCertificate", InputStream.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent);
+ } else {
+ Method pemCertificate = ClientCertificateCredentialBuilder.class
+ .getDeclaredMethod("pfxCertificate", InputStream.class, String.class);
+ pemCertificate.setAccessible(true);
+ pemCertificate.invoke(certificate, certificateContent, clientCertificatePassword);
+ }
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ builder.credential(certificate.build());
+ }
+ }
+
+ // If client id is not present, ensure client secret, certificate, tenant id and client certificate
+ // password are not present
+ if (clientId == null) {
+ Optional<String> provided = getFirstNotNull(configuration, CLIENT_SECRET_FIELD_NAME,
+ CLIENT_CERTIFICATE_FIELD_NAME, CLIENT_CERTIFICATE_PASSWORD_FIELD_NAME, TENANT_ID_FIELD_NAME);
+ if (provided.isPresent()) {
+ throw new CompilationException(PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT, provided.get(),
+ SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+ }
+ }
+
+ try {
+ return builder.buildClient();
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ public static List<BlobItem> listBlobItems(BlobServiceClient blobServiceClient,
+ Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ List<BlobItem> filesOnly = new ArrayList<>();
+
+ // Ensure the validity of include/exclude
+ ExternalDataUtils.validateIncludeExclude(configuration);
+
+ BlobContainerClient blobContainer;
+ try {
+ blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+ listBlobsOptions.setPrefix(ExternalDataUtils.getPrefix(configuration));
+ Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+ // Collect the paths to files only
+ collectAndFilterBlobFiles(blobItems, includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Collects and filters the files only, and excludes any folders
+ *
+ * @param items storage items
+ * @param predicate predicate to test with for file filtration
+ * @param matchers include/exclude matchers to test against
+ * @param filesOnly List containing the files only (excluding folders)
+ */
+ private static void collectAndFilterBlobFiles(Iterable<BlobItem> items,
+ BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<BlobItem> filesOnly) {
+ for (BlobItem item : items) {
+ String uri = item.getName();
+
+ // skip folders
+ if (uri.endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, uri)) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ public static List<PathItem> listDatalakePathItems(DataLakeServiceClient client,
+ Map<String, String> configuration, IncludeExcludeMatcher includeExcludeMatcher,
+ IWarningCollector warningCollector) throws CompilationException {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+
+ List<PathItem> filesOnly = new ArrayList<>();
+
+ // Ensure the validity of include/exclude
+ ExternalDataUtils.validateIncludeExclude(configuration);
+
+ DataLakeFileSystemClient fileSystemClient;
+ try {
+ fileSystemClient = client.getFileSystemClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListPathsOptions listOptions = new ListPathsOptions();
+ boolean recursive = Boolean.parseBoolean(configuration.get(RECURSIVE_FIELD_NAME));
+ listOptions.setRecursive(recursive);
+ listOptions.setPath(ExternalDataUtils.getPrefix(configuration, false));
+ PagedIterable<PathItem> pathItems = fileSystemClient.listPaths(listOptions, null);
+
+ // Collect the paths to files only
+ collectAndFilterDatalakeFiles(pathItems, includeExcludeMatcher.getPredicate(),
+ includeExcludeMatcher.getMatchersList(), filesOnly);
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = Warning.of(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Collects and filters the files only, and excludes any folders
+ *
+ * @param items storage items
+ * @param predicate predicate to test with for file filtration
+ * @param matchers include/exclude matchers to test against
+ * @param filesOnly List containing the files only (excluding folders)
+ */
+ private static void collectAndFilterDatalakeFiles(Iterable<PathItem> items,
+ BiPredicate<List<Matcher>, String> predicate, List<Matcher> matchers, List<PathItem> filesOnly) {
+ for (PathItem item : items) {
+ String uri = item.getName();
+
+ // skip folders
+ if (uri.endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, uri)) {
+ filesOnly.add(item);
+ }
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateAzureBlobProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ BlobServiceClient blobServiceClient;
+ try {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ blobServiceClient = buildAzureBlobClient(configuration);
+ BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
+ listBlobsOptions.setPrefix(getPrefix(configuration));
+ Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
+
+ if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateAzureDataLakeProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ DataLakeServiceClient dataLakeServiceClient;
+ try {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ dataLakeServiceClient = buildAzureDatalakeClient(configuration);
+ DataLakeFileSystemClient fileSystemClient = dataLakeServiceClient.getFileSystemClient(container);
+
+ // Get all objects in a container and extract the paths to files
+ ListPathsOptions listPathsOptions = new ListPathsOptions();
+ listPathsOptions.setPath(getPrefix(configuration));
+ Iterable<PathItem> blobItems = fileSystemClient.listPaths(listPathsOptions, null);
+
+ if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+ Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+ } catch (CompilationException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ /**
+ * Builds the Azure Blob storage client using the provided configuration
+ *
+ * @param configuration properties
+ * @see <a href="https://docs.microsoft.com/en-us/azure/databricks/data/data-sources/azure/azure-storage">Azure Blob storage</a>
+ */
+ public static void configureAzureHdfsJobConf(JobConf conf, Map<String, String> configuration, String endPoint) {
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
+ String accountKey = configuration.get(ACCOUNT_KEY_FIELD_NAME);
+ String sharedAccessSignature = configuration.get(SHARED_ACCESS_SIGNATURE_FIELD_NAME);
+
+ //Disable caching S3 FileSystem
+ HDFSUtils.disableHadoopFileSystemCache(conf, HADOOP_AZURE_BLOB_PROTOCOL);
+
+ //Key for Hadoop configuration
+ StringBuilder hadoopKey = new StringBuilder();
+ //Value for Hadoop configuration
+ String hadoopValue;
+ if (accountKey != null || sharedAccessSignature != null) {
+ if (accountKey != null) {
+ hadoopKey.append(HADOOP_AZURE_FS_ACCOUNT_KEY).append('.');
+ //Set only the AccountKey
+ hadoopValue = accountKey;
+ } else {
+ //Use SAS for Hadoop FS as connectionString is provided
+ hadoopKey.append(HADOOP_AZURE_FS_SAS).append('.');
+ //Setting the container is required for SAS
+ hadoopKey.append(container).append('.');
+ //Set the connection string for SAS
+ hadoopValue = sharedAccessSignature;
+ }
+ //Set the endPoint, which includes the AccountName
+ hadoopKey.append(endPoint);
+ //Tells Hadoop we are reading from Blob Storage
+ conf.set(hadoopKey.toString(), hadoopValue);
+ }
+ }
+ }
+
+ public static class GCS {
+ private GCS() {
+ throw new AssertionError("do not instantiate");
+
+ }
+
+ //TODO(htowaileb): Add validation step similar to other externals, which also checks if empty bucket
+ //upon creating the external dataset
+
+ /**
+ * Builds the client using the provided configuration
+ *
+ * @param configuration properties
+ * @return clientasterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+ * @throws CompilationException CompilationException
+ */
+ public static Storage buildClient(Map<String, String> configuration) throws CompilationException {
+ String jsonCredentials = configuration.get(JSON_CREDENTIALS_FIELD_NAME);
+
+ StorageOptions.Builder builder = StorageOptions.newBuilder();
+
+ // Use credentials if available
+ if (jsonCredentials != null) {
+ try (InputStream credentialsStream = new ByteArrayInputStream(jsonCredentials.getBytes())) {
+ builder.setCredentials(ServiceAccountCredentials.fromStream(credentialsStream));
+ } catch (IOException ex) {
+ throw new CompilationException(EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
+ }
+ }
+
+ return builder.build().getService();
+ }
+
+ /**
* Validate external dataset properties
*
* @param configuration properties
@@ -975,28 +1852,29 @@
throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
}
+ // parquet is not supported for google cloud storage
+ if (isParquetFormat(configuration)) {
+ throw new CompilationException(INVALID_REQ_PARAM_VAL, srcLoc, KEY_FORMAT,
+ configuration.get(KEY_FORMAT));
+ }
+
validateIncludeExclude(configuration);
+ String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- // Check if the bucket is present
- BlobServiceClient blobServiceClient;
try {
- String container = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
- blobServiceClient = buildAzureClient(configuration);
- BlobContainerClient blobContainer = blobServiceClient.getBlobContainerClient(container);
+ BlobListOption limitOption = BlobListOption.pageSize(1);
+ BlobListOption prefixOption = BlobListOption.prefix(getPrefix(configuration));
+ Storage storage = buildClient(configuration);
+ Page<Blob> items = storage.list(container, limitOption, prefixOption);
- // Get all objects in a container and extract the paths to files
- ListBlobsOptions listBlobsOptions = new ListBlobsOptions();
- listBlobsOptions.setPrefix(getPrefix(configuration));
- Iterable<BlobItem> blobItems = blobContainer.listBlobs(listBlobsOptions, null);
-
- if (!blobItems.iterator().hasNext() && collector.shouldWarn()) {
+ if (!items.iterateAll().iterator().hasNext() && collector.shouldWarn()) {
Warning warning = Warning.of(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
collector.warn(warning);
}
} catch (CompilationException ex) {
throw ex;
} catch (Exception ex) {
- throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, getMessageOrToString(ex));
}
}
}
@@ -1016,4 +1894,8 @@
}
return maxArgSz;
}
+
+ private static Optional<String> getFirstNotNull(Map<String, String> configuration, String... parameters) {
+ return Arrays.stream(parameters).filter(field -> configuration.get(field) != null).findFirst();
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index 126b86f..28a0766 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -18,14 +18,18 @@
*/
package org.apache.asterix.external.util;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Base64;
import java.util.List;
import java.util.Map;
import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.ErrorCode;
@@ -36,7 +40,10 @@
import org.apache.asterix.external.input.record.reader.hdfs.parquet.MapredParquetInputFormat;
import org.apache.asterix.external.input.record.reader.hdfs.parquet.ParquetReadSupport;
import org.apache.asterix.external.input.stream.HDFSInputStream;
-import org.apache.asterix.hivecompat.io.RCFileInputFormat;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.runtime.projection.DataProjectionInfo;
+import org.apache.asterix.runtime.projection.FunctionCallInformation;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -52,6 +59,8 @@
import org.apache.hyracks.api.context.ICCContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.hdfs.scheduler.Scheduler;
import org.apache.parquet.hadoop.ParquetInputFormat;
@@ -171,8 +180,6 @@
return ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT;
case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
return ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT;
- case ExternalDataConstants.INPUT_FORMAT_RC:
- return ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT;
case ExternalDataConstants.INPUT_FORMAT_PARQUET:
return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT;
default:
@@ -187,8 +194,6 @@
return TextInputFormat.class;
case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
return SequenceFileInputFormat.class;
- case ExternalDataConstants.INPUT_FORMAT_RC:
- return RCFileInputFormat.class;
case ExternalDataConstants.INPUT_FORMAT_PARQUET:
return MapredParquetInputFormat.class;
default:
@@ -200,7 +205,7 @@
JobConf conf = new JobConf();
String localShortCircuitSocketPath = configuration.get(ExternalDataConstants.KEY_LOCAL_SOCKET_PATH);
String formatClassName = HDFSUtils.getInputFormatClassName(configuration);
- final String url = configuration.get(ExternalDataConstants.KEY_HDFS_URL);
+ String url = configuration.get(ExternalDataConstants.KEY_HDFS_URL);
//Allow hdfs adapter to read from local-files. However, this only works in a single-node configuration.
if (url != null && url.trim().startsWith("hdfs")) {
@@ -221,9 +226,17 @@
if (ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(formatClassName)) {
//Parquet configurations
conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, ParquetReadSupport.class.getName());
- //Set the requested fields. Default * which means all fields
- final String requestedFields = configuration.get(ExternalDataConstants.KEY_REQUESTED_FIELDS);
- conf.set(ExternalDataConstants.KEY_REQUESTED_FIELDS, requestedFields != null ? requestedFields : "*");
+ //Get requested values
+ String requestedValues = configuration.get(ExternalDataConstants.KEY_REQUESTED_FIELDS);
+ if (requestedValues == null) {
+ //No value is requested, return the entire record
+ requestedValues = DataProjectionInfo.ALL_FIELDS_TYPE.getTypeName();
+ } else {
+ //Subset of the values were requested, set the functionCallInformation
+ conf.set(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION,
+ configuration.get(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION));
+ }
+ conf.set(ExternalDataConstants.KEY_REQUESTED_FIELDS, requestedValues);
}
return conf;
@@ -232,20 +245,10 @@
public static AlgebricksAbsolutePartitionConstraint getPartitionConstraints(IApplicationContext appCtx,
AlgebricksAbsolutePartitionConstraint clusterLocations) {
if (clusterLocations == null) {
- IClusterStateManager clusterStateManager = ((ICcApplicationContext) appCtx).getClusterStateManager();
- ArrayList<String> locs = new ArrayList<>();
- Map<String, String[]> stores = appCtx.getMetadataProperties().getStores();
- for (String node : stores.keySet()) {
- int numIODevices = clusterStateManager.getIODevices(node).length;
- for (int k = 0; k < numIODevices; k++) {
- locs.add(node);
- }
- }
- String[] cluster = new String[locs.size()];
- cluster = locs.toArray(cluster);
- return new AlgebricksAbsolutePartitionConstraint(cluster);
+ return ((ICcApplicationContext) appCtx).getClusterStateManager().getClusterLocations();
}
return clusterLocations;
+
}
public static RecordIdType getRecordIdType(Map<String, String> configuration) {
@@ -254,10 +257,102 @@
case ExternalDataConstants.INPUT_FORMAT_TEXT:
case ExternalDataConstants.INPUT_FORMAT_SEQUENCE:
return RecordIdType.OFFSET;
- case ExternalDataConstants.INPUT_FORMAT_RC:
- return RecordIdType.RC;
default:
return null;
}
}
+
+ public static ARecordType getExpectedType(Configuration configuration) throws IOException {
+ String encoded = configuration.get(ExternalDataConstants.KEY_REQUESTED_FIELDS, "");
+ if (encoded.isEmpty() || encoded.equals(DataProjectionInfo.ALL_FIELDS_TYPE.getTypeName())) {
+ //By default, return the entire records
+ return DataProjectionInfo.ALL_FIELDS_TYPE;
+ } else if (encoded.equals(DataProjectionInfo.EMPTY_TYPE.getTypeName())) {
+ //No fields were requested
+ return DataProjectionInfo.EMPTY_TYPE;
+ }
+ //A subset of the fields was requested
+ Base64.Decoder decoder = Base64.getDecoder();
+ byte[] typeBytes = decoder.decode(encoded);
+ DataInputStream dataInputStream = new DataInputStream(new ByteArrayInputStream(typeBytes));
+ return DataProjectionInfo.createTypeField(dataInputStream);
+ }
+
+ public static void setFunctionCallInformationMap(Map<String, FunctionCallInformation> funcCallInfoMap,
+ Configuration conf) throws IOException {
+ String stringFunctionCallInfoMap = ExternalDataUtils.serializeFunctionCallInfoToString(funcCallInfoMap);
+ conf.set(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION, stringFunctionCallInfoMap);
+ }
+
+ public static Map<String, FunctionCallInformation> getFunctionCallInformationMap(Configuration conf)
+ throws IOException {
+ String encoded = conf.get(ExternalDataConstants.KEY_HADOOP_ASTERIX_FUNCTION_CALL_INFORMATION, "");
+ if (!encoded.isEmpty()) {
+ Base64.Decoder decoder = Base64.getDecoder();
+ byte[] functionCallInfoMapBytes = decoder.decode(encoded);
+ DataInputStream dataInputStream = new DataInputStream(new ByteArrayInputStream(functionCallInfoMapBytes));
+ return DataProjectionInfo.createFunctionCallInformationMap(dataInputStream);
+ }
+ return null;
+ }
+
+ public static void setWarnings(List<Warning> warnings, Configuration conf) throws IOException {
+ ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+
+ StringBuilder stringBuilder = new StringBuilder();
+ Base64.Encoder encoder = Base64.getEncoder();
+ for (int i = 0; i < warnings.size(); i++) {
+ Warning warning = warnings.get(i);
+ warning.writeFields(dataOutputStream);
+ stringBuilder.append(encoder.encodeToString(byteArrayOutputStream.toByteArray()));
+ //Warnings are separated by ','
+ stringBuilder.append(',');
+ byteArrayOutputStream.reset();
+ }
+ conf.set(ExternalDataConstants.KEY_HADOOP_ASTERIX_WARNINGS_LIST, stringBuilder.toString());
+ }
+
+ public static void issueWarnings(IWarningCollector warningCollector, Configuration conf) throws IOException {
+ String warnings = conf.get(ExternalDataConstants.KEY_HADOOP_ASTERIX_WARNINGS_LIST, "");
+ if (!warnings.isEmpty()) {
+ String[] encodedWarnings = warnings.split(",");
+ Base64.Decoder decoder = Base64.getDecoder();
+ for (int i = 0; i < encodedWarnings.length; i++) {
+ /*
+ * This should create a small number of objects as warnings are reported only once by AsterixDB's
+ * hadoop readers
+ */
+ byte[] warningBytes = decoder.decode(encodedWarnings[i]);
+ DataInputStream dataInputStream = new DataInputStream(new ByteArrayInputStream(warningBytes));
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.create(dataInputStream));
+ }
+ }
+ //Remove reported warnings
+ conf.unset(ExternalDataConstants.KEY_HADOOP_ASTERIX_WARNINGS_LIST);
+ }
+ }
+
+ /**
+ * Hadoop can cache FileSystem instance if reading the same file. This method allows for disabling the cache
+ *
+ * @param conf Hadoop configuration
+ * @param protocol fs scheme (or protocol). e.g., s3a
+ */
+ public static void disableHadoopFileSystemCache(Configuration conf, String protocol) {
+ //Disable fs cache
+ conf.set(String.format(ExternalDataConstants.KEY_HADOOP_DISABLE_FS_CACHE_TEMPLATE, protocol),
+ ExternalDataConstants.TRUE);
+ }
+
+ /**
+ * Check whether the provided path is empty
+ *
+ * @param job Hadoop Configuration
+ * @return <code>true</code> if the path is empty, <code>false</code> otherwise
+ */
+ public static boolean isEmpty(JobConf job) {
+ return job.get(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, "").isEmpty();
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory
index 7c2daed..95cfd13 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory
@@ -18,7 +18,6 @@
org.apache.asterix.external.parser.factory.ADMDataParserFactory
org.apache.asterix.external.parser.factory.JSONDataParserFactory
org.apache.asterix.external.parser.factory.DelimitedDataParserFactory
-org.apache.asterix.external.parser.factory.HiveDataParserFactory
org.apache.asterix.external.parser.factory.RecordWithMetadataParserFactory
org.apache.asterix.external.parser.factory.RSSParserFactory
org.apache.asterix.external.parser.factory.TweetParserFactory
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
index d66c44e..dceed82 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
@@ -21,4 +21,9 @@
org.apache.asterix.external.input.record.reader.stream.StreamRecordReaderFactory
org.apache.asterix.external.input.record.reader.http.HttpServerRecordReaderFactory
org.apache.asterix.external.input.record.reader.aws.AwsS3ReaderFactory
-org.apache.asterix.external.input.record.reader.azure.AzureBlobReaderFactory
+org.apache.asterix.external.input.record.reader.aws.parquet.AwsS3ParquetReaderFactory
+org.apache.asterix.external.input.record.reader.gcs.GCSReaderFactory
+org.apache.asterix.external.input.record.reader.azure.blob.AzureBlobReaderFactory
+org.apache.asterix.external.input.record.reader.azure.datalake.AzureDataLakeReaderFactory
+org.apache.asterix.external.input.record.reader.azure.parquet.AzureBlobParquetReaderFactory
+org.apache.asterix.external.input.record.reader.azure.parquet.AzureDataLakeParquetReaderFactory
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
new file mode 100644
index 0000000..c3423da
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+
+org.apache.asterix.external.parser.evaluators.RuntimeParserFunctionRegistrant
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/LosslessADMJSONDataParserTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/LosslessADMJSONDataParserTest.java
new file mode 100644
index 0000000..e457ea2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/LosslessADMJSONDataParserTest.java
@@ -0,0 +1,287 @@
+/*
+ * 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.external.parser.test;
+
+import static org.apache.asterix.om.types.AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
+import org.apache.asterix.external.parser.LosslessADMJSONDataParser;
+import org.apache.asterix.formats.nontagged.LosslessADMJSONPrinterFactoryProvider;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABinary;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.ACircle;
+import org.apache.asterix.om.base.ADate;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.ADayTimeDuration;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.ADuration;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.ALine;
+import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.AMutableUUID;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.om.base.APoint3D;
+import org.apache.asterix.om.base.APolygon;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.base.AUUID;
+import org.apache.asterix.om.base.AUnorderedList;
+import org.apache.asterix.om.base.AYearMonthDuration;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.fasterxml.jackson.core.JsonFactory;
+
+@RunWith(Parameterized.class)
+public class LosslessADMJSONDataParserTest {
+
+ static final IAObject[] PRIMITIVE_VALUES = new IAObject[] {
+ //
+ AMissing.MISSING,
+ //
+ ANull.NULL,
+ //
+ ABoolean.TRUE,
+ //
+ ABoolean.FALSE,
+ //
+ new AInt8((byte) 8),
+ //
+ new AInt8((byte) -8),
+ //
+ new AInt16((short) 16),
+ //
+ new AInt16((short) -16),
+ //
+ new AInt32((short) 32),
+ //
+ new AInt32((short) -32),
+ //
+ new AInt64(64),
+ //
+ new AInt64(-64),
+ //
+ new AFloat(1.5f),
+ //
+ new AFloat(-1.5f),
+ //
+ new AFloat(Float.POSITIVE_INFINITY),
+ //
+ new AFloat(Float.NEGATIVE_INFINITY),
+ //
+ new AFloat(Float.NaN),
+ //
+ new ADouble(1.5d),
+ //
+ new ADouble(-1.5d),
+ //
+ new ADouble(Double.POSITIVE_INFINITY),
+ //
+ new ADouble(Double.NEGATIVE_INFINITY),
+ //
+ new ADouble(Double.NaN),
+ //
+ new AString(""),
+ //
+ new AString(" "),
+ //
+ new AString(":"),
+ //
+ new AString("hello"),
+ //
+ new ADate(2),
+ //
+ new ATime((int) TimeUnit.HOURS.toMillis(3)),
+ //
+ new ADateTime(TimeUnit.DAYS.toMillis(4)),
+ //
+ new AYearMonthDuration(2),
+ //
+ new ADayTimeDuration((int) TimeUnit.HOURS.toMillis(3)),
+ //
+ new ADuration(4, (int) TimeUnit.HOURS.toMillis(5)),
+ //
+ new ABinary(new byte[] {}),
+ //
+ new ABinary(new byte[] { 1, 2, 3, 4 }),
+ //
+ createUUID(UUID.randomUUID()),
+ //
+ new APoint(1.5, -2.5),
+ //
+ new APoint3D(-1.5, 2.5, 3.5),
+ //
+ new ACircle(new APoint(1.5, -2.5), 3.5),
+ //
+ new ALine(new APoint(-1.5, -2.5), new APoint(3.5, 4.5)),
+ //
+ new ARectangle(new APoint(-1.5, -2.5), new APoint(3.5, 4.5)),
+ //
+ new APolygon(new APoint[] { new APoint(-1.5, -2.5), new APoint(-1.5, 2.5), new APoint(1.5, 2.5),
+ new APoint(1.5, -2.5) }) };
+ private final String label;
+ private final IAObject inValue;
+ private final IAObject expectedOutValue;
+
+ public LosslessADMJSONDataParserTest(String label, IAObject inValue, IAObject expectedOutValue) {
+ this.label = label;
+ this.inValue = Objects.requireNonNull(inValue);
+ this.expectedOutValue = expectedOutValue == null ? inValue : expectedOutValue;
+ }
+
+ @Parameterized.Parameters(name = "LosslessADMJSONDataParserTest {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ List<Object[]> tests = new ArrayList<>();
+ testsForPrimitives(tests);
+ testsForArrays(tests);
+ testsForObjects(tests);
+ return tests;
+ }
+
+ private static void testsForPrimitives(List<Object[]> outTests) {
+ for (IAObject v : PRIMITIVE_VALUES) {
+ outTests.add(testcase(v));
+ }
+ }
+
+ private static void testsForArrays(List<Object[]> outTests) {
+ for (IAObject v1 : PRIMITIVE_VALUES) {
+ for (IAObject v2 : PRIMITIVE_VALUES) {
+ // array of primitives
+ AOrderedList ol1 = new AOrderedList(FULL_OPEN_ORDEREDLIST_TYPE, Arrays.asList(v1, v2));
+ outTests.add(testcase(ol1));
+ // multiset of primitives (printed as array)
+ AUnorderedList ul1 =
+ new AUnorderedList(AUnorderedListType.FULLY_OPEN_UNORDEREDLIST_TYPE, Arrays.asList(v1, v2));
+ outTests.add(testcase(ul1, ol1));
+ // array of arrays
+ AOrderedList ol2 = new AOrderedList(FULL_OPEN_ORDEREDLIST_TYPE, Arrays.asList(v2, v1));
+ AOrderedList ol3 = new AOrderedList(FULL_OPEN_ORDEREDLIST_TYPE, Arrays.asList(ol1, ol2));
+ outTests.add(testcase(ol3));
+ }
+ }
+ }
+
+ private static void testsForObjects(List<Object[]> outTests) {
+ // flat
+ List<String> fieldNames = new ArrayList<>();
+ List<IAType> fieldTypes = new ArrayList<>();
+ List<IAObject> fieldValues = new ArrayList<>();
+ for (IAObject v : PRIMITIVE_VALUES) {
+ if (v.getType().getTypeTag() != ATypeTag.BIGINT) {
+ continue;
+ }
+ fieldNames.add("f" + fieldNames.size());
+ fieldTypes.add(v.getType());
+ fieldValues.add(v);
+ }
+ ARecordType rt0 =
+ new ARecordType("rt0", fieldNames.toArray(new String[0]), fieldTypes.toArray(new IAType[0]), true);
+ ARecord r0 = new ARecord(rt0, fieldValues.toArray(new IAObject[0]));
+ outTests.add(testcase(r0));
+
+ // nested
+ ARecordType rt1 = new ARecordType("rt1", new String[] { "n1", "n2" }, new IAType[] { rt0, rt0 }, true);
+ ARecord r1 = new ARecord(rt1, new IAObject[] { r0, r0 });
+ outTests.add(testcase(r1));
+ }
+
+ private static Object[] testcase(IAObject v) {
+ return testcase(v, v);
+ }
+
+ private static Object[] testcase(IAObject v, IAObject expectedOutValue) {
+ return new Object[] { String.format("%s(%s)", v.getType().getTypeName(), v), v, expectedOutValue };
+ }
+
+ private static AUUID createUUID(UUID uuid) {
+ try {
+ AMutableUUID m = new AMutableUUID();
+ char[] text = uuid.toString().toCharArray();
+ m.parseUUIDString(text, 0, text.length);
+ return m;
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ @Test
+ @SuppressWarnings("unchecked")
+ public void test() throws Exception {
+ ByteArrayAccessibleOutputStream baosInSer = new ByteArrayAccessibleOutputStream();
+ ISerializerDeserializer serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(inValue.getType());
+ serde.serialize(inValue, new DataOutputStream(baosInSer));
+
+ ByteArrayAccessibleOutputStream baosPrint = new ByteArrayAccessibleOutputStream();
+ IPrinter printer =
+ LosslessADMJSONPrinterFactoryProvider.INSTANCE.getPrinterFactory(inValue.getType()).createPrinter();
+ printer.print(baosInSer.getByteArray(), 0, baosInSer.getLength(),
+ new PrintStream(baosPrint, true, StandardCharsets.UTF_8));
+
+ ByteArrayAccessibleOutputStream baosParse = new ByteArrayAccessibleOutputStream();
+ LosslessADMJSONDataParser lp = new LosslessADMJSONDataParser(new JsonFactory());
+ lp.setInputStream(new ByteArrayAccessibleInputStream(baosPrint.getByteArray(), 0, baosPrint.getLength()));
+ lp.parseAnyValue(new DataOutputStream(baosParse));
+
+ IAObject outValue = AObjectSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+ new ByteArrayAccessibleInputStream(baosParse.getByteArray(), 0, baosParse.getLength())));
+
+ if (!expectedOutValue.deepEqual(outValue)) {
+ Assert.fail(String.format(
+ "%s print/parse test failed. In value: %s, Expected out value: %s, Actual out value: %s, Encoded value: %s",
+ label, inValue, expectedOutValue, outValue,
+ new String(baosPrint.getByteArray(), 0, baosPrint.getLength(), StandardCharsets.UTF_8)));
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ParserFactoryProviderLoadParserTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ParserFactoryProviderLoadParserTest.java
index 26cefa8..7624605 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ParserFactoryProviderLoadParserTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ParserFactoryProviderLoadParserTest.java
@@ -22,7 +22,6 @@
import org.apache.asterix.external.api.IDataParserFactory;
import org.apache.asterix.external.parser.factory.ADMDataParserFactory;
import org.apache.asterix.external.parser.factory.DelimitedDataParserFactory;
-import org.apache.asterix.external.parser.factory.HiveDataParserFactory;
import org.apache.asterix.external.parser.factory.RSSParserFactory;
import org.apache.asterix.external.parser.factory.TweetParserFactory;
import org.apache.asterix.external.provider.ParserFactoryProvider;
@@ -42,8 +41,6 @@
result = result && factory instanceof ADMDataParserFactory;
factory = ParserFactoryProvider.getDataParserFactory("rss");
result = result && factory instanceof RSSParserFactory;
- factory = ParserFactoryProvider.getDataParserFactory("hive");
- result = result && factory instanceof HiveDataParserFactory;
factory = ParserFactoryProvider.getDataParserFactory("twitter-status");
result = result && factory instanceof TweetParserFactory;
Assert.assertTrue(result);
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
index fdf0093..2fb56c7 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -39,12 +38,7 @@
public class CountHashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CountHashedGramTokensDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CountHashedGramTokensDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
index 0fac74c..c565a98 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -39,12 +38,7 @@
public class CountHashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CountHashedWordTokensDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CountHashedWordTokensDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
index de5ad44..17a9ee5 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceCheckDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.EditDistanceCheckEvaluator;
@@ -34,12 +33,7 @@
public class EditDistanceCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new EditDistanceCheckDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = EditDistanceCheckDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
index e721a9a..8ad81a9 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceContainsDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.EditDistanceContainsEvaluator;
@@ -34,12 +33,7 @@
public class EditDistanceContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new EditDistanceContainsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = EditDistanceContainsDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
index 1edb205..9bd03ea 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.EditDistanceEvaluator;
@@ -34,12 +33,7 @@
public class EditDistanceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new EditDistanceDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = EditDistanceDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
index cbba298..2c692ca 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceListIsFilterableDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -60,12 +59,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new EditDistanceListIsFilterableDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = EditDistanceListIsFilterableDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
index 8cacde0..3915405 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/EditDistanceStringIsFilterableDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -42,12 +41,7 @@
public class EditDistanceStringIsFilterableDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new EditDistanceStringIsFilterableDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = EditDistanceStringIsFilterableDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
index 9145dcc..47fd5e3 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -38,12 +37,7 @@
public class GramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GramTokensDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GramTokensDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
index 91af6f0..6504ead 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -38,12 +37,7 @@
public class HashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new HashedGramTokensDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = HashedGramTokensDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
index 9e7ecaf..874a359 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -39,12 +38,7 @@
public class HashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new HashedWordTokensDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = HashedWordTokensDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
index da331b0..5c67d09 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.fuzzyjoin.similarity.SimilarityFilters;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -51,12 +50,7 @@
private static final long serialVersionUID = 1L;
private final static FunctionIdentifier FID =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "prefix-len@3", 3);
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PrefixLenDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = PrefixLenDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
index 25942f8..62cc43d 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/PrefixLenJaccardDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,12 +49,7 @@
public class PrefixLenJaccardDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PrefixLenJaccardDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = PrefixLenJaccardDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
index 78d0507..95fd5bf 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
@@ -32,7 +32,6 @@
import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetric;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -56,12 +55,7 @@
private static final long serialVersionUID = 1L;
private final static FunctionIdentifier FID =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "similarity@7", 7);
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SimilarityDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SimilarityDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
index 167d34e..3393fcf 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardCheckDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardCheckEvaluator;
@@ -34,12 +33,7 @@
public class SimilarityJaccardCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SimilarityJaccardCheckDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SimilarityJaccardCheckDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
index dff0d22..9c3fbb1 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardEvaluator;
@@ -34,12 +33,7 @@
public class SimilarityJaccardDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SimilarityJaccardDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SimilarityJaccardDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
index e0599d6..a7e357f 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixCheckDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AFloat;
import org.apache.asterix.om.base.AMutableFloat;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.BuiltinType;
@@ -44,12 +43,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SimilarityJaccardPrefixCheckDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SimilarityJaccardPrefixCheckDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
index a3efd7c..7aa68ab 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardPrefixDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardPrefixEvaluator;
@@ -32,12 +31,7 @@
public class SimilarityJaccardPrefixDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SimilarityJaccardPrefixDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SimilarityJaccardPrefixDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
index b33fc18..f8731fc 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedCheckDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardSortedCheckEvaluator;
@@ -35,12 +34,7 @@
public class SimilarityJaccardSortedCheckDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SimilarityJaccardSortedCheckDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SimilarityJaccardSortedCheckDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
index 71e747b..33e4030 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityJaccardSortedDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.SimilarityJaccardSortedEvaluator;
@@ -35,12 +34,7 @@
public class SimilarityJaccardSortedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SimilarityJaccardSortedDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SimilarityJaccardSortedDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index a52c47c..5a01007 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -34,7 +34,6 @@
import org.apache.asterix.fuzzyjoin.IntArray;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -59,12 +58,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SpatialIntersectDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SpatialIntersectDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
index 805ac9a..2d52484 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -39,12 +38,7 @@
public class WordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new WordTokensDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = WordTokensDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
index 9caa7ae..c9bc330 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.aggregates;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -35,12 +34,7 @@
public class STUnionAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STUnionAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STUnionAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
index deef79a..0e4d748 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
@@ -53,6 +53,8 @@
import org.apache.asterix.geo.evaluators.functions.STIsSimpleDescriptor;
import org.apache.asterix.geo.evaluators.functions.STLengthDescriptor;
import org.apache.asterix.geo.evaluators.functions.STLineFromMultiPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMBRDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMBREnlargeDescriptor;
import org.apache.asterix.geo.evaluators.functions.STMDescriptor;
import org.apache.asterix.geo.evaluators.functions.STMakeEnvelopeDescriptorSRID;
import org.apache.asterix.geo.evaluators.functions.STMakePoint3DDescriptor;
@@ -153,5 +155,7 @@
fc.add(STIntersectionDescriptor.FACTORY);
fc.add(STSymDifferenceDescriptor.FACTORY);
fc.add(STPolygonizeDescriptor.FACTORY);
+ fc.add(STMBRDescriptor.FACTORY);
+ fc.add(STMBREnlargeDescriptor.FACTORY);
}
}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
index 29c2fd8..e41aebb 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
@@ -29,6 +29,7 @@
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.AGeometry;
import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.ARectangle;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -122,6 +123,9 @@
} else if (finalResult instanceof OGCGeometry) {
out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+ } else if (finalResult instanceof ARectangle) {
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE)
+ .serialize(finalResult, out);
}
} catch (IOException e) {
throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
index a00e5a8..e0e43bb 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STAsBinaryDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STAsBinaryDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STAsBinaryDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
index 30ebdcc..760261d 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STAsGeoJSONDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STAsGeoJSONDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STAsGeoJSONDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
index d3ac964..12bdb30 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STAsTextDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STAsTextDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STAsTextDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
index 286c9c7..9bd6ba1 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STBoundaryDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STBoundaryDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STBoundaryDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
index 0235e52..34c21bf 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STContainsDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STContainsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STContainsDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
index 5781e2b..a0c10f7 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STCoordDimDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STCoordDimDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STCoordDimDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
index a2bd59e..c276e7e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STCrossesDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STCrossesDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STCrossesDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
index 48b0f9e..8a49d4b 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STDifferenceDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STDifferenceDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
index 6c0391b..34d7282 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STDimensionDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STDimensionDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STDimensionDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
index 01827ee..0a890b9 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STDisjointDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STDisjointDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STDisjointDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
index 997dad8..75b8b41 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STDistanceDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STDistanceDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STDistanceDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
index 659d100..ab0ba5c 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STEndPointDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STEndPointDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STEndPointDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
index bde4f50..13555c6 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STEnvelopeDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STEnvelopeDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STEnvelopeDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
index 8de5451..e0c0cf8 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STEqualsDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STEqualsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STEqualsDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
index 3df1aa3..5888b63 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STExteriorRingDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STExteriorRingDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STExteriorRingDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
index e0795f1..73272ae 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -50,12 +49,7 @@
public class STGeomFromTextDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STGeomFromTextDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STGeomFromTextDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
index 04ea807..aca726e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -51,12 +50,7 @@
public class STGeomFromTextSRIDDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STGeomFromTextSRIDDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STGeomFromTextSRIDDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
index e4f3be3..177bde4 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
@@ -24,7 +24,6 @@
import java.io.IOException;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -43,12 +42,7 @@
public class STGeomFromWKBDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STGeomFromWKBDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STGeomFromWKBDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
index 72b59f8..8a432d1 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STGeomentryTypeDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STGeomentryTypeDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STGeomentryTypeDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
index 848241c..5d9642e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STGeometryNDescriptor extends AbstractSTGeometryNDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STGeometryNDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STGeometryNDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
index dcf2b9f..5f6d956 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STInteriorRingNDescriptor extends AbstractSTGeometryNDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STInteriorRingNDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STInteriorRingNDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
index 2653266..5a3d2bf 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STIntersectionDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STIntersectionDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STIntersectionDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
index a442975..34a1407 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STIntersectsDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STIntersectsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STIntersectsDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
index 13f86b6..ff55f9f 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -36,12 +35,7 @@
public class STIsClosedDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STIsClosedDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STIsClosedDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
index 1d42b74..544fdf3 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STIsCollectionDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STIsCollectionDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STIsCollectionDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
index 496e416..1adcb0e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STIsEmptyDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STIsEmptyDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STIsEmptyDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
index 3a56197..02ea049a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STIsRingDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STIsRingDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STIsRingDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
index d905528..b8b2d63 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STIsSimpleDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STIsSimpleDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STIsSimpleDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
index 91d31fc..7357813 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -33,12 +32,7 @@
public class STLengthDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STLengthDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STLengthDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
index df7b42c..b716b5c 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -34,12 +33,7 @@
public class STLineFromMultiPointDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STLineFromMultiPointDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STLineFromMultiPointDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java
new file mode 100644
index 0000000..55e358c
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * 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.geo.evaluators.functions;
+
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class STMBRDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STMBRDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+
+ AMutableRectangle aRectangle = new AMutableRectangle(null, null);
+ AMutablePoint[] aPoint = { new AMutablePoint(0, 0), new AMutablePoint(0, 0) };
+ Envelope env = new Envelope();
+ geometry.getEsriGeometry().queryEnvelope(env);
+ aPoint[0].setValue(env.getXMin(), env.getYMin());
+ aPoint[1].setValue(env.getXMax(), env.getYMax());
+ aRectangle.setValue(aPoint[0], aPoint[1]);
+ return aRectangle;
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_MBR;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java
new file mode 100644
index 0000000..b3bfb33
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.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.geo.evaluators.functions;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class STMBREnlargeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STMBREnlargeDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_MBR_ENLARGE;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new IScalarEvaluator() {
+ private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private DataOutput out = resultStorage.getDataOutput();
+ private IPointable inputArg0 = new VoidPointable();
+ private IPointable inputArg1 = new VoidPointable();
+ private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+ private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+ private ByteArrayAccessibleInputStream inStream =
+ new ByteArrayAccessibleInputStream(new byte[0], 0, 0);
+ private DataInputStream dataIn = new DataInputStream(inStream);
+
+ Envelope env = new Envelope();
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ARectangle> rectangleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1)) {
+ return;
+ }
+
+ byte[] data0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+ int len = inputArg0.getLength();
+
+ byte[] data1 = inputArg1.getByteArray();
+ int offset1 = inputArg1.getStartOffset();
+
+ if (data0[offset0] != ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data0[offset0],
+ ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ inStream.setContent(data0, offset0 + 1, len - 1);
+ OGCGeometry geometry =
+ AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+ geometry.getEsriGeometry().queryEnvelope(env);
+ double expandValue =
+ ATypeHierarchy.getDoubleValue(getIdentifier().getName(), 0, data1, offset1);
+ AMutableRectangle expandedMBR = new AMutableRectangle(
+ new AMutablePoint(env.getXMin() - expandValue, env.getYMin() - expandValue),
+ new AMutablePoint(env.getXMax() + expandValue, env.getYMax() + expandValue));
+ rectangleSerde.serialize(expandedMBR, out);
+ result.set(resultStorage);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
index 67f966a..e208771 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STMDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STMDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STMDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
index 8615268..3aa9e7b 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
@@ -23,7 +23,6 @@
import java.nio.ByteBuffer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
@@ -43,12 +42,7 @@
public class STMakeEnvelopeDescriptorSRID extends AbstractGetValDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STMakeEnvelopeDescriptorSRID();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STMakeEnvelopeDescriptorSRID::new;
private static final long serialVersionUID = 1L;
@@ -103,6 +97,9 @@
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+
+ resultStorage.reset();
+
eval0.evaluate(tuple, inputArg0);
byte[] data0 = inputArg0.getByteArray();
int offset0 = inputArg0.getStartOffset();
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
index e157f52..4a5cf7e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
import org.apache.asterix.om.base.AGeometry;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -44,12 +43,7 @@
public class STMakePoint3DDescriptor extends AbstractGetValDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STMakePoint3DDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STMakePoint3DDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
index 3505ac6..55baa37 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
import org.apache.asterix.om.base.AGeometry;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -44,12 +43,7 @@
public class STMakePoint3DWithMDescriptor extends AbstractGetValDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STMakePoint3DWithMDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STMakePoint3DWithMDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
index 86c4d01..c3b7b36 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
import org.apache.asterix.om.base.AGeometry;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -44,12 +43,7 @@
public class STMakePointDescriptor extends AbstractGetValDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STMakePointDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STMakePointDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
index aca1094..309d325 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -33,12 +32,7 @@
public class STNPointsDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STNPointsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STNPointsDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
index caed1ff..f8e313d 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,12 +30,7 @@
public class STNRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STNRingsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STNRingsDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
index 2d66712..e87d1f4 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STNumGeometriesDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STNumGeometriesDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STNumGeometriesDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
index f529a18..19f2e11 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STNumInteriorRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STNumInteriorRingsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STNumInteriorRingsDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
index 59ca76f..61286b0 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STOverlapsDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STOverlapsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STOverlapsDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
index b7491e0..e0495f5 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STPointNDescriptor extends AbstractSTGeometryNDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STPointNDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STPointNDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
index e10b3a0..e2fcd8a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
@@ -32,7 +32,6 @@
import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
@@ -58,12 +57,7 @@
public class STPolygonizeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STPolygonizeDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STPolygonizeDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
index 61b801b..c3dc132 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,12 +49,7 @@
public class STRelateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STRelateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STRelateDescriptor::new;
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
index 470cc82..e50678a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STSRIDDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STSRIDDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STSRIDDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
index fd16405..127b0d5 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STStartPointDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STStartPointDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STStartPointDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
index 3a60f35..e4d0507 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STSymDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STSymDifferenceDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STSymDifferenceDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
index d7595d9..ddf6550 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STTouchesDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STTouchesDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STTouchesDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
index 693723e..1cfea10 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STUnionDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STUnionDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STUnionDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
index 809779f..c78e48f 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STWithinDescriptor extends AbstractSTDoubleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STWithinDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STWithinDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
index 109d7f8..608e55cc 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STXDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STXDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STXDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
index 887a423..2431f3f 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,12 +30,7 @@
public class STXMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STXMaxDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STXMaxDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
index 98a23b5..66db0c6 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,12 +30,7 @@
public class STXMinDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STXMinDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STXMinDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
index 8f867a7..32baa0a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STYDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STYDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STYDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
index 5d0b0f8..7b82915 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,12 +30,7 @@
public class STYMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STYMaxDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STYMaxDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
index 2b2d723..1cc5e88 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,12 +30,7 @@
public class STYMinDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STYMinDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STYMinDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
index 2079ab0..739775a 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,12 +29,7 @@
public class STZDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STZDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STZDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
index ab58365..cb3f54e 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STZMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STZMaxDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STZMaxDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
index c8731ee..3f6f6b3 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.geo.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -29,12 +28,7 @@
public class STZMinDescriptor extends AbstractSTSingleGeometryDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new STZMinDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = STZMinDescriptor::new;
@Override
protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
diff --git a/asterixdb/asterix-hivecompat/pom.xml b/asterixdb/asterix-hivecompat/pom.xml
deleted file mode 100644
index f8ea3fa..0000000
--- a/asterixdb/asterix-hivecompat/pom.xml
+++ /dev/null
@@ -1,80 +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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <artifactId>apache-asterixdb</artifactId>
- <groupId>org.apache.asterix</groupId>
- <version>0.9.8-SNAPSHOT</version>
- </parent>
- <licenses>
- <license>
- <name>Apache License, Version 2.0</name>
- <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
- <distribution>repo</distribution>
- <comments>A business-friendly OSS license</comments>
- </license>
- </licenses>
- <artifactId>asterix-hivecompat</artifactId>
- <properties>
- <root.dir>${basedir}/..</root.dir>
- <source-format.skip>true</source-format.skip>
- </properties>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-checkstyle-plugin</artifactId>
- <configuration>
- <excludes>**/hivecompat/**/*</excludes>
- </configuration>
- </plugin>
- </plugins>
- </build>
- <dependencies>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-common</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-serde</artifactId>
- </dependency>
- <dependency>
- <groupId>commons-logging</groupId>
- <artifactId>commons-logging-api</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hive.shims</groupId>
- <artifactId>hive-shims-common</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-common</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-mapreduce-client-core</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.logging.log4j</groupId>
- <artifactId>log4j-1.2-api</artifactId>
- </dependency>
- </dependencies>
-</project>
diff --git a/asterixdb/asterix-hivecompat/src/main/appended-resources/META-INF/LICENSE b/asterixdb/asterix-hivecompat/src/main/appended-resources/META-INF/LICENSE
deleted file mode 100644
index b0cc878..0000000
--- a/asterixdb/asterix-hivecompat/src/main/appended-resources/META-INF/LICENSE
+++ /dev/null
@@ -1,9 +0,0 @@
-===
- AsterixDB includes source code with separate copyright notices and
- license terms. Your use of this source code is subject to the terms
- and condition of the following licenses.
-===
- Source files org/apache/asterix/hivecompat/io/* are derived from portions
- of Apache Hive Query Language v0.13.0 (org.apache.hive:hive-exec),
- made available under The Apache License, Version 2.0:
----
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/CodecPool.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/CodecPool.java
deleted file mode 100644
index aab4c72..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/CodecPool.java
+++ /dev/null
@@ -1,164 +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.
- */
-package org.apache.asterix.hivecompat.io;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.hadoop.io.compress.Decompressor;
-
-/**
- * A global compressor/decompressor pool used to save and reuse (possibly
- * native) compression/decompression codecs.
- */
-public final class CodecPool {
- private static final Log LOG = LogFactory.getLog(CodecPool.class);
-
- /**
- * A global compressor pool used to save the expensive
- * construction/destruction of (possibly native) decompression codecs.
- */
- private static final Map<Class<Compressor>, List<Compressor>> COMPRESSOR_POOL =
- new HashMap<Class<Compressor>, List<Compressor>>();
-
- /**
- * A global decompressor pool used to save the expensive
- * construction/destruction of (possibly native) decompression codecs.
- */
- private static final Map<Class<Decompressor>, List<Decompressor>> DECOMPRESSOR_POOL =
- new HashMap<Class<Decompressor>, List<Decompressor>>();
-
- private static <T> T borrow(Map<Class<T>, List<T>> pool,
- Class<? extends T> codecClass) {
- T codec = null;
-
- // Check if an appropriate codec is available
- synchronized (pool) {
- if (pool.containsKey(codecClass)) {
- List<T> codecList = pool.get(codecClass);
-
- if (codecList != null) {
- synchronized (codecList) {
- if (!codecList.isEmpty()) {
- codec = codecList.remove(codecList.size() - 1);
- }
- }
- }
- }
- }
-
- return codec;
- }
-
- private static <T> void payback(Map<Class<T>, List<T>> pool, T codec) {
- if (codec != null) {
- Class<T> codecClass = (Class<T>) codec.getClass();
- synchronized (pool) {
- if (!pool.containsKey(codecClass)) {
- pool.put(codecClass, new ArrayList<T>());
- }
-
- List<T> codecList = pool.get(codecClass);
- synchronized (codecList) {
- codecList.add(codec);
- }
- }
- }
- }
-
- /**
- * Get a {@link Compressor} for the given {@link CompressionCodec} from the
- * pool or a new one.
- *
- * @param codec
- * the <code>CompressionCodec</code> for which to get the
- * <code>Compressor</code>
- * @return <code>Compressor</code> for the given <code>CompressionCodec</code>
- * from the pool or a new one
- */
- public static Compressor getCompressor(CompressionCodec codec) {
- Compressor compressor = borrow(COMPRESSOR_POOL, codec.getCompressorType());
- if (compressor == null) {
- compressor = codec.createCompressor();
- LOG.info("Got brand-new compressor");
- } else {
- LOG.debug("Got recycled compressor");
- }
- return compressor;
- }
-
- /**
- * Get a {@link Decompressor} for the given {@link CompressionCodec} from the
- * pool or a new one.
- *
- * @param codec
- * the <code>CompressionCodec</code> for which to get the
- * <code>Decompressor</code>
- * @return <code>Decompressor</code> for the given
- * <code>CompressionCodec</code> the pool or a new one
- */
- public static Decompressor getDecompressor(CompressionCodec codec) {
- Decompressor decompressor = borrow(DECOMPRESSOR_POOL, codec
- .getDecompressorType());
- if (decompressor == null) {
- decompressor = codec.createDecompressor();
- LOG.info("Got brand-new decompressor");
- } else {
- LOG.debug("Got recycled decompressor");
- }
- return decompressor;
- }
-
- /**
- * Return the {@link Compressor} to the pool.
- *
- * @param compressor
- * the <code>Compressor</code> to be returned to the pool
- */
- public static void returnCompressor(Compressor compressor) {
- if (compressor == null) {
- return;
- }
- compressor.reset();
- payback(COMPRESSOR_POOL, compressor);
- }
-
- /**
- * Return the {@link Decompressor} to the pool.
- *
- * @param decompressor
- * the <code>Decompressor</code> to be returned to the pool
- */
- public static void returnDecompressor(Decompressor decompressor) {
- if (decompressor == null) {
- return;
- }
- decompressor.reset();
- payback(DECOMPRESSOR_POOL, decompressor);
- }
-
- private CodecPool() {
- // prevent instantiation
- }
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/InputFormatChecker.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/InputFormatChecker.java
deleted file mode 100644
index 4ac8a59..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/InputFormatChecker.java
+++ /dev/null
@@ -1,40 +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.
- */
-
-package org.apache.asterix.hivecompat.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hive.conf.HiveConf;
-
-/**
- * Check for validity of the input files.
- */
-public interface InputFormatChecker {
-
- /**
- * This method is used to validate the input files.
- *
- */
- boolean validateInput(FileSystem fs, HiveConf conf,
- ArrayList<FileStatus> files) throws IOException;
-
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/NonSyncDataInputBuffer.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/NonSyncDataInputBuffer.java
deleted file mode 100644
index 8cb890b..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/NonSyncDataInputBuffer.java
+++ /dev/null
@@ -1,512 +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.
- */
-
-package org.apache.asterix.hivecompat.io;
-
-import java.io.DataInput;
-import java.io.EOFException;
-import java.io.FilterInputStream;
-import java.io.IOException;
-import java.io.PushbackInputStream;
-import java.io.UTFDataFormatException;
-
-import org.apache.hadoop.hive.common.io.NonSyncByteArrayInputStream;
-
-/**
- * A thread-not-safe version of Hadoop's DataInputBuffer, which removes all
- * synchronized modifiers.
- */
-public class NonSyncDataInputBuffer extends FilterInputStream implements
- DataInput {
-
- private final NonSyncByteArrayInputStream buffer;
-
- byte[] buff = new byte[16];
-
- /** Constructs a new empty buffer. */
- public NonSyncDataInputBuffer() {
- this(new NonSyncByteArrayInputStream());
- }
-
- private NonSyncDataInputBuffer(NonSyncByteArrayInputStream buffer) {
- super(buffer);
- this.buffer = buffer;
- }
-
- /** Resets the data that the buffer reads. */
- public void reset(byte[] input, int length) {
- buffer.reset(input, 0, length);
- }
-
- /** Resets the data that the buffer reads. */
- public void reset(byte[] input, int start, int length) {
- buffer.reset(input, start, length);
- }
-
- /** Returns the current position in the input. */
- public int getPosition() {
- return buffer.getPosition();
- }
-
- /** Returns the length of the input. */
- public int getLength() {
- return buffer.getLength();
- }
-
- /**
- * Reads bytes from the source stream into the byte array <code>buffer</code>.
- * The number of bytes actually read is returned.
- *
- * @param buffer
- * the buffer to read bytes into
- * @return the number of bytes actually read or -1 if end of stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final int read(byte[] buffer) throws IOException {
- return in.read(buffer, 0, buffer.length);
- }
-
- /**
- * Read at most <code>length</code> bytes from this DataInputStream and stores
- * them in byte array <code>buffer</code> starting at <code>offset</code>.
- * Answer the number of bytes actually read or -1 if no bytes were read and
- * end of stream was encountered.
- *
- * @param buffer
- * the byte array in which to store the read bytes.
- * @param offset
- * the offset in <code>buffer</code> to store the read bytes.
- * @param length
- * the maximum number of bytes to store in <code>buffer</code>.
- * @return the number of bytes actually read or -1 if end of stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final int read(byte[] buffer, int offset, int length)
- throws IOException {
- return in.read(buffer, offset, length);
- }
-
- /**
- * Reads a boolean from this stream.
- *
- * @return the next boolean value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final boolean readBoolean() throws IOException {
- int temp = in.read();
- if (temp < 0) {
- throw new EOFException();
- }
- return temp != 0;
- }
-
- /**
- * Reads an 8-bit byte value from this stream.
- *
- * @return the next byte value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final byte readByte() throws IOException {
- int temp = in.read();
- if (temp < 0) {
- throw new EOFException();
- }
- return (byte) temp;
- }
-
- /**
- * Reads a 16-bit character value from this stream.
- *
- * @return the next <code>char</code> value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- private int readToBuff(int count) throws IOException {
- int offset = 0;
-
- while (offset < count) {
- int bytesRead = in.read(buff, offset, count - offset);
- if (bytesRead == -1) {
- return bytesRead;
- }
- offset += bytesRead;
- }
- return offset;
- }
-
- @Override
- public final char readChar() throws IOException {
- if (readToBuff(2) < 0) {
- throw new EOFException();
- }
- return (char) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
-
- }
-
- /**
- * Reads a 64-bit <code>double</code> value from this stream.
- *
- * @return the next <code>double</code> value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final double readDouble() throws IOException {
- return Double.longBitsToDouble(readLong());
- }
-
- /**
- * Reads a 32-bit <code>float</code> value from this stream.
- *
- * @return the next <code>float</code> value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final float readFloat() throws IOException {
- return Float.intBitsToFloat(readInt());
- }
-
- /**
- * Reads bytes from this stream into the byte array <code>buffer</code>. This
- * method will block until <code>buffer.length</code> number of bytes have
- * been read.
- *
- * @param buffer
- * to read bytes into
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final void readFully(byte[] buffer) throws IOException {
- readFully(buffer, 0, buffer.length);
- }
-
- /**
- * Reads bytes from this stream and stores them in the byte array
- * <code>buffer</code> starting at the position <code>offset</code>. This
- * method blocks until <code>count</code> bytes have been read.
- *
- * @param buffer
- * the byte array into which the data is read
- * @param offset
- * the offset the operation start at
- * @param length
- * the maximum number of bytes to read
- *
- * @throws IOException
- * if a problem occurs while reading from this stream
- * @throws EOFException
- * if reaches the end of the stream before enough bytes have been
- * read
- */
- @Override
- public final void readFully(byte[] buffer, int offset, int length)
- throws IOException {
- if (length < 0) {
- throw new IndexOutOfBoundsException();
- }
- if (length == 0) {
- return;
- }
- if (in == null || buffer == null) {
- throw new NullPointerException("Null Pointer to underlying input stream");
- }
-
- if (offset < 0 || offset > buffer.length - length) {
- throw new IndexOutOfBoundsException();
- }
- while (length > 0) {
- int result = in.read(buffer, offset, length);
- if (result < 0) {
- throw new EOFException();
- }
- offset += result;
- length -= result;
- }
- }
-
- /**
- * Reads a 32-bit integer value from this stream.
- *
- * @return the next <code>int</code> value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final int readInt() throws IOException {
- if (readToBuff(4) < 0) {
- throw new EOFException();
- }
- return ((buff[0] & 0xff) << 24) | ((buff[1] & 0xff) << 16)
- | ((buff[2] & 0xff) << 8) | (buff[3] & 0xff);
- }
-
- /**
- * Answers a <code>String</code> representing the next line of text available
- * in this BufferedReader. A line is represented by 0 or more characters
- * followed by <code>'\n'</code>, <code>'\r'</code>, <code>"\n\r"</code> or
- * end of stream. The <code>String</code> does not include the newline
- * sequence.
- *
- * @return the contents of the line or null if no characters were read before
- * end of stream.
- *
- * @throws IOException
- * If the DataInputStream is already closed or some other IO error
- * occurs.
- *
- * @deprecated Use BufferedReader
- */
- @Deprecated
- @Override
- public final String readLine() throws IOException {
- StringBuilder line = new StringBuilder(80); // Typical line length
- boolean foundTerminator = false;
- while (true) {
- int nextByte = in.read();
- switch (nextByte) {
- case -1:
- if (line.length() == 0 && !foundTerminator) {
- return null;
- }
- return line.toString();
- case (byte) '\r':
- if (foundTerminator) {
- ((PushbackInputStream) in).unread(nextByte);
- return line.toString();
- }
- foundTerminator = true;
- /* Have to be able to peek ahead one byte */
- if (!(in.getClass() == PushbackInputStream.class)) {
- in = new PushbackInputStream(in);
- }
- break;
- case (byte) '\n':
- return line.toString();
- default:
- if (foundTerminator) {
- ((PushbackInputStream) in).unread(nextByte);
- return line.toString();
- }
- line.append((char) nextByte);
- }
- }
- }
-
- /**
- * Reads a 64-bit <code>long</code> value from this stream.
- *
- * @return the next <code>long</code> value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final long readLong() throws IOException {
- if (readToBuff(8) < 0) {
- throw new EOFException();
- }
- int i1 = ((buff[0] & 0xff) << 24) | ((buff[1] & 0xff) << 16)
- | ((buff[2] & 0xff) << 8) | (buff[3] & 0xff);
- int i2 = ((buff[4] & 0xff) << 24) | ((buff[5] & 0xff) << 16)
- | ((buff[6] & 0xff) << 8) | (buff[7] & 0xff);
-
- return ((i1 & 0xffffffffL) << 32) | (i2 & 0xffffffffL);
- }
-
- /**
- * Reads a 16-bit <code>short</code> value from this stream.
- *
- * @return the next <code>short</code> value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final short readShort() throws IOException {
- if (readToBuff(2) < 0) {
- throw new EOFException();
- }
- return (short) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
- }
-
- /**
- * Reads an unsigned 8-bit <code>byte</code> value from this stream and
- * returns it as an int.
- *
- * @return the next unsigned byte value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final int readUnsignedByte() throws IOException {
- int temp = in.read();
- if (temp < 0) {
- throw new EOFException();
- }
- return temp;
- }
-
- /**
- * Reads a 16-bit unsigned <code>short</code> value from this stream and
- * returns it as an int.
- *
- * @return the next unsigned <code>short</code> value from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final int readUnsignedShort() throws IOException {
- if (readToBuff(2) < 0) {
- throw new EOFException();
- }
- return (char) (((buff[0] & 0xff) << 8) | (buff[1] & 0xff));
- }
-
- /**
- * Reads a UTF format String from this Stream.
- *
- * @return the next UTF String from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- @Override
- public final String readUTF() throws IOException {
- return decodeUTF(readUnsignedShort());
- }
-
- String decodeUTF(int utfSize) throws IOException {
- return decodeUTF(utfSize, this);
- }
-
- private static String decodeUTF(int utfSize, DataInput in) throws IOException {
- byte[] buf = new byte[utfSize];
- char[] out = new char[utfSize];
- in.readFully(buf, 0, utfSize);
-
- return convertUTF8WithBuf(buf, out, 0, utfSize);
- }
-
- /**
- * Reads a UTF format String from the DataInput Stream <code>in</code>.
- *
- * @param in
- * the input stream to read from
- * @return the next UTF String from the source stream.
- *
- * @throws IOException
- * If a problem occurs reading from this DataInputStream.
- *
- */
- public static final String readUTF(DataInput in) throws IOException {
- return decodeUTF(in.readUnsignedShort(), in);
- }
-
- /**
- * Skips <code>count</code> number of bytes in this stream. Subsequent
- * <code>read()</code>'s will not return these bytes unless
- * <code>reset()</code> is used.
- *
- * @param count
- * the number of bytes to skip.
- * @return the number of bytes actually skipped.
- *
- * @throws IOException
- * If the stream is already closed or another IOException occurs.
- */
- @Override
- public final int skipBytes(int count) throws IOException {
- int skipped = 0;
- long skip;
- while (skipped < count && (skip = in.skip(count - skipped)) != 0) {
- skipped += skip;
- }
- if (skipped < 0) {
- throw new EOFException();
- }
- return skipped;
- }
-
- public static String convertUTF8WithBuf(byte[] buf, char[] out, int offset,
- int utfSize) throws UTFDataFormatException {
- int count = 0, s = 0, a;
- while (count < utfSize) {
- if ((out[s] = (char) buf[offset + count++]) < '\u0080') {
- s++;
- } else if (((a = out[s]) & 0xe0) == 0xc0) {
- if (count >= utfSize) {
- throw new UTFDataFormatException();
- }
- int b = buf[count++];
- if ((b & 0xC0) != 0x80) {
- throw new UTFDataFormatException();
- }
- out[s++] = (char) (((a & 0x1F) << 6) | (b & 0x3F));
- } else if ((a & 0xf0) == 0xe0) {
- if (count + 1 >= utfSize) {
- throw new UTFDataFormatException();
- }
- int b = buf[count++];
- int c = buf[count++];
- if (((b & 0xC0) != 0x80) || ((c & 0xC0) != 0x80)) {
- throw new UTFDataFormatException();
- }
- out[s++] = (char) (((a & 0x0F) << 12) | ((b & 0x3F) << 6) | (c & 0x3F));
- } else {
- throw new UTFDataFormatException();
- }
- }
- return new String(out, 0, s);
- }
-
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/NonSyncDataOutputBuffer.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/NonSyncDataOutputBuffer.java
deleted file mode 100644
index 878f130..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/NonSyncDataOutputBuffer.java
+++ /dev/null
@@ -1,88 +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.
- */
-package org.apache.asterix.hivecompat.io;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.hive.common.io.NonSyncByteArrayOutputStream;
-
-/**
- * A thread-not-safe version of Hadoop's DataOutputBuffer, which removes all
- * synchronized modifiers.
- */
-public class NonSyncDataOutputBuffer extends DataOutputStream {
-
- private final NonSyncByteArrayOutputStream buffer;
-
- /** Constructs a new empty buffer. */
- public NonSyncDataOutputBuffer() {
- this(new NonSyncByteArrayOutputStream());
- }
-
- private NonSyncDataOutputBuffer(NonSyncByteArrayOutputStream buffer) {
- super(buffer);
- this.buffer = buffer;
- }
-
- /**
- * Returns the current contents of the buffer. Data is only valid to
- * {@link #getLength()}.
- */
- public byte[] getData() {
- return buffer.getData();
- }
-
- /** Returns the length of the valid data currently in the buffer. */
- public int getLength() {
- return buffer.getLength();
- }
-
- /** Resets the buffer to empty. */
- public NonSyncDataOutputBuffer reset() {
- written = 0;
- buffer.reset();
- return this;
- }
-
- /** Writes bytes from a DataInput directly into the buffer. */
- public void write(DataInput in, int length) throws IOException {
- buffer.write(in, length);
- }
-
- @Override
- public void write(int b) throws IOException {
- buffer.write(b);
- incCount(1);
- }
-
- @Override
- public void write(byte b[], int off, int len) throws IOException {
- buffer.write(b, off, len);
- incCount(len);
- }
-
- private void incCount(int value) {
- if (written + value < 0) {
- written = Integer.MAX_VALUE;
- } else {
- written += value;
- }
- }
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFile.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFile.java
deleted file mode 100644
index 0c5613a..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFile.java
+++ /dev/null
@@ -1,2049 +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.
- */
-
-package org.apache.asterix.hivecompat.io;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.rmi.server.UID;
-import java.security.MessageDigest;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable;
-import org.apache.hadoop.hive.serde2.columnar.LazyDecompressionCallback;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile.Metadata;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VersionMismatchException;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionInputStream;
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * <code>RCFile</code>s, short of Record Columnar File, are flat files
- * consisting of binary key/value pairs, which shares much similarity with
- * <code>SequenceFile</code>.
- *
- * RCFile stores columns of a table in a record columnar way. It first
- * partitions rows horizontally into row splits. and then it vertically
- * partitions each row split in a columnar way. RCFile first stores the meta
- * data of a row split, as the key part of a record, and all the data of a row
- * split as the value part. When writing, RCFile.Writer first holds records'
- * value bytes in memory, and determines a row split if the raw bytes size of
- * buffered records overflow a given parameter<tt>Writer.columnsBufferSize</tt>,
- * which can be set like: <code>conf.setInt(COLUMNS_BUFFER_SIZE_CONF_STR,
- 4 * 1024 * 1024)</code> .
- * <p>
- * <code>RCFile</code> provides {@link Writer}, {@link Reader} and classes for
- * writing, reading respectively.
- * </p>
- *
- * <p>
- * RCFile stores columns of a table in a record columnar way. It first
- * partitions rows horizontally into row splits. and then it vertically
- * partitions each row split in a columnar way. RCFile first stores the meta
- * data of a row split, as the key part of a record, and all the data of a row
- * split as the value part.
- * </p>
- *
- * <p>
- * RCFile compresses values in a more fine-grained manner then record level
- * compression. However, It currently does not support compress the key part
- * yet. The actual compression algorithm used to compress key and/or values can
- * be specified by using the appropriate {@link CompressionCodec}.
- * </p>
- *
- * <p>
- * The {@link Reader} is used to read and explain the bytes of RCFile.
- * </p>
- *
- * <h4 id="Formats">RCFile Formats</h4>
- *
- *
- * <h5 id="Header">RC Header</h5>
- * <ul>
- * <li>version - 3 bytes of magic header <b>RCF</b>, followed by 1 byte of
- * actual version number (e.g. RCF1)</li>
- * <li>compression - A boolean which specifies if compression is turned on for
- * keys/values in this file.</li>
- * <li>compression codec - <code>CompressionCodec</code> class which is used
- * for compression of keys and/or values (if compression is enabled).</li>
- * <li>metadata - {@link Metadata} for this file.</li>
- * <li>sync - A sync marker to denote end of the header.</li>
- * </ul>
- *
- * <h5>RCFile Format</h5>
- * <ul>
- * <li><a href="#Header">Header</a></li>
- * <li>Record
- * <li>Key part
- * <ul>
- * <li>Record length in bytes</li>
- * <li>Key length in bytes</li>
- * <li>Number_of_rows_in_this_record(vint)</li>
- * <li>Column_1_ondisk_length(vint)</li>
- * <li>Column_1_row_1_value_plain_length</li>
- * <li>Column_1_row_2_value_plain_length</li>
- * <li>...</li>
- * <li>Column_2_ondisk_length(vint)</li>
- * <li>Column_2_row_1_value_plain_length</li>
- * <li>Column_2_row_2_value_plain_length</li>
- * <li>...</li>
- * </ul>
- * </li>
- * </li>
- * <li>Value part
- * <ul>
- * <li>Compressed or plain data of [column_1_row_1_value,
- * column_1_row_2_value,....]</li>
- * <li>Compressed or plain data of [column_2_row_1_value,
- * column_2_row_2_value,....]</li>
- * </ul>
- * </li>
- * </ul>
- * <p>
- * <pre>
- * {@code
- * The following is a pseudo-BNF grammar for RCFile. Comments are prefixed
- * with dashes:
- *
- * rcfile ::=
- * <file-header>
- * <rcfile-rowgroup>+
- *
- * file-header ::=
- * <file-version-header>
- * <file-key-class-name> (only exists if version is seq6)
- * <file-value-class-name> (only exists if version is seq6)
- * <file-is-compressed>
- * <file-is-block-compressed> (only exists if version is seq6)
- * [<file-compression-codec-class>]
- * <file-header-metadata>
- * <file-sync-field>
- *
- * -- The normative RCFile implementation included with Hive is actually
- * -- based on a modified version of Hadoop's SequenceFile code. Some
- * -- things which should have been modified were not, including the code
- * -- that writes out the file version header. Consequently, RCFile and
- * -- SequenceFile originally shared the same version header. A newer
- * -- release has created a unique version string.
- *
- * file-version-header ::= Byte[4] {'S', 'E', 'Q', 6}
- * | Byte[4] {'R', 'C', 'F', 1}
- *
- * -- The name of the Java class responsible for reading the key buffer
- * -- component of the rowgroup.
- *
- * file-key-class-name ::=
- * Text {"org.apache.asterix.hivecompat.io.RCFile$KeyBuffer"}
- *
- * -- The name of the Java class responsible for reading the value buffer
- * -- component of the rowgroup.
- *
- * file-value-class-name ::=
- * Text {"org.apache.asterix.hivecompat.io.RCFile$ValueBuffer"}
- *
- * -- Boolean variable indicating whether or not the file uses compression
- * -- for the key and column buffer sections.
- *
- * file-is-compressed ::= Byte[1]
- *
- * -- A boolean field indicating whether or not the file is block compressed.
- * -- This field is *always* false. According to comments in the original
- * -- RCFile implementation this field was retained for backwards
- * -- compatability with the SequenceFile format.
- *
- * file-is-block-compressed ::= Byte[1] {false}
- *
- * -- The Java class name of the compression codec iff <file-is-compressed>
- * -- is true. The named class must implement
- * -- org.apache.hadoop.io.compress.CompressionCodec.
- * -- The expected value is org.apache.hadoop.io.compress.GzipCodec.
- *
- * file-compression-codec-class ::= Text
- *
- * -- A collection of key-value pairs defining metadata values for the
- * -- file. The Map is serialized using standard JDK serialization, i.e.
- * -- an Int corresponding to the number of key-value pairs, followed by
- * -- Text key and value pairs. The following metadata properties are
- * -- mandatory for all RCFiles:
- * --
- * -- hive.io.rcfile.column.number: the number of columns in the RCFile
- *
- * file-header-metadata ::= Map<Text, Text>
- *
- * -- A 16 byte marker that is generated by the writer. This marker appears
- * -- at regular intervals at the beginning of rowgroup-headers, and is
- * -- intended to enable readers to skip over corrupted rowgroups.
- *
- * file-sync-hash ::= Byte[16]
- *
- * -- Each row group is split into three sections: a header, a set of
- * -- key buffers, and a set of column buffers. The header section includes
- * -- an optional sync hash, information about the size of the row group, and
- * -- the total number of rows in the row group. Each key buffer
- * -- consists of run-length encoding data which is used to decode
- * -- the length and offsets of individual fields in the corresponding column
- * -- buffer.
- *
- * rcfile-rowgroup ::=
- * <rowgroup-header>
- * <rowgroup-key-data>
- * <rowgroup-column-buffers>
- *
- * rowgroup-header ::=
- * [<rowgroup-sync-marker>, <rowgroup-sync-hash>]
- * <rowgroup-record-length>
- * <rowgroup-key-length>
- * <rowgroup-compressed-key-length>
- *
- * -- rowgroup-key-data is compressed if the column data is compressed.
- * rowgroup-key-data ::=
- * <rowgroup-num-rows>
- * <rowgroup-key-buffers>
- *
- * -- An integer (always -1) signaling the beginning of a sync-hash
- * -- field.
- *
- * rowgroup-sync-marker ::= Int
- *
- * -- A 16 byte sync field. This must match the <file-sync-hash> value read
- * -- in the file header.
- *
- * rowgroup-sync-hash ::= Byte[16]
- *
- * -- The record-length is the sum of the number of bytes used to store
- * -- the key and column parts, i.e. it is the total length of the current
- * -- rowgroup.
- *
- * rowgroup-record-length ::= Int
- *
- * -- Total length in bytes of the rowgroup's key sections.
- *
- * rowgroup-key-length ::= Int
- *
- * -- Total compressed length in bytes of the rowgroup's key sections.
- *
- * rowgroup-compressed-key-length ::= Int
- *
- * -- Number of rows in the current rowgroup.
- *
- * rowgroup-num-rows ::= VInt
- *
- * -- One or more column key buffers corresponding to each column
- * -- in the RCFile.
- *
- * rowgroup-key-buffers ::= <rowgroup-key-buffer>+
- *
- * -- Data in each column buffer is stored using a run-length
- * -- encoding scheme that is intended to reduce the cost of
- * -- repeated column field values. This mechanism is described
- * -- in more detail in the following entries.
- *
- * rowgroup-key-buffer ::=
- * <column-buffer-length>
- * <column-buffer-uncompressed-length>
- * <column-key-buffer-length>
- * <column-key-buffer>
- *
- * -- The serialized length on disk of the corresponding column buffer.
- *
- * column-buffer-length ::= VInt
- *
- * -- The uncompressed length of the corresponding column buffer. This
- * -- is equivalent to column-buffer-length if the RCFile is not compressed.
- *
- * column-buffer-uncompressed-length ::= VInt
- *
- * -- The length in bytes of the current column key buffer
- *
- * column-key-buffer-length ::= VInt
- *
- * -- The column-key-buffer contains a sequence of serialized VInt values
- * -- corresponding to the byte lengths of the serialized column fields
- * -- in the corresponding rowgroup-column-buffer. For example, consider
- * -- an integer column that contains the consecutive values 1, 2, 3, 44.
- * -- The RCFile format stores these values as strings in the column buffer,
- * -- e.g. "12344". The length of each column field is recorded in
- * -- the column-key-buffer as a sequence of VInts: 1,1,1,2. However,
- * -- if the same length occurs repeatedly, then we replace repeated
- * -- run lengths with the complement (i.e. negative) of the number of
- * -- repetitions, so 1,1,1,2 becomes 1,~2,2.
- *
- * column-key-buffer ::= Byte[column-key-buffer-length]
- *
- * rowgroup-column-buffers ::= <rowgroup-value-buffer>+
- *
- * -- RCFile stores all column data as strings regardless of the
- * -- underlying column type. The strings are neither length-prefixed or
- * -- null-terminated, and decoding them into individual fields requires
- * -- the use of the run-length information contained in the corresponding
- * -- column-key-buffer.
- *
- * rowgroup-column-buffer ::= Byte[column-buffer-length]
- *
- * Byte ::= An eight-bit byte
- *
- * VInt ::= Variable length integer. The high-order bit of each byte
- * indicates whether more bytes remain to be read. The low-order seven
- * bits are appended as increasingly more significant bits in the
- * resulting integer value.
- *
- * Int ::= A four-byte integer in big-endian format.
- *
- * Text ::= VInt, Chars (Length prefixed UTF-8 characters)
- * }
- * </pre>
- * </p>
- */
-public class RCFile {
-
- private static final Log LOG = LogFactory.getLog(RCFile.class);
-
- public static final String RECORD_INTERVAL_CONF_STR = "hive.io.rcfile.record.interval";
-
- public static final String COLUMN_NUMBER_METADATA_STR = "hive.io.rcfile.column.number";
-
- public static final String COLUMN_NUMBER_CONF_STR = "hive.io.rcfile.column.number.conf";
-
- public static final String TOLERATE_CORRUPTIONS_CONF_STR =
- "hive.io.rcfile.tolerate.corruptions";
-
- // HACK: We actually need BlockMissingException, but that is not available
- // in all hadoop versions.
- public static final String BLOCK_MISSING_MESSAGE =
- "Could not obtain block";
-
- // All of the versions should be place in this list.
- private static final int ORIGINAL_VERSION = 0; // version with SEQ
- private static final int NEW_MAGIC_VERSION = 1; // version with RCF
-
- private static final int CURRENT_VERSION = NEW_MAGIC_VERSION;
-
- // The first version of RCFile used the sequence file header.
- private static final byte[] ORIGINAL_MAGIC = new byte[] {
- (byte) 'S', (byte) 'E', (byte) 'Q'};
- // the version that was included with the original magic, which is mapped
- // into ORIGINAL_VERSION
- private static final byte ORIGINAL_MAGIC_VERSION_WITH_METADATA = 6;
-
- private static final byte[] ORIGINAL_MAGIC_VERSION = new byte[] {
- (byte) 'S', (byte) 'E', (byte) 'Q', ORIGINAL_MAGIC_VERSION_WITH_METADATA
- };
-
- // The 'magic' bytes at the beginning of the RCFile
- private static final byte[] MAGIC = new byte[] {
- (byte) 'R', (byte) 'C', (byte) 'F'};
-
- private static final int SYNC_ESCAPE = -1; // "length" of sync entries
- private static final int SYNC_HASH_SIZE = 16; // number of bytes in hash
- private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE; // escape + hash
-
- /** The number of bytes between sync points. */
- public static final int SYNC_INTERVAL = 100 * SYNC_SIZE;
-
- /**
- * KeyBuffer is the key of each record in RCFile. Its on-disk layout is as
- * below:
- *
- * <ul>
- * <li>record length in bytes,it is the sum of bytes used to store the key
- * part and the value part.</li>
- * <li>Key length in bytes, it is how many bytes used by the key part.</li>
- * <li>number_of_rows_in_this_record(vint),</li>
- * <li>column_1_ondisk_length(vint),</li>
- * <li>column_1_row_1_value_plain_length,</li>
- * <li>column_1_row_2_value_plain_length,</li>
- * <li>....</li>
- * <li>column_2_ondisk_length(vint),</li>
- * <li>column_2_row_1_value_plain_length,</li>
- * <li>column_2_row_2_value_plain_length,</li>
- * <li>.... .</li>
- * <li>{the end of the key part}</li>
- * </ul>
- */
- public static class KeyBuffer implements WritableComparable {
- // each column's length in the value
- private int[] eachColumnValueLen = null;
- private int[] eachColumnUncompressedValueLen = null;
- // stores each cell's length of a column in one DataOutputBuffer element
- private NonSyncDataOutputBuffer[] allCellValLenBuffer = null;
- // how many rows in this split
- private int numberRows = 0;
- // how many columns
- private int columnNumber = 0;
-
- // return the number of columns recorded in this file's header
- public int getColumnNumber() {
- return columnNumber;
- }
-
- @SuppressWarnings("unused")
- @Deprecated
- public KeyBuffer(){
- }
-
- KeyBuffer(int columnNum) {
- columnNumber = columnNum;
- eachColumnValueLen = new int[columnNumber];
- eachColumnUncompressedValueLen = new int[columnNumber];
- allCellValLenBuffer = new NonSyncDataOutputBuffer[columnNumber];
- }
-
- @SuppressWarnings("unused")
- @Deprecated
- KeyBuffer(int numberRows, int columnNum) {
- this(columnNum);
- this.numberRows = numberRows;
- }
-
- public void nullColumn(int columnIndex) {
- eachColumnValueLen[columnIndex] = 0;
- eachColumnUncompressedValueLen[columnIndex] = 0;
- allCellValLenBuffer[columnIndex] = new NonSyncDataOutputBuffer();
- }
-
- /**
- * add in a new column's meta data.
- *
- * @param columnValueLen
- * this total bytes number of this column's values in this split
- * @param colValLenBuffer
- * each cell's length of this column's in this split
- */
- void setColumnLenInfo(int columnValueLen,
- NonSyncDataOutputBuffer colValLenBuffer,
- int columnUncompressedValueLen, int columnIndex) {
- eachColumnValueLen[columnIndex] = columnValueLen;
- eachColumnUncompressedValueLen[columnIndex] = columnUncompressedValueLen;
- allCellValLenBuffer[columnIndex] = colValLenBuffer;
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- eachColumnValueLen = new int[columnNumber];
- eachColumnUncompressedValueLen = new int[columnNumber];
- allCellValLenBuffer = new NonSyncDataOutputBuffer[columnNumber];
-
- numberRows = WritableUtils.readVInt(in);
- for (int i = 0; i < columnNumber; i++) {
- eachColumnValueLen[i] = WritableUtils.readVInt(in);
- eachColumnUncompressedValueLen[i] = WritableUtils.readVInt(in);
- int bufLen = WritableUtils.readVInt(in);
- if (allCellValLenBuffer[i] == null) {
- allCellValLenBuffer[i] = new NonSyncDataOutputBuffer();
- } else {
- allCellValLenBuffer[i].reset();
- }
- allCellValLenBuffer[i].write(in, bufLen);
- }
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- // out.writeInt(numberRows);
- WritableUtils.writeVLong(out, numberRows);
- for (int i = 0; i < eachColumnValueLen.length; i++) {
- WritableUtils.writeVLong(out, eachColumnValueLen[i]);
- WritableUtils.writeVLong(out, eachColumnUncompressedValueLen[i]);
- NonSyncDataOutputBuffer colRowsLenBuf = allCellValLenBuffer[i];
- int bufLen = colRowsLenBuf.getLength();
- WritableUtils.writeVLong(out, bufLen);
- out.write(colRowsLenBuf.getData(), 0, bufLen);
- }
- }
-
- /**
- * get number of bytes to store the keyBuffer.
- *
- * @return number of bytes used to store this KeyBuffer on disk
- * @throws IOException
- */
- public int getSize() throws IOException {
- int ret = 0;
- ret += WritableUtils.getVIntSize(numberRows);
- for (int i = 0; i < eachColumnValueLen.length; i++) {
- ret += WritableUtils.getVIntSize(eachColumnValueLen[i]);
- ret += WritableUtils.getVIntSize(eachColumnUncompressedValueLen[i]);
- ret += WritableUtils.getVIntSize(allCellValLenBuffer[i].getLength());
- ret += allCellValLenBuffer[i].getLength();
- }
-
- return ret;
- }
-
- @Override
- public int compareTo(Object arg0) {
- throw new RuntimeException("compareTo not supported in class "
- + this.getClass().getName());
- }
-
- public int[] getEachColumnUncompressedValueLen() {
- return eachColumnUncompressedValueLen;
- }
-
- public int[] getEachColumnValueLen() {
- return eachColumnValueLen;
- }
-
- /**
- * @return the numberRows
- */
- public int getNumberRows() {
- return numberRows;
- }
- }
-
- /**
- * ValueBuffer is the value of each record in RCFile. Its on-disk layout is as
- * below:
- * <ul>
- * <li>Compressed or plain data of [column_1_row_1_value,
- * column_1_row_2_value,....]</li>
- * <li>Compressed or plain data of [column_2_row_1_value,
- * column_2_row_2_value,....]</li>
- * </ul>
- */
- public static class ValueBuffer implements WritableComparable {
-
- class LazyDecompressionCallbackImpl implements LazyDecompressionCallback {
-
- int index = -1;
- int colIndex = -1;
-
- public LazyDecompressionCallbackImpl(int index, int colIndex) {
- super();
- this.index = index;
- this.colIndex = colIndex;
- }
-
- @Override
- public byte[] decompress() throws IOException {
-
- if (decompressedFlag[index] || codec == null) {
- return loadedColumnsValueBuffer[index].getData();
- }
-
- NonSyncDataOutputBuffer compressedData = compressedColumnsValueBuffer[index];
- decompressBuffer.reset();
- DataInputStream valueIn = new DataInputStream(deflatFilter);
- deflatFilter.resetState();
- if (deflatFilter instanceof SchemaAwareCompressionInputStream) {
- ((SchemaAwareCompressionInputStream)deflatFilter).setColumnIndex(colIndex);
- }
- decompressBuffer.reset(compressedData.getData(),
- keyBuffer.eachColumnValueLen[colIndex]);
-
- NonSyncDataOutputBuffer decompressedColBuf = loadedColumnsValueBuffer[index];
- decompressedColBuf.reset();
- decompressedColBuf.write(valueIn,
- keyBuffer.eachColumnUncompressedValueLen[colIndex]);
- decompressedFlag[index] = true;
- numCompressed--;
- return decompressedColBuf.getData();
- }
- }
-
- // used to load columns' value into memory
- private NonSyncDataOutputBuffer[] loadedColumnsValueBuffer = null;
- private NonSyncDataOutputBuffer[] compressedColumnsValueBuffer = null;
- private boolean[] decompressedFlag = null;
- private int numCompressed;
- private LazyDecompressionCallbackImpl[] lazyDecompressCallbackObjs = null;
- private boolean lazyDecompress = true;
-
- boolean inited = false;
-
- // used for readFields
- KeyBuffer keyBuffer;
- private int columnNumber = 0;
-
- // set true for columns that needed to skip loading into memory.
- boolean[] skippedColIDs = null;
-
- CompressionCodec codec;
-
- Decompressor valDecompressor = null;
- NonSyncDataInputBuffer decompressBuffer = new NonSyncDataInputBuffer();
- CompressionInputStream deflatFilter = null;
-
- @SuppressWarnings("unused")
- @Deprecated
- public ValueBuffer() throws IOException {
- }
-
- @SuppressWarnings("unused")
- @Deprecated
- public ValueBuffer(KeyBuffer keyBuffer) throws IOException {
- this(keyBuffer, keyBuffer.columnNumber, null, null, true);
- }
-
- @SuppressWarnings("unused")
- @Deprecated
- public ValueBuffer(KeyBuffer keyBuffer, boolean[] skippedColIDs)
- throws IOException {
- this(keyBuffer, keyBuffer.columnNumber, skippedColIDs, null, true);
- }
-
- @SuppressWarnings("unused")
- @Deprecated
- public ValueBuffer(KeyBuffer currentKey, int columnNumber,
- boolean[] skippedCols, CompressionCodec codec) throws IOException {
- this(currentKey, columnNumber, skippedCols, codec, true);
- }
-
- public ValueBuffer(KeyBuffer currentKey, int columnNumber,
- boolean[] skippedCols, CompressionCodec codec, boolean lazyDecompress)
- throws IOException {
- this.lazyDecompress = lazyDecompress;
- keyBuffer = currentKey;
- this.columnNumber = columnNumber;
-
- if (skippedCols != null && skippedCols.length > 0) {
- skippedColIDs = skippedCols;
- } else {
- skippedColIDs = new boolean[columnNumber];
- for (int i = 0; i < skippedColIDs.length; i++) {
- skippedColIDs[i] = false;
- }
- }
-
- int skipped = 0;
- for (boolean currentSkip : skippedColIDs) {
- if (currentSkip) {
- skipped++;
- }
- }
- loadedColumnsValueBuffer = new NonSyncDataOutputBuffer[columnNumber
- - skipped];
- decompressedFlag = new boolean[columnNumber - skipped];
- lazyDecompressCallbackObjs = new LazyDecompressionCallbackImpl[columnNumber
- - skipped];
- compressedColumnsValueBuffer = new NonSyncDataOutputBuffer[columnNumber
- - skipped];
- this.codec = codec;
- if (codec != null) {
- valDecompressor = CodecPool.getDecompressor(codec);
- deflatFilter = codec.createInputStream(decompressBuffer,
- valDecompressor);
- }
- if (codec != null) {
- numCompressed = decompressedFlag.length;
- } else {
- numCompressed = 0;
- }
- for (int k = 0, readIndex = 0; k < columnNumber; k++) {
- if (skippedColIDs[k]) {
- continue;
- }
- loadedColumnsValueBuffer[readIndex] = new NonSyncDataOutputBuffer();
- if (codec != null) {
- decompressedFlag[readIndex] = false;
- lazyDecompressCallbackObjs[readIndex] = new LazyDecompressionCallbackImpl(
- readIndex, k);
- compressedColumnsValueBuffer[readIndex] = new NonSyncDataOutputBuffer();
- } else {
- decompressedFlag[readIndex] = true;
- }
- readIndex++;
- }
- }
-
- @SuppressWarnings("unused")
- @Deprecated
- public void setColumnValueBuffer(NonSyncDataOutputBuffer valBuffer,
- int addIndex) {
- loadedColumnsValueBuffer[addIndex] = valBuffer;
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- int addIndex = 0;
- int skipTotal = 0;
- for (int i = 0; i < columnNumber; i++) {
- int vaRowsLen = keyBuffer.eachColumnValueLen[i];
- // skip this column
- if (skippedColIDs[i]) {
- skipTotal += vaRowsLen;
- continue;
- }
-
- if (skipTotal != 0) {
- in.skipBytes(skipTotal);
- skipTotal = 0;
- }
-
- NonSyncDataOutputBuffer valBuf;
- if (codec != null){
- // load into compressed buf first
- valBuf = compressedColumnsValueBuffer[addIndex];
- } else {
- valBuf = loadedColumnsValueBuffer[addIndex];
- }
- valBuf.reset();
- valBuf.write(in, vaRowsLen);
- if (codec != null) {
- decompressedFlag[addIndex] = false;
- if (!lazyDecompress) {
- lazyDecompressCallbackObjs[addIndex].decompress();
- decompressedFlag[addIndex] = true;
- }
- }
- addIndex++;
- }
- if (codec != null) {
- numCompressed = decompressedFlag.length;
- }
-
- if (skipTotal != 0) {
- in.skipBytes(skipTotal);
- }
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- if (codec != null) {
- for (NonSyncDataOutputBuffer currentBuf : compressedColumnsValueBuffer) {
- out.write(currentBuf.getData(), 0, currentBuf.getLength());
- }
- } else {
- for (NonSyncDataOutputBuffer currentBuf : loadedColumnsValueBuffer) {
- out.write(currentBuf.getData(), 0, currentBuf.getLength());
- }
- }
- }
-
- public void nullColumn(int columnIndex) {
- if (codec != null) {
- compressedColumnsValueBuffer[columnIndex].reset();
- } else {
- loadedColumnsValueBuffer[columnIndex].reset();
- }
- }
-
- public void clearColumnBuffer() throws IOException {
- decompressBuffer.reset();
- }
-
- public void close() {
- for (NonSyncDataOutputBuffer element : loadedColumnsValueBuffer) {
- IOUtils.closeStream(element);
- }
- if (codec != null) {
- IOUtils.closeStream(decompressBuffer);
- if (valDecompressor != null) {
- // Make sure we only return valDecompressor once.
- CodecPool.returnDecompressor(valDecompressor);
- valDecompressor = null;
- }
- }
- }
-
- @Override
- public int compareTo(Object arg0) {
- throw new RuntimeException("compareTo not supported in class "
- + this.getClass().getName());
- }
- }
-
- /**
- * Create a metadata object with alternating key-value pairs.
- * Eg. metadata(key1, value1, key2, value2)
- */
- public static Metadata createMetadata(Text... values) {
- if (values.length % 2 != 0) {
- throw new IllegalArgumentException("Must have a matched set of " +
- "key-value pairs. " + values.length+
- " strings supplied.");
- }
- Metadata result = new Metadata();
- for(int i=0; i < values.length; i += 2) {
- result.set(values[i], values[i+1]);
- }
- return result;
- }
-
- /**
- * Write KeyBuffer/ValueBuffer pairs to a RCFile. RCFile's format is
- * compatible with SequenceFile's.
- *
- */
- public static class Writer {
-
- Configuration conf;
- FSDataOutputStream out;
-
- CompressionCodec codec = null;
- Metadata metadata = null;
-
- // Insert a globally unique 16-byte value every few entries, so that one
- // can seek into the middle of a file and then synchronize with record
- // starts and ends by scanning for this value.
- long lastSyncPos; // position of last sync
- byte[] sync; // 16 random bytes
- {
- try {
- MessageDigest digester = MessageDigest.getInstance("MD5");
- long time = System.currentTimeMillis();
- digester.update((new UID() + "@" + time).getBytes());
- sync = digester.digest();
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- // how many records the writer buffers before it writes to disk
- private int RECORD_INTERVAL = Integer.MAX_VALUE;
- // the max size of memory for buffering records before writes them out
- private int columnsBufferSize = 4 * 1024 * 1024; // 4M
- // the conf string for COLUMNS_BUFFER_SIZE
- public static String COLUMNS_BUFFER_SIZE_CONF_STR = "hive.io.rcfile.record.buffer.size";
-
- // how many records already buffered
- private int bufferedRecords = 0;
-
- private final ColumnBuffer[] columnBuffers;
-
- private int columnNumber = 0;
-
- private final int[] columnValuePlainLength;
-
- KeyBuffer key = null;
- private final int[] plainTotalColumnLength;
- private final int[] comprTotalColumnLength;
-
- boolean useNewMagic = true;
-
- /*
- * used for buffering appends before flush them out
- */
- class ColumnBuffer {
- // used for buffer a column's values
- NonSyncDataOutputBuffer columnValBuffer;
- // used to store each value's length
- NonSyncDataOutputBuffer valLenBuffer;
-
- /*
- * use a run-length encoding. We only record run length if a same
- * 'prevValueLen' occurs more than one time. And we negative the run
- * length to distinguish a runLength and a normal value length. For
- * example, if the values' lengths are 1,1,1,2, we record 1, ~2,2. And for
- * value lengths 1,2,3 we record 1,2,3.
- */
- int runLength = 0;
- int prevValueLength = -1;
-
- ColumnBuffer() throws IOException {
- columnValBuffer = new NonSyncDataOutputBuffer();
- valLenBuffer = new NonSyncDataOutputBuffer();
- }
-
- public void append(BytesRefWritable data) throws IOException {
- data.writeDataTo(columnValBuffer);
- int currentLen = data.getLength();
-
- if (prevValueLength < 0) {
- startNewGroup(currentLen);
- return;
- }
-
- if (currentLen != prevValueLength) {
- flushGroup();
- startNewGroup(currentLen);
- } else {
- runLength++;
- }
- }
-
- private void startNewGroup(int currentLen) {
- prevValueLength = currentLen;
- runLength = 0;
- }
-
- public void clear() throws IOException {
- valLenBuffer.reset();
- columnValBuffer.reset();
- prevValueLength = -1;
- runLength = 0;
- }
-
- public void flushGroup() throws IOException {
- if (prevValueLength >= 0) {
- WritableUtils.writeVLong(valLenBuffer, prevValueLength);
- if (runLength > 0) {
- WritableUtils.writeVLong(valLenBuffer, ~runLength);
- }
- runLength = -1;
- prevValueLength = -1;
- }
- }
- }
-
- public long getLength() throws IOException {
- return out.getPos();
- }
-
- /** Constructs a RCFile Writer. */
- public Writer(FileSystem fs, Configuration conf, Path name) throws IOException {
- this(fs, conf, name, null, new Metadata(), null);
- }
-
- /**
- * Constructs a RCFile Writer.
- *
- * @param fs
- * the file system used
- * @param conf
- * the configuration file
- * @param name
- * the file name
- * @throws IOException
- */
- public Writer(FileSystem fs, Configuration conf, Path name,
- Progressable progress, CompressionCodec codec) throws IOException {
- this(fs, conf, name, progress, new Metadata(), codec);
- }
-
- /**
- * Constructs a RCFile Writer.
- *
- * @param fs
- * the file system used
- * @param conf
- * the configuration file
- * @param name
- * the file name
- * @param progress a progress meter to update as the file is written
- * @param metadata a string to string map in the file header
- * @throws IOException
- */
- public Writer(FileSystem fs, Configuration conf, Path name,
- Progressable progress, Metadata metadata, CompressionCodec codec) throws IOException {
- this(fs, conf, name, fs.getConf().getInt("io.file.buffer.size", 4096),
- ShimLoader.getHadoopShims().getDefaultReplication(fs, name),
- ShimLoader.getHadoopShims().getDefaultBlockSize(fs, name), progress,
- metadata, codec);
- }
-
- /**
- *
- * Constructs a RCFile Writer.
- *
- * @param fs
- * the file system used
- * @param conf
- * the configuration file
- * @param name
- * the file name
- * @param bufferSize the size of the file buffer
- * @param replication the number of replicas for the file
- * @param blockSize the block size of the file
- * @param progress the progress meter for writing the file
- * @param metadata a string to string map in the file header
- * @throws IOException
- */
- public Writer(FileSystem fs, Configuration conf, Path name, int bufferSize,
- short replication, long blockSize, Progressable progress,
- Metadata metadata, CompressionCodec codec) throws IOException {
- RECORD_INTERVAL = conf.getInt(RECORD_INTERVAL_CONF_STR, RECORD_INTERVAL);
- columnNumber = conf.getInt(COLUMN_NUMBER_CONF_STR, 0);
-
- if (metadata == null) {
- metadata = new Metadata();
- }
- metadata.set(new Text(COLUMN_NUMBER_METADATA_STR), new Text(""
- + columnNumber));
-
- columnsBufferSize = conf.getInt(COLUMNS_BUFFER_SIZE_CONF_STR,
- 4 * 1024 * 1024);
-
- columnValuePlainLength = new int[columnNumber];
-
- columnBuffers = new ColumnBuffer[columnNumber];
- for (int i = 0; i < columnNumber; i++) {
- columnBuffers[i] = new ColumnBuffer();
- }
-
- init(conf, fs.create(name, true, bufferSize, replication,
- blockSize, progress), codec, metadata);
- initializeFileHeader();
- writeFileHeader();
- finalizeFileHeader();
- key = new KeyBuffer(columnNumber);
-
- plainTotalColumnLength = new int[columnNumber];
- comprTotalColumnLength = new int[columnNumber];
- }
-
- /** Write the initial part of file header. */
- void initializeFileHeader() throws IOException {
- if (useNewMagic) {
- out.write(MAGIC);
- out.write(CURRENT_VERSION);
- } else {
- out.write(ORIGINAL_MAGIC_VERSION);
- }
- }
-
- /** Write the final part of file header. */
- void finalizeFileHeader() throws IOException {
- out.write(sync); // write the sync bytes
- out.flush(); // flush header
- }
-
- boolean isCompressed() {
- return codec != null;
- }
-
- /** Write and flush the file header. */
- void writeFileHeader() throws IOException {
- if (useNewMagic) {
- out.writeBoolean(isCompressed());
- } else {
- Text.writeString(out, KeyBuffer.class.getName());
- Text.writeString(out, ValueBuffer.class.getName());
- out.writeBoolean(isCompressed());
- out.writeBoolean(false);
- }
-
- if (isCompressed()) {
- Text.writeString(out, (codec.getClass()).getName());
- }
- metadata.write(out);
- }
-
- void init(Configuration conf, FSDataOutputStream out,
- CompressionCodec codec, Metadata metadata) throws IOException {
- this.conf = conf;
- this.out = out;
- this.codec = codec;
- this.metadata = metadata;
- this.useNewMagic =
- conf.getBoolean(HiveConf.ConfVars.HIVEUSEEXPLICITRCFILEHEADER.varname, true);
- }
-
- /** Returns the compression codec of data in this file. */
- @SuppressWarnings("unused")
- @Deprecated
- public CompressionCodec getCompressionCodec() {
- return codec;
- }
-
- /** create a sync point. */
- public void sync() throws IOException {
- if (sync != null && lastSyncPos != out.getPos()) {
- out.writeInt(SYNC_ESCAPE); // mark the start of the sync
- out.write(sync); // write sync
- lastSyncPos = out.getPos(); // update lastSyncPos
- }
- }
-
- /** Returns the configuration of this file. */
- @SuppressWarnings("unused")
- @Deprecated
- Configuration getConf() {
- return conf;
- }
-
- private void checkAndWriteSync() throws IOException {
- if (sync != null && out.getPos() >= lastSyncPos + SYNC_INTERVAL) {
- sync();
- }
- }
-
- private int columnBufferSize = 0;
-
- /**
- * Append a row of values. Currently it only can accept <
- * {@link BytesRefArrayWritable}. If its <code>size()</code> is less than the
- * column number in the file, zero bytes are appended for the empty columns.
- * If its size() is greater then the column number in the file, the exceeded
- * columns' bytes are ignored.
- *
- * @param val a BytesRefArrayWritable with the list of serialized columns
- * @throws IOException
- */
- public void append(Writable val) throws IOException {
-
- if (!(val instanceof BytesRefArrayWritable)) {
- throw new UnsupportedOperationException(
- "Currently the writer can only accept BytesRefArrayWritable");
- }
-
- BytesRefArrayWritable columns = (BytesRefArrayWritable) val;
- int size = columns.size();
- for (int i = 0; i < size; i++) {
- BytesRefWritable cu = columns.get(i);
- int plainLen = cu.getLength();
- columnBufferSize += plainLen;
- columnValuePlainLength[i] += plainLen;
- columnBuffers[i].append(cu);
- }
-
- if (size < columnNumber) {
- for (int i = columns.size(); i < columnNumber; i++) {
- columnBuffers[i].append(BytesRefWritable.ZeroBytesRefWritable);
- }
- }
-
- bufferedRecords++;
- if ((columnBufferSize > columnsBufferSize)
- || (bufferedRecords >= RECORD_INTERVAL)) {
- flushRecords();
- }
- }
-
- private void flushRecords() throws IOException {
-
- key.numberRows = bufferedRecords;
-
- Compressor compressor = null;
- NonSyncDataOutputBuffer valueBuffer = null;
- CompressionOutputStream deflateFilter = null;
- DataOutputStream deflateOut = null;
- boolean isCompressed = isCompressed();
- int valueLength = 0;
- if (isCompressed) {
- ReflectionUtils.setConf(codec, this.conf);
- compressor = CodecPool.getCompressor(codec);
- valueBuffer = new NonSyncDataOutputBuffer();
- deflateFilter = codec.createOutputStream(valueBuffer, compressor);
- deflateOut = new DataOutputStream(deflateFilter);
- }
-
- for (int columnIndex = 0; columnIndex < columnNumber; columnIndex++) {
- ColumnBuffer currentBuf = columnBuffers[columnIndex];
- currentBuf.flushGroup();
-
- NonSyncDataOutputBuffer columnValue = currentBuf.columnValBuffer;
- int colLen;
- int plainLen = columnValuePlainLength[columnIndex];
-
- if (isCompressed) {
- if (deflateFilter instanceof SchemaAwareCompressionOutputStream) {
- ((SchemaAwareCompressionOutputStream)deflateFilter).
- setColumnIndex(columnIndex);
- }
- deflateFilter.resetState();
- deflateOut.write(columnValue.getData(), 0, columnValue.getLength());
- deflateOut.flush();
- deflateFilter.finish();
- // find how much compressed data was added for this column
- colLen = valueBuffer.getLength() - valueLength;
- } else {
- colLen = columnValuePlainLength[columnIndex];
- }
- valueLength += colLen;
- key.setColumnLenInfo(colLen, currentBuf.valLenBuffer, plainLen,
- columnIndex);
- plainTotalColumnLength[columnIndex] += plainLen;
- comprTotalColumnLength[columnIndex] += colLen;
- columnValuePlainLength[columnIndex] = 0;
- }
-
- int keyLength = key.getSize();
- if (keyLength < 0) {
- throw new IOException("negative length keys not allowed: " + key);
- }
- if (compressor != null) {
- CodecPool.returnCompressor(compressor);
- }
-
- // Write the key out
- writeKey(key, keyLength + valueLength, keyLength);
- // write the value out
- if (isCompressed) {
- out.write(valueBuffer.getData(), 0, valueBuffer.getLength());
- } else {
- for(int columnIndex=0; columnIndex < columnNumber; ++columnIndex) {
- NonSyncDataOutputBuffer buf =
- columnBuffers[columnIndex].columnValBuffer;
- out.write(buf.getData(), 0, buf.getLength());
- }
- }
-
- // clear the columnBuffers
- clearColumnBuffers();
-
- bufferedRecords = 0;
- columnBufferSize = 0;
- }
-
- /**
- * flush a block out without doing anything except compressing the key part.
- */
- public void flushBlock(KeyBuffer keyBuffer, ValueBuffer valueBuffer,
- int recordLen, int keyLength,
- @SuppressWarnings("unused") int compressedKeyLen) throws IOException {
- writeKey(keyBuffer, recordLen, keyLength);
- valueBuffer.write(out);
- }
-
- private void writeKey(KeyBuffer keyBuffer, int recordLen,
- int keyLength) throws IOException {
- checkAndWriteSync(); // sync
- out.writeInt(recordLen); // total record length
- out.writeInt(keyLength); // key portion length
-
- if(this.isCompressed()) {
- Compressor compressor = CodecPool.getCompressor(codec);
- NonSyncDataOutputBuffer compressionBuffer =
- new NonSyncDataOutputBuffer();
- CompressionOutputStream deflateFilter =
- codec.createOutputStream(compressionBuffer, compressor);
- DataOutputStream deflateOut = new DataOutputStream(deflateFilter);
- //compress key and write key out
- compressionBuffer.reset();
- deflateFilter.resetState();
- keyBuffer.write(deflateOut);
- deflateOut.flush();
- deflateFilter.finish();
- int compressedKeyLen = compressionBuffer.getLength();
- out.writeInt(compressedKeyLen);
- out.write(compressionBuffer.getData(), 0, compressedKeyLen);
- CodecPool.returnCompressor(compressor);
- } else {
- out.writeInt(keyLength);
- keyBuffer.write(out);
- }
- }
-
- private void clearColumnBuffers() throws IOException {
- for (int i = 0; i < columnNumber; i++) {
- columnBuffers[i].clear();
- }
- }
-
- public synchronized void close() throws IOException {
- if (bufferedRecords > 0) {
- flushRecords();
- }
- clearColumnBuffers();
-
- if (out != null) {
-
- // Close the underlying stream if we own it...
- out.flush();
- out.close();
- out = null;
- }
- for (int i = 0; i < columnNumber; i++) {
- LOG.info("Column#" + i + " : Plain Total Column Value Length: "
- + plainTotalColumnLength[i]
- + ", Compr Total Column Value Length: " + comprTotalColumnLength[i]);
- }
- }
- }
-
- /**
- * Read KeyBuffer/ValueBuffer pairs from a RCFile.
- *
- */
- public static class Reader {
- private static class SelectedColumn {
- public int colIndex;
- public int rowReadIndex;
- public int runLength;
- public int prvLength;
- public boolean isNulled;
- }
- private final Path file;
- private final FSDataInputStream in;
-
- private byte version;
-
- private CompressionCodec codec = null;
- private Metadata metadata = null;
-
- private final byte[] sync = new byte[SYNC_HASH_SIZE];
- private final byte[] syncCheck = new byte[SYNC_HASH_SIZE];
- private boolean syncSeen;
- private long lastSeenSyncPos = 0;
-
- private long headerEnd;
- private final long end;
- private int currentKeyLength;
- private int currentRecordLength;
-
- private final Configuration conf;
-
- private final ValueBuffer currentValue;
-
- private int readRowsIndexInBuffer = 0;
-
- private int recordsNumInValBuffer = 0;
-
- private int columnNumber = 0;
-
- private int loadColumnNum;
-
- private int passedRowsNum = 0;
-
- // Should we try to tolerate corruption? Default is No.
- private boolean tolerateCorruptions = false;
-
- private boolean decompress = false;
-
- private Decompressor keyDecompressor;
- NonSyncDataOutputBuffer keyDecompressedData = new NonSyncDataOutputBuffer();
-
- //Current state of each selected column - e.g. current run length, etc.
- // The size of the array is equal to the number of selected columns
- private final SelectedColumn[] selectedColumns;
-
- // map of original column id -> index among selected columns
- private final int[] revPrjColIDs;
-
- // column value lengths for each of the selected columns
- private final NonSyncDataInputBuffer[] colValLenBufferReadIn;
-
- /** Create a new RCFile reader. */
- public Reader(FileSystem fs, Path file, Configuration conf) throws IOException {
- this(fs, file, conf.getInt("io.file.buffer.size", 4096), conf, 0, fs
- .getFileStatus(file).getLen());
- }
-
- /** Create a new RCFile reader. */
- public Reader(FileSystem fs, Path file, int bufferSize, Configuration conf,
- long start, long length) throws IOException {
- tolerateCorruptions = conf.getBoolean(
- TOLERATE_CORRUPTIONS_CONF_STR, false);
- conf.setInt("io.file.buffer.size", bufferSize);
- this.file = file;
- in = openFile(fs, file, bufferSize, length);
- this.conf = conf;
- end = start + length;
- boolean succeed = false;
- try {
- if (start > 0) {
- seek(0);
- init();
- seek(start);
- } else {
- init();
- }
- succeed = true;
- } finally {
- if (!succeed) {
- if (in != null) {
- try {
- in.close();
- } catch(IOException e) {
- if (LOG != null && LOG.isDebugEnabled()) {
- LOG.debug("Exception in closing " + in, e);
- }
- }
- }
- }
- }
-
- columnNumber = Integer.parseInt(metadata.get(
- new Text(COLUMN_NUMBER_METADATA_STR)).toString());
-
- List<Integer> notSkipIDs = ColumnProjectionUtils
- .getReadColumnIDs(conf);
- boolean[] skippedColIDs = new boolean[columnNumber];
- if(ColumnProjectionUtils.isReadAllColumns(conf)) {
- Arrays.fill(skippedColIDs, false);
- } else if (notSkipIDs.size() > 0) {
- Arrays.fill(skippedColIDs, true);
- for (int read : notSkipIDs) {
- if (read < columnNumber) {
- skippedColIDs[read] = false;
- }
- }
- } else {
- // select count(1)
- Arrays.fill(skippedColIDs, true);
- }
- loadColumnNum = columnNumber;
- if (skippedColIDs.length > 0) {
- for (boolean skippedColID : skippedColIDs) {
- if (skippedColID) {
- loadColumnNum -= 1;
- }
- }
- }
-
-
- revPrjColIDs = new int[columnNumber];
- // get list of selected column IDs
- selectedColumns = new SelectedColumn[loadColumnNum];
- colValLenBufferReadIn = new NonSyncDataInputBuffer[loadColumnNum];
- for (int i = 0, j = 0; i < columnNumber; ++i) {
- if (!skippedColIDs[i]) {
- SelectedColumn col = new SelectedColumn();
- col.colIndex = i;
- col.runLength = 0;
- col.prvLength = -1;
- col.rowReadIndex = 0;
- selectedColumns[j] = col;
- colValLenBufferReadIn[j] = new NonSyncDataInputBuffer();
- revPrjColIDs[i] = j;
- j++;
- } else {
- revPrjColIDs[i] = -1;
- }
- }
-
- currentKey = createKeyBuffer();
- boolean lazyDecompress = !tolerateCorruptions;
- currentValue = new ValueBuffer(
- null, columnNumber, skippedColIDs, codec, lazyDecompress);
- }
-
- /**
- * Return the metadata (Text to Text map) that was written into the
- * file.
- */
- public Metadata getMetadata() {
- return metadata;
- }
-
- /**
- * Return the metadata value associated with the given key.
- * @param key the metadata key to retrieve
- */
- public Text getMetadataValueOf(Text key) {
- return metadata.get(key);
- }
-
- /**
- * Override this method to specialize the type of
- * {@link FSDataInputStream} returned.
- */
- protected FSDataInputStream openFile(FileSystem fs, Path file,
- int bufferSize, long length) throws IOException {
- return fs.open(file, bufferSize);
- }
-
- private void init() throws IOException {
- byte[] magic = new byte[MAGIC.length];
- in.readFully(magic);
-
- if (Arrays.equals(magic, ORIGINAL_MAGIC)) {
- byte vers = in.readByte();
- if (vers != ORIGINAL_MAGIC_VERSION_WITH_METADATA) {
- throw new IOException(file + " is a version " + vers +
- " SequenceFile instead of an RCFile.");
- }
- version = ORIGINAL_VERSION;
- } else {
- if (!Arrays.equals(magic, MAGIC)) {
- throw new IOException(file + " not a RCFile and has magic of " +
- new String(magic));
- }
-
- // Set 'version'
- version = in.readByte();
- if (version > CURRENT_VERSION) {
- throw new VersionMismatchException((byte) CURRENT_VERSION, version);
- }
- }
-
- if (version == ORIGINAL_VERSION) {
- try {
- Class<?> keyCls = conf.getClassByName(Text.readString(in));
- Class<?> valCls = conf.getClassByName(Text.readString(in));
- if (!keyCls.equals(KeyBuffer.class)
- || !valCls.equals(ValueBuffer.class)) {
- throw new IOException(file + " not a RCFile");
- }
- } catch (ClassNotFoundException e) {
- throw new IOException(file + " not a RCFile", e);
- }
- }
-
- decompress = in.readBoolean(); // is compressed?
-
- if (version == ORIGINAL_VERSION) {
- // is block-compressed? it should be always false.
- boolean blkCompressed = in.readBoolean();
- if (blkCompressed) {
- throw new IOException(file + " not a RCFile.");
- }
- }
-
- // setup the compression codec
- if (decompress) {
- String codecClassname = Text.readString(in);
- try {
- Class<? extends CompressionCodec> codecClass = conf.getClassByName(
- codecClassname).asSubclass(CompressionCodec.class);
- codec = ReflectionUtils.newInstance(codecClass, conf);
- } catch (ClassNotFoundException cnfe) {
- throw new IllegalArgumentException(
- "Unknown codec: " + codecClassname, cnfe);
- }
- keyDecompressor = CodecPool.getDecompressor(codec);
- }
-
- metadata = new Metadata();
- metadata.readFields(in);
-
- in.readFully(sync); // read sync bytes
- headerEnd = in.getPos();
- }
-
- /** Return the current byte position in the input file. */
- public synchronized long getPosition() throws IOException {
- return in.getPos();
- }
-
- /**
- * Set the current byte position in the input file.
- *
- * <p>
- * The position passed must be a position returned by
- * {@link RCFile.Writer#getLength()} when writing this file. To seek to an
- * arbitrary position, use {@link RCFile.Reader#sync(long)}. In another
- * words, the current seek can only seek to the end of the file. For other
- * positions, use {@link RCFile.Reader#sync(long)}.
- */
- public synchronized void seek(long position) throws IOException {
- in.seek(position);
- }
-
- /**
- * Resets the values which determine if there are more rows in the buffer
- *
- * This can be used after one calls seek or sync, if one called next before that.
- * Otherwise, the seek or sync will have no effect, it will continue to get rows from the
- * buffer built up from the call to next.
- */
- public synchronized void resetBuffer() {
- readRowsIndexInBuffer = 0;
- recordsNumInValBuffer = 0;
- }
-
- /** Seek to the next sync mark past a given position. */
- public synchronized void sync(long position) throws IOException {
- if (position + SYNC_SIZE >= end) {
- seek(end);
- return;
- }
-
- //this is to handle syn(pos) where pos < headerEnd.
- if (position < headerEnd) {
- // seek directly to first record
- in.seek(headerEnd);
- // note the sync marker "seen" in the header
- syncSeen = true;
- return;
- }
-
- try {
- seek(position + 4); // skip escape
-
- int prefix = sync.length;
- int n = conf.getInt("io.bytes.per.checksum", 512);
- byte[] buffer = new byte[prefix+n];
- n = (int)Math.min(n, end - in.getPos());
- /* fill array with a pattern that will never match sync */
- Arrays.fill(buffer, (byte)(~sync[0]));
- while(n > 0 && (in.getPos() + n) <= end) {
- position = in.getPos();
- in.readFully(buffer, prefix, n);
- /* the buffer has n+sync bytes */
- for(int i = 0; i < n; i++) {
- int j;
- for(j = 0; j < sync.length && sync[j] == buffer[i+j]; j++) {
- /* nothing */
- }
- if(j == sync.length) {
- /* simplified from (position + (i - prefix) + sync.length) - SYNC_SIZE */
- in.seek(position + i - SYNC_SIZE);
- return;
- }
- }
- /* move the last 16 bytes to the prefix area */
- System.arraycopy(buffer, buffer.length - prefix, buffer, 0, prefix);
- n = (int)Math.min(n, end - in.getPos());
- }
- } catch (ChecksumException e) { // checksum failure
- handleChecksumException(e);
- }
- }
-
- private void handleChecksumException(ChecksumException e) throws IOException {
- if (conf.getBoolean("io.skip.checksum.errors", false)) {
- LOG.warn("Bad checksum at " + getPosition() + ". Skipping entries.");
- sync(getPosition() + conf.getInt("io.bytes.per.checksum", 512));
- } else {
- throw e;
- }
- }
-
- private KeyBuffer createKeyBuffer() {
- return new KeyBuffer(columnNumber);
- }
-
- /**
- * Read and return the next record length, potentially skipping over a sync
- * block.
- *
- * @return the length of the next record or -1 if there is no next record
- * @throws IOException
- */
- private synchronized int readRecordLength() throws IOException {
- if (in.getPos() >= end) {
- return -1;
- }
- int length = in.readInt();
- if (sync != null && length == SYNC_ESCAPE) { // process
- // a
- // sync entry
- lastSeenSyncPos = in.getPos() - 4; // minus SYNC_ESCAPE's length
- in.readFully(syncCheck); // read syncCheck
- if (!Arrays.equals(sync, syncCheck)) {
- throw new IOException("File is corrupt!");
- }
- syncSeen = true;
- if (in.getPos() >= end) {
- return -1;
- }
- length = in.readInt(); // re-read length
- } else {
- syncSeen = false;
- }
- return length;
- }
-
- private void seekToNextKeyBuffer() throws IOException {
- if (!keyInit) {
- return;
- }
- if (!currentValue.inited) {
- in.skip(currentRecordLength - currentKeyLength);
- }
- }
-
- private int compressedKeyLen = 0;
- NonSyncDataInputBuffer keyDataIn = new NonSyncDataInputBuffer();
- NonSyncDataInputBuffer keyDecompressBuffer = new NonSyncDataInputBuffer();
- NonSyncDataOutputBuffer keyTempBuffer = new NonSyncDataOutputBuffer();
-
- KeyBuffer currentKey = null;
- boolean keyInit = false;
-
- protected int nextKeyBuffer() throws IOException {
- seekToNextKeyBuffer();
- currentRecordLength = readRecordLength();
- if (currentRecordLength == -1) {
- keyInit = false;
- return -1;
- }
- currentKeyLength = in.readInt();
- compressedKeyLen = in.readInt();
- if (decompress) {
- keyTempBuffer.reset();
- keyTempBuffer.write(in, compressedKeyLen);
- keyDecompressBuffer.reset(keyTempBuffer.getData(), compressedKeyLen);
- CompressionInputStream deflatFilter = codec.createInputStream(
- keyDecompressBuffer, keyDecompressor);
- DataInputStream compressedIn = new DataInputStream(deflatFilter);
- deflatFilter.resetState();
- keyDecompressedData.reset();
- keyDecompressedData.write(compressedIn, currentKeyLength);
- keyDataIn.reset(keyDecompressedData.getData(), currentKeyLength);
- currentKey.readFields(keyDataIn);
- } else {
- currentKey.readFields(in);
- }
-
- keyInit = true;
- currentValue.inited = false;
-
- readRowsIndexInBuffer = 0;
- recordsNumInValBuffer = currentKey.numberRows;
-
- for (int selIx = 0; selIx < selectedColumns.length; selIx++) {
- SelectedColumn col = selectedColumns[selIx];
- int colIx = col.colIndex;
- NonSyncDataOutputBuffer buf = currentKey.allCellValLenBuffer[colIx];
- colValLenBufferReadIn[selIx].reset(buf.getData(), buf.getLength());
- col.rowReadIndex = 0;
- col.runLength = 0;
- col.prvLength = -1;
- col.isNulled = colValLenBufferReadIn[selIx].getLength() == 0;
- }
-
- return currentKeyLength;
- }
-
- protected void currentValueBuffer() throws IOException {
- if (!keyInit) {
- nextKeyBuffer();
- }
- currentValue.keyBuffer = currentKey;
- currentValue.clearColumnBuffer();
- currentValue.readFields(in);
- currentValue.inited = true;
- }
-
- public boolean nextBlock() throws IOException {
- int keyLength = nextKeyBuffer();
- if(keyLength > 0) {
- currentValueBuffer();
- return true;
- }
- return false;
- }
-
- private boolean rowFetched = false;
-
- // use this buffer to hold column's cells value length for usages in
- // getColumn(), instead of using colValLenBufferReadIn directly.
- private final NonSyncDataInputBuffer fetchColumnTempBuf = new NonSyncDataInputBuffer();
-
- /**
- * Fetch all data in the buffer for a given column. This is useful for
- * columnar operators, which perform operations on an array data of one
- * column. It should be used together with {@link #nextColumnsBatch()}.
- * Calling getColumn() with not change the result of
- * {@link #next(LongWritable)} and
- * {@link #getCurrentRow(BytesRefArrayWritable)}.
- *
- * @param columnID the number of the column to get 0 to N-1
- * @throws IOException
- */
- public BytesRefArrayWritable getColumn(int columnID,
- BytesRefArrayWritable rest) throws IOException {
- int selColIdx = revPrjColIDs[columnID];
- if (selColIdx == -1) {
- return null;
- }
-
- if (rest == null) {
- rest = new BytesRefArrayWritable();
- }
-
- rest.resetValid(recordsNumInValBuffer);
-
- if (!currentValue.inited) {
- currentValueBuffer();
- }
-
- int columnNextRowStart = 0;
- fetchColumnTempBuf.reset(currentKey.allCellValLenBuffer[columnID]
- .getData(), currentKey.allCellValLenBuffer[columnID].getLength());
- SelectedColumn selCol = selectedColumns[selColIdx];
- byte[] uncompData = null;
- ValueBuffer.LazyDecompressionCallbackImpl decompCallBack = null;
- boolean decompressed = currentValue.decompressedFlag[selColIdx];
- if (decompressed) {
- uncompData =
- currentValue.loadedColumnsValueBuffer[selColIdx].getData();
- } else {
- decompCallBack = currentValue.lazyDecompressCallbackObjs[selColIdx];
- }
- for (int i = 0; i < recordsNumInValBuffer; i++) {
- colAdvanceRow(selColIdx, selCol);
- int length = selCol.prvLength;
-
- BytesRefWritable currentCell = rest.get(i);
-
- if (decompressed) {
- currentCell.set(uncompData, columnNextRowStart, length);
- } else {
- currentCell.set(decompCallBack, columnNextRowStart, length);
- }
- columnNextRowStart = columnNextRowStart + length;
- }
- return rest;
- }
-
- /**
- * Read in next key buffer and throw any data in current key buffer and
- * current value buffer. It will influence the result of
- * {@link #next(LongWritable)} and
- * {@link #getCurrentRow(BytesRefArrayWritable)}
- *
- * @return whether there still has records or not
- * @throws IOException
- */
- @SuppressWarnings("unused")
- @Deprecated
- public synchronized boolean nextColumnsBatch() throws IOException {
- passedRowsNum += (recordsNumInValBuffer - readRowsIndexInBuffer);
- return nextKeyBuffer() > 0;
- }
-
- /**
- * Returns how many rows we fetched with next(). It only means how many rows
- * are read by next(). The returned result may be smaller than actual number
- * of rows passed by, because {@link #seek(long)},
- * {@link #nextColumnsBatch()} can change the underlying key buffer and
- * value buffer.
- *
- * @return next row number
- * @throws IOException
- */
- public synchronized boolean next(LongWritable readRows) throws IOException {
- if (hasRecordsInBuffer()) {
- readRows.set(passedRowsNum);
- readRowsIndexInBuffer++;
- passedRowsNum++;
- rowFetched = false;
- return true;
- } else {
- keyInit = false;
- }
-
- int ret = -1;
- if (tolerateCorruptions) {
- ret = nextKeyValueTolerateCorruptions();
- } else {
- try {
- ret = nextKeyBuffer();
- } catch (EOFException eof) {
- eof.printStackTrace();
- }
- }
- return (ret > 0) && next(readRows);
- }
-
- private int nextKeyValueTolerateCorruptions() throws IOException {
- long currentOffset = in.getPos();
- int ret;
- try {
- ret = nextKeyBuffer();
- this.currentValueBuffer();
- } catch (IOException ioe) {
- // A BlockMissingException indicates a temporary error,
- // not a corruption. Re-throw this exception.
- String msg = ioe.getMessage();
- if (msg != null && msg.startsWith(BLOCK_MISSING_MESSAGE)) {
- LOG.warn("Re-throwing block-missing exception" + ioe);
- throw ioe;
- }
- // We have an IOException other than a BlockMissingException.
- LOG.warn("Ignoring IOException in file " + file +
- " after offset " + currentOffset, ioe);
- ret = -1;
- } catch (Throwable t) {
- // We got an exception that is not IOException
- // (typically OOM, IndexOutOfBounds, InternalError).
- // This is most likely a corruption.
- LOG.warn("Ignoring unknown error in " + file +
- " after offset " + currentOffset, t);
- ret = -1;
- }
- return ret;
- }
-
- public boolean hasRecordsInBuffer() {
- return readRowsIndexInBuffer < recordsNumInValBuffer;
- }
-
- /**
- * get the current row used,make sure called {@link #next(LongWritable)}
- * first.
- *
- * @throws IOException
- */
- public synchronized void getCurrentRow(BytesRefArrayWritable ret) throws IOException {
-
- if (!keyInit || rowFetched) {
- return;
- }
-
- if (tolerateCorruptions) {
- if (!currentValue.inited) {
- currentValueBuffer();
- }
- ret.resetValid(columnNumber);
- } else {
- if (!currentValue.inited) {
- currentValueBuffer();
- // do this only when not initialized, but we may need to find a way to
- // tell the caller how to initialize the valid size
- ret.resetValid(columnNumber);
- }
- }
-
- // we do not use BytesWritable here to avoid the byte-copy from
- // DataOutputStream to BytesWritable
- if (currentValue.numCompressed > 0) {
- for (int j = 0; j < selectedColumns.length; ++j) {
- SelectedColumn col = selectedColumns[j];
- int i = col.colIndex;
-
- if (col.isNulled) {
- ret.set(i, null);
- } else {
- BytesRefWritable ref = ret.unCheckedGet(i);
-
- colAdvanceRow(j, col);
-
- if (currentValue.decompressedFlag[j]) {
- ref.set(currentValue.loadedColumnsValueBuffer[j].getData(),
- col.rowReadIndex, col.prvLength);
- } else {
- ref.set(currentValue.lazyDecompressCallbackObjs[j],
- col.rowReadIndex, col.prvLength);
- }
- col.rowReadIndex += col.prvLength;
- }
- }
- } else {
- // This version of the loop eliminates a condition check and branch
- // and is measurably faster (20% or so)
- for (int j = 0; j < selectedColumns.length; ++j) {
- SelectedColumn col = selectedColumns[j];
- int i = col.colIndex;
-
- if (col.isNulled) {
- ret.set(i, null);
- } else {
- BytesRefWritable ref = ret.unCheckedGet(i);
-
- colAdvanceRow(j, col);
- ref.set(currentValue.loadedColumnsValueBuffer[j].getData(),
- col.rowReadIndex, col.prvLength);
- col.rowReadIndex += col.prvLength;
- }
- }
- }
- rowFetched = true;
- }
-
- /**
- * Advance column state to the next now: update offsets, run lengths etc
- * @param selCol - index among selectedColumns
- * @param col - column object to update the state of. prvLength will be
- * set to the new read position
- * @throws IOException
- */
- private void colAdvanceRow(int selCol, SelectedColumn col) throws IOException {
- if (col.runLength > 0) {
- --col.runLength;
- } else {
- int length = (int) WritableUtils.readVLong(colValLenBufferReadIn[selCol]);
- if (length < 0) {
- // we reach a runlength here, use the previous length and reset
- // runlength
- col.runLength = (~length) - 1;
- } else {
- col.prvLength = length;
- col.runLength = 0;
- }
- }
- }
-
- /** Returns true iff the previous call to next passed a sync mark. */
- @SuppressWarnings("unused")
- public boolean syncSeen() {
- return syncSeen;
- }
-
- /** Returns the last seen sync position. */
- public long lastSeenSyncPos() {
- return lastSeenSyncPos;
- }
-
- /** Returns the name of the file. */
- @Override
- public String toString() {
- return file.toString();
- }
-
- @SuppressWarnings("unused")
- public boolean isCompressedRCFile() {
- return this.decompress;
- }
-
- /** Close the reader. */
- public void close() {
- IOUtils.closeStream(in);
- currentValue.close();
- if (decompress) {
- IOUtils.closeStream(keyDecompressedData);
- if (keyDecompressor != null) {
- // Make sure we only return keyDecompressor once.
- CodecPool.returnDecompressor(keyDecompressor);
- keyDecompressor = null;
- }
- }
- }
-
- /**
- * return the KeyBuffer object used in the reader. Internally in each
- * reader, there is only one KeyBuffer object, which gets reused for every
- * block.
- */
- public KeyBuffer getCurrentKeyBufferObj() {
- return this.currentKey;
- }
-
- /**
- * return the ValueBuffer object used in the reader. Internally in each
- * reader, there is only one ValueBuffer object, which gets reused for every
- * block.
- */
- public ValueBuffer getCurrentValueBufferObj() {
- return this.currentValue;
- }
-
- //return the current block's length
- public int getCurrentBlockLength() {
- return this.currentRecordLength;
- }
-
- //return the current block's key length
- public int getCurrentKeyLength() {
- return this.currentKeyLength;
- }
-
- //return the current block's compressed key length
- public int getCurrentCompressedKeyLen() {
- return this.compressedKeyLen;
- }
-
- //return the CompressionCodec used for this file
- public CompressionCodec getCompressionCodec() {
- return this.codec;
- }
-
- }
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFileInputFormat.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFileInputFormat.java
deleted file mode 100644
index 19a8f8a..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFileInputFormat.java
+++ /dev/null
@@ -1,83 +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.
- */
-
-package org.apache.asterix.hivecompat.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * RCFileInputFormat.
- *
- * @param <K>
- * @param <V>
- */
-public class RCFileInputFormat<K extends LongWritable, V extends BytesRefArrayWritable>
- extends FileInputFormat<K, V> implements InputFormatChecker {
-
- public RCFileInputFormat() {
- setMinSplitSize(SequenceFile.SYNC_INTERVAL);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public RecordReader<K, V> getRecordReader(InputSplit split, JobConf job,
- Reporter reporter) throws IOException {
-
- reporter.setStatus(split.toString());
-
- return new RCFileRecordReader(job, (FileSplit) split);
- }
-
- @Override
- public boolean validateInput(FileSystem fs, HiveConf conf,
- ArrayList<FileStatus> files) throws IOException {
- if (files.size() <= 0) {
- return false;
- }
- for (int fileId = 0; fileId < files.size(); fileId++) {
- RCFile.Reader reader = null;
- try {
- reader = new RCFile.Reader(fs, files.get(fileId)
- .getPath(), conf);
- reader.close();
- reader = null;
- } catch (IOException e) {
- return false;
- } finally {
- if (null != reader) {
- reader.close();
- }
- }
- }
- return true;
- }
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFileRecordReader.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFileRecordReader.java
deleted file mode 100644
index 27ce58c..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/RCFileRecordReader.java
+++ /dev/null
@@ -1,213 +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.
- */
-
-package org.apache.asterix.hivecompat.io;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.WeakHashMap;
-
-import org.apache.asterix.hivecompat.io.RCFile.KeyBuffer;
-import org.apache.asterix.hivecompat.io.RCFile.Reader;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * RCFileRecordReader.
- *
- * @param <K>
- * @param <V>
- */
-public class RCFileRecordReader<K extends LongWritable, V extends BytesRefArrayWritable>
- implements RecordReader<LongWritable, BytesRefArrayWritable> {
-
- private final Reader in;
- private final long start;
- private final long end;
- private boolean more = true;
- protected Configuration conf;
- private final FileSplit split;
- private final boolean useCache;
-
- private static RCFileSyncCache syncCache = new RCFileSyncCache();
-
- private static final class RCFileSyncEntry {
- long end;
- long endSync;
- }
-
- private static final class RCFileSyncCache {
-
- private final Map<String, RCFileSyncEntry> cache;
-
- public RCFileSyncCache() {
- cache = Collections.synchronizedMap(new WeakHashMap<String, RCFileSyncEntry>());
- }
-
- public void put(FileSplit split, long endSync) {
- Path path = split.getPath();
- long end = split.getStart() + split.getLength();
- String key = path.toString()+"+"+String.format("%d",end);
-
- RCFileSyncEntry entry = new RCFileSyncEntry();
- entry.end = end;
- entry.endSync = endSync;
- if(entry.endSync >= entry.end) {
- cache.put(key, entry);
- }
- }
-
- public long get(FileSplit split) {
- Path path = split.getPath();
- long start = split.getStart();
- String key = path.toString()+"+"+String.format("%d",start);
- RCFileSyncEntry entry = cache.get(key);
- if(entry != null) {
- return entry.endSync;
- }
- return -1;
- }
- }
-
- public RCFileRecordReader(Configuration conf, FileSplit split)
- throws IOException {
-
- Path path = split.getPath();
- FileSystem fs = path.getFileSystem(conf);
- this.in = new RCFile.Reader(fs, path, conf);
- this.end = split.getStart() + split.getLength();
- this.conf = conf;
- this.split = split;
-
- useCache = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVEUSERCFILESYNCCACHE);
-
- if (split.getStart() > in.getPosition()) {
- long oldSync = useCache ? syncCache.get(split) : -1;
- if(oldSync == -1) {
- in.sync(split.getStart()); // sync to start
- } else {
- in.seek(oldSync);
- }
- }
-
- this.start = in.getPosition();
-
- more = start < end;
- }
-
- public Class<?> getKeyClass() {
- return LongWritable.class;
- }
-
- public Class<?> getValueClass() {
- return BytesRefArrayWritable.class;
- }
-
- public LongWritable createKey() {
- return (LongWritable) ReflectionUtils.newInstance(getKeyClass(), conf);
- }
-
- public BytesRefArrayWritable createValue() {
- return (BytesRefArrayWritable) ReflectionUtils.newInstance(getValueClass(),
- conf);
- }
-
- public boolean nextBlock() throws IOException {
- return in.nextBlock();
- }
-
- @Override
- public boolean next(LongWritable key, BytesRefArrayWritable value)
- throws IOException {
-
- more = next(key);
-
- if (more) {
- in.getCurrentRow(value);
- }
- return more;
- }
-
- protected boolean next(LongWritable key) throws IOException {
- if (!more) {
- return false;
- }
-
- more = in.next(key);
-
- long lastSeenSyncPos = in.lastSeenSyncPos();
-
- if (lastSeenSyncPos >= end) {
- if(useCache) {
- syncCache.put(split, lastSeenSyncPos);
- }
- more = false;
- return more;
- }
- return more;
- }
-
- /**
- * Return the progress within the input split.
- *
- * @return 0.0 to 1.0 of the input byte range
- */
- public float getProgress() throws IOException {
- if (end == start) {
- return 0.0f;
- } else {
- return Math.min(1.0f, (in.getPosition() - start) / (float) (end - start));
- }
- }
-
- public long getPos() throws IOException {
- return in.getPosition();
- }
-
- public KeyBuffer getKeyBuffer() {
- return in.getCurrentKeyBufferObj();
- }
-
- protected void seek(long pos) throws IOException {
- in.seek(pos);
- }
-
- public void sync(long pos) throws IOException {
- in.sync(pos);
- }
-
- public void resetBuffer() {
- in.resetBuffer();
- }
-
- public long getStart() {
- return start;
- }
-
- public void close() throws IOException {
- in.close();
- }
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/SchemaAwareCompressionInputStream.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/SchemaAwareCompressionInputStream.java
deleted file mode 100644
index fdaa581..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/SchemaAwareCompressionInputStream.java
+++ /dev/null
@@ -1,43 +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.
- */
-
-package org.apache.asterix.hivecompat.io;
-
-import java.io.InputStream;
-
-import org.apache.hadoop.io.compress.CompressionInputStream;
-
-/**
- *
- * SchemaAwareCompressionInputStream adds the ability to inform the compression
- * stream what column is being read.
- *
- */
-public abstract class SchemaAwareCompressionInputStream extends CompressionInputStream {
-
- protected SchemaAwareCompressionInputStream(InputStream in) throws java.io.IOException {
- super(in);
- }
-
- /**
- * The column being read
- *
- * @param columnIndex the index of the column. Use -1 for non-column data
- */
- public abstract void setColumnIndex(int columnIndex);
-}
diff --git a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/SchemaAwareCompressionOutputStream.java b/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/SchemaAwareCompressionOutputStream.java
deleted file mode 100644
index 9bda8df..0000000
--- a/asterixdb/asterix-hivecompat/src/main/java/org/apache/asterix/hivecompat/io/SchemaAwareCompressionOutputStream.java
+++ /dev/null
@@ -1,44 +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.
- */
-
-package org.apache.asterix.hivecompat.io;
-
-import java.io.OutputStream;
-
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-
-/**
- *
- * SchemaAwareCompressionOutputStream adds the ability to inform the comression stream
- * the current column being compressed.
- *
- */
-public abstract class SchemaAwareCompressionOutputStream extends CompressionOutputStream {
-
- protected SchemaAwareCompressionOutputStream(OutputStream out) {
- super(out);
- }
-
- /**
- *
- * The column being output
- *
- * @param columnIndex the index of the column. Use -1 for non-column data
- */
- public abstract void setColumnIndex(int columnIndex);
-}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
index 744d3cc..612f79f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/AbstractExpression.java
@@ -58,4 +58,15 @@
public List<IExpressionAnnotation> getHints() {
return hints;
}
+
+ public <T extends IExpressionAnnotation> T findHint(Class<T> hintClass) {
+ if (hints != null) {
+ for (IExpressionAnnotation hint : hints) {
+ if (hint.getClass().equals(hintClass)) {
+ return hintClass.cast(hint);
+ }
+ }
+ }
+ return null;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
index 3df490c..1645d76 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
@@ -23,7 +23,9 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.Warning;
@@ -38,6 +40,8 @@
FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames, boolean isStored)
throws CompilationException;
+ ViewDecl parseViewBody(DatasetFullyQualifiedName viewName) throws CompilationException;
+
/**
* Gets the warnings generated during parsing
*/
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
index f28de55..6099cb6 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
@@ -25,6 +25,9 @@
import org.apache.asterix.lang.common.expression.AbstractCallExpression;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
+import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.VarIdentifier;
public interface IQueryRewriter {
@@ -37,13 +40,12 @@
* rewriting context
* @param allowNonStoredUdfCalls
* whether calls to non-stored user-defined functions should be resolved
- * @param inlineUdfs
+ * @param inlineUdfsAndViews
* whether user defined functions should be inlines
* @param externalVars
- * statement parameters (external variables)
*/
void rewrite(LangRewritingContext context, IReturningStatement topExpr, boolean allowNonStoredUdfCalls,
- boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException;
+ boolean inlineUdfsAndViews, Collection<VarIdentifier> externalVars) throws CompilationException;
/**
* Find the function calls used by a given expression
@@ -55,4 +57,12 @@
* Find all external variables (positional and named variables) in given expression
*/
Set<VariableExpr> getExternalVariables(Expression expr) throws CompilationException;
+
+ VarIdentifier toExternalVariableName(String statementParameterName);
+
+ String toFunctionParameterName(VarIdentifier paramVar);
+
+ Query createFunctionAccessorQuery(FunctionDecl functionDecl);
+
+ Query createViewAccessorQuery(ViewDecl viewDecl);
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
index 9d983b0..f50d040 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
@@ -19,7 +19,6 @@
package org.apache.asterix.lang.common.base;
import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.metadata.declared.MetadataProvider;
public interface IStatementRewriter {
@@ -36,8 +35,4 @@
* a metadata provider
*/
void rewrite(Statement statement, MetadataProvider metadataProvider) throws CompilationException;
-
- String toExternalVariableName(String statementParameterName);
-
- String toFunctionParameterName(VarIdentifier paramVar);
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 182fc08..2594cdd 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -46,6 +46,21 @@
private Category() {
}
+
+ public static String toString(byte category) {
+ switch (category) {
+ case QUERY:
+ return "query";
+ case UPDATE:
+ return "update";
+ case DDL:
+ return "ddl";
+ case PROCEDURE:
+ return "procedure";
+ default:
+ throw new IllegalArgumentException(String.valueOf(category));
+ }
+ }
}
enum Kind {
@@ -58,6 +73,7 @@
UPSERT,
UPDATE,
FUNCTION_DECL,
+ VIEW_DECL,
LOAD,
NODEGROUP_DECL,
NODEGROUP_DROP,
@@ -68,6 +84,7 @@
WRITE,
CREATE_INDEX,
CREATE_DATAVERSE,
+ CREATE_VIEW,
CREATE_FULL_TEXT_FILTER,
CREATE_FULL_TEXT_CONFIG,
INDEX_DROP,
@@ -89,6 +106,7 @@
LIBRARY_DROP,
CREATE_SYNONYM,
SYNONYM_DROP,
+ VIEW_DROP,
COMPACT,
EXTERNAL_DATASET_REFRESH,
SUBSCRIBE_FEED,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
index af8c725..0793cae 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
@@ -30,6 +30,7 @@
public class OrderbyClause extends AbstractClause {
private List<Expression> orderbyList;
private List<OrderModifier> modifierList;
+ private List<NullOrderModifier> nullModifierList;
private RangeMap rangeMap; // can be null
private int numFrames = -1;
private int numTuples = -1;
@@ -38,9 +39,11 @@
// Default constructor.
}
- public OrderbyClause(List<Expression> orderbyList, List<OrderModifier> modifierList) {
+ public OrderbyClause(List<Expression> orderbyList, List<OrderModifier> modifierList,
+ List<NullOrderModifier> nullModifierList) {
this.orderbyList = orderbyList;
this.modifierList = modifierList;
+ this.nullModifierList = nullModifierList;
}
public List<Expression> getOrderbyList() {
@@ -59,6 +62,14 @@
this.modifierList = modifierList;
}
+ public List<NullOrderModifier> getNullModifierList() {
+ return nullModifierList;
+ }
+
+ public void setNullModifierList(List<NullOrderModifier> nullModifierList) {
+ this.nullModifierList = nullModifierList;
+ }
+
@Override
public ClauseType getClauseType() {
return ClauseType.ORDER_BY_CLAUSE;
@@ -69,6 +80,11 @@
DESC
}
+ public enum NullOrderModifier {
+ FIRST,
+ LAST
+ }
+
@Override
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
return visitor.visit(this, arg);
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
index a37a366..3deedfd 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
@@ -77,6 +77,8 @@
case GE:
case LT:
case LE:
+ case DISTINCT:
+ case NOT_DISTINCT:
return true;
default:
return false;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java
index a35d975..8715774 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java
@@ -78,6 +78,7 @@
}
public enum Quantifier {
+ SOME_AND_EVERY,
EVERY,
SOME
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 675d0d3..95cccb0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -18,14 +18,16 @@
*/
package org.apache.asterix.lang.common.rewrites;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.common.util.FunctionUtil;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.hyracks.algebricks.core.algebra.base.Counter;
import org.apache.hyracks.api.exceptions.IWarningCollector;
@@ -34,15 +36,17 @@
private final MetadataProvider metadataProvider;
private final IWarningCollector warningCollector;
private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
+ private final Map<DatasetFullyQualifiedName, ViewDecl> declaredViews;
private final Counter varCounter;
private int systemVarCounter = 1;
private final Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
public LangRewritingContext(MetadataProvider metadataProvider, List<FunctionDecl> declaredFunctions,
- IWarningCollector warningCollector, int varCounter) {
+ List<ViewDecl> declaredViews, IWarningCollector warningCollector, int varCounter) {
this.metadataProvider = metadataProvider;
this.warningCollector = warningCollector;
- this.declaredFunctions = FunctionUtil.getFunctionMap(declaredFunctions);
+ this.declaredFunctions = createMap(declaredFunctions, FunctionDecl::getSignature);
+ this.declaredViews = createMap(declaredViews, ViewDecl::getViewName);
this.varCounter = new Counter(varCounter);
}
@@ -93,4 +97,19 @@
public Map<FunctionSignature, FunctionDecl> getDeclaredFunctions() {
return declaredFunctions;
}
+
+ public Map<DatasetFullyQualifiedName, ViewDecl> getDeclaredViews() {
+ return declaredViews;
+ }
+
+ private static <K, V> Map<K, V> createMap(List<V> values, java.util.function.Function<V, K> keyMapper) {
+ if (values == null || values.isEmpty()) {
+ return Collections.emptyMap();
+ }
+ Map<K, V> result = new HashMap<>();
+ for (V v : values) {
+ result.put(keyMapper.apply(v), v);
+ }
+ return Collections.unmodifiableMap(result);
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 468006c..aabd76d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -20,6 +20,7 @@
import java.util.Collections;
import java.util.List;
+import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
@@ -34,6 +35,7 @@
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.util.OptionalBoolean;
public class CreateIndexStatement extends AbstractStatement {
@@ -48,10 +50,14 @@
private final int gramLength;
// Specific to FullText indexes.
private final String fullTextConfigName;
+ private final OptionalBoolean excludeUnknownKey;
+ private final OptionalBoolean castDefaultNull;
+ private final Map<String, String> castConfig;
public CreateIndexStatement(DataverseName dataverseName, Identifier datasetName, Identifier indexName,
IndexType indexType, List<IndexedElement> indexedElements, boolean enforced, int gramLength,
- String fullTextConfigName, boolean ifNotExists) {
+ String fullTextConfigName, boolean ifNotExists, Boolean excludeUnknownKey, Boolean castDefaultNull,
+ Map<String, String> castConfig) {
this.dataverseName = dataverseName;
this.datasetName = Objects.requireNonNull(datasetName);
this.indexName = Objects.requireNonNull(indexName);
@@ -61,6 +67,9 @@
this.gramLength = gramLength;
this.ifNotExists = ifNotExists;
this.fullTextConfigName = fullTextConfigName;
+ this.excludeUnknownKey = OptionalBoolean.ofNullable(excludeUnknownKey);
+ this.castDefaultNull = OptionalBoolean.ofNullable(castDefaultNull);
+ this.castConfig = castConfig == null ? Collections.emptyMap() : castConfig;
}
public String getFullTextConfigName() {
@@ -91,6 +100,22 @@
return enforced;
}
+ public boolean hasExcludeUnknownKey() {
+ return excludeUnknownKey.isPresent();
+ }
+
+ public OptionalBoolean getExcludeUnknownKey() {
+ return excludeUnknownKey;
+ }
+
+ public boolean hasCastDefaultNull() {
+ return castDefaultNull.isPresent();
+ }
+
+ public OptionalBoolean getCastDefaultNull() {
+ return castDefaultNull;
+ }
+
public int getGramLength() {
return gramLength;
}
@@ -99,6 +124,10 @@
return this.ifNotExists;
}
+ public Map<String, String> getCastConfig() {
+ return castConfig;
+ }
+
@Override
public Kind getKind() {
return Statement.Kind.CREATE_INDEX;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateViewStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateViewStatement.java
new file mode 100644
index 0000000..7e4a0ef
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateViewStatement.java
@@ -0,0 +1,181 @@
+/*
+ * 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.lang.common.statement;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.expression.TypeExpression;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public final class CreateViewStatement extends AbstractStatement {
+
+ private final DataverseName dataverseName;
+
+ private final String viewName;
+
+ private final TypeExpression itemType;
+
+ private final String viewBody;
+
+ private final Expression viewBodyExpression;
+
+ private final Map<String, String> viewConfig;
+
+ private final KeyDecl primaryKeyDecl;
+
+ private final List<ForeignKeyDecl> foreignKeyDecls;
+
+ private final Boolean defaultNull;
+
+ private final boolean replaceIfExists;
+
+ private final boolean ifNotExists;
+
+ public CreateViewStatement(DataverseName dataverseName, String viewName, TypeExpression itemType, String viewBody,
+ Expression viewBodyExpression, Boolean defaultNull, Map<String, String> viewConfig, KeyDecl primaryKeyDecl,
+ List<ForeignKeyDecl> foreignKeyDecls, boolean replaceIfExists, boolean ifNotExists) {
+ this.dataverseName = dataverseName;
+ this.viewName = Objects.requireNonNull(viewName);
+ this.itemType = itemType;
+ this.viewBody = Objects.requireNonNull(viewBody);
+ this.viewBodyExpression = Objects.requireNonNull(viewBodyExpression);
+ this.defaultNull = defaultNull;
+ this.viewConfig = viewConfig;
+ this.primaryKeyDecl = primaryKeyDecl;
+ this.foreignKeyDecls = foreignKeyDecls;
+ this.replaceIfExists = replaceIfExists;
+ this.ifNotExists = ifNotExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Statement.Kind.CREATE_VIEW;
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getViewName() {
+ return viewName;
+ }
+
+ public boolean hasItemType() {
+ return itemType != null;
+ }
+
+ public TypeExpression getItemType() {
+ return itemType;
+ }
+
+ public String getViewBody() {
+ return viewBody;
+ }
+
+ public Expression getViewBodyExpression() {
+ return viewBodyExpression;
+ }
+
+ public boolean getReplaceIfExists() {
+ return replaceIfExists;
+ }
+
+ public boolean getIfNotExists() {
+ return ifNotExists;
+ }
+
+ // Typed view parameters
+
+ public Boolean getDefaultNull() {
+ return defaultNull;
+ }
+
+ public KeyDecl getPrimaryKeyDecl() {
+ return primaryKeyDecl;
+ }
+
+ public List<ForeignKeyDecl> getForeignKeyDecls() {
+ return foreignKeyDecls;
+ }
+
+ public Map<String, String> getViewConfiguration() {
+ return viewConfig;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ public static class KeyDecl {
+
+ protected final List<List<String>> fields;
+
+ protected final List<Integer> sourceIndicators;
+
+ public KeyDecl(List<List<String>> fields, List<Integer> sourceIndicators) {
+ this.fields = fields;
+ this.sourceIndicators = sourceIndicators;
+ }
+
+ public List<List<String>> getFields() {
+ return fields;
+ }
+
+ public List<Integer> getSourceIndicators() {
+ return sourceIndicators;
+ }
+ }
+
+ public static class ForeignKeyDecl extends KeyDecl {
+
+ private final DataverseName referencedDataverseName;
+
+ private final Identifier referencedDatasetName;
+
+ public ForeignKeyDecl(List<List<String>> fields, List<Integer> sourceIndicators,
+ DataverseName referencedDataverseName, Identifier referencedDatasetName) {
+ super(fields, sourceIndicators);
+ this.referencedDataverseName = referencedDataverseName;
+ this.referencedDatasetName = referencedDatasetName;
+ }
+
+ public DataverseName getReferencedDataverseName() {
+ return referencedDataverseName;
+ }
+
+ public Identifier getReferencedDatasetName() {
+ return referencedDatasetName;
+ }
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
index 829c407..b901819 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
@@ -28,14 +28,25 @@
private DataverseName dataverseName;
+ private boolean ifExists;
+
public DataverseDecl(DataverseName dataverseName) {
+ this(dataverseName, false);
+ }
+
+ public DataverseDecl(DataverseName dataverseName, boolean ifExists) {
this.dataverseName = dataverseName;
+ this.ifExists = ifExists;
}
public DataverseName getDataverseName() {
return dataverseName;
}
+ public boolean getIfExists() {
+ return ifExists;
+ }
+
@Override
public Kind getKind() {
return Statement.Kind.DATAVERSE_DECL;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ViewDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ViewDecl.java
new file mode 100644
index 0000000..1e59f8d
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ViewDecl.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Objects;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public final class ViewDecl extends AbstractStatement {
+
+ private final DatasetFullyQualifiedName viewName;
+
+ private final Expression viewBody;
+
+ private Expression viewBodyNormalized;
+
+ public ViewDecl(DatasetFullyQualifiedName viewName, Expression viewBody) {
+ this.viewName = Objects.requireNonNull(viewName);
+ this.viewBody = Objects.requireNonNull(viewBody);
+ }
+
+ public DatasetFullyQualifiedName getViewName() {
+ return viewName;
+ }
+
+ public Expression getViewBody() {
+ return viewBody;
+ }
+
+ public Expression getNormalizedViewBody() {
+ return viewBodyNormalized;
+ }
+
+ public void setNormalizedViewBody(Expression expr) {
+ viewBodyNormalized = expr;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.VIEW_DECL;
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.QUERY;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ViewDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ViewDropStatement.java
new file mode 100644
index 0000000..9f8577c
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ViewDropStatement.java
@@ -0,0 +1,69 @@
+/*
+ * 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.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class ViewDropStatement extends AbstractStatement {
+
+ private final DataverseName dataverseName;
+
+ private final Identifier viewName;
+
+ private final boolean ifExists;
+
+ public ViewDropStatement(DataverseName dataverseName, Identifier viewName, boolean ifExists) {
+ this.dataverseName = dataverseName;
+ this.viewName = viewName;
+ this.ifExists = ifExists;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public Identifier getViewName() {
+ return viewName;
+ }
+
+ public boolean getIfExists() {
+ return ifExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Statement.Kind.VIEW_DROP;
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
index f4f2ae1..e207e75 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
@@ -41,7 +41,9 @@
IN("in"),
NOT_IN("not_in"),
BETWEEN("between"),
- NOT_BETWEEN("not_between");
+ NOT_BETWEEN("not_between"),
+ DISTINCT("distinct"),
+ NOT_DISTINCT("not_distinct");
private static final OperatorType[] VALUES = values();
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
index 552ca39..0aef69f 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
@@ -18,9 +18,14 @@
*/
package org.apache.asterix.lang.common.util;
+import java.util.List;
+
import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.expression.RecordTypeDefinition;
+import org.apache.asterix.lang.common.expression.TypeExpression;
+import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.object.base.AdmObjectNode;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.AUnionType;
@@ -62,7 +67,7 @@
* ***********************************************
*/
private static final ARecordType WITH_OBJECT_TYPE = getWithObjectType();
- private static final AdmObjectNode EMPTY_WITH_OBJECT = new AdmObjectNode();
+ static final AdmObjectNode EMPTY_WITH_OBJECT = new AdmObjectNode();
private DatasetDeclParametersUtil() {
}
@@ -127,4 +132,38 @@
final IAType[] schemeType = { BuiltinType.ASTRING };
return new ARecordType(NODE_GROUP_NAME, schemeName, schemeType, false);
}
+
+ /**
+ * Adjusts dataset inline type definition if it has primary key specification:
+ * forces NOT UNKNOWN on fields that are part of primary key.
+ */
+ public static void adjustInlineTypeDecl(TypeExpression typeDecl, List<List<String>> primaryKeyFields,
+ List<Integer> primaryKeySources, boolean isMeta) {
+ switch (typeDecl.getTypeKind()) {
+ case RECORD:
+ RecordTypeDefinition recordTypeDef = (RecordTypeDefinition) typeDecl;
+ for (int i = 0, n = primaryKeyFields.size(); i < n; i++) {
+ List<String> primaryKeyPath = primaryKeyFields.get(i);
+ if (primaryKeyPath.size() == 1) {
+ String primaryKeyFieldName = primaryKeyPath.get(0);
+ boolean isMetaSource =
+ primaryKeySources != null && primaryKeySources.get(i) == Index.META_RECORD_INDICATOR;
+ boolean isSameSource = isMetaSource == isMeta;
+ if (isSameSource) {
+ int fieldIdx = recordTypeDef.getFieldNames().indexOf(primaryKeyFieldName);
+ if (fieldIdx >= 0) {
+ recordTypeDef.getMissableFields().set(fieldIdx, false);
+ recordTypeDef.getNullableFields().set(fieldIdx, false);
+ }
+ }
+ }
+ }
+ break;
+ case TYPEREFERENCE:
+ // this is not an inline type decl
+ break;
+ default:
+ throw new IllegalStateException(typeDecl.getTypeKind().toString());
+ }
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
index dd21152..cab1aca 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
@@ -18,21 +18,39 @@
*/
package org.apache.asterix.lang.common.util;
+import static org.apache.asterix.lang.common.base.Literal.Type.DOUBLE;
+
+import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
+import java.util.HashSet;
import java.util.List;
+import java.util.Map;
+import java.util.Set;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IQueryRewriter;
import org.apache.asterix.lang.common.base.Literal;
+import org.apache.asterix.lang.common.expression.AbstractCallExpression;
import org.apache.asterix.lang.common.expression.FieldBinding;
import org.apache.asterix.lang.common.expression.ListConstructor;
import org.apache.asterix.lang.common.expression.LiteralExpr;
import org.apache.asterix.lang.common.expression.RecordConstructor;
import org.apache.asterix.lang.common.literal.DoubleLiteral;
+import org.apache.asterix.lang.common.literal.FloatLiteral;
+import org.apache.asterix.lang.common.literal.IntegerLiteral;
import org.apache.asterix.lang.common.literal.LongIntegerLiteral;
import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
import org.apache.asterix.object.base.AdmArrayNode;
import org.apache.asterix.object.base.AdmBigIntNode;
import org.apache.asterix.object.base.AdmBooleanNode;
@@ -41,6 +59,14 @@
import org.apache.asterix.object.base.AdmObjectNode;
import org.apache.asterix.object.base.AdmStringNode;
import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.exceptions.TypeMismatchException;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+import com.google.common.graph.GraphBuilder;
+import com.google.common.graph.Graphs;
+import com.google.common.graph.MutableGraph;
public class ExpressionUtils {
private ExpressionUtils() {
@@ -113,4 +139,195 @@
}
return null;
}
+
+ public static Boolean getBooleanLiteral(Expression arg) {
+ if (arg.getKind() == Expression.Kind.LITERAL_EXPRESSION) {
+ Literal item = ((LiteralExpr) arg).getValue();
+ switch (item.getLiteralType()) {
+ case TRUE:
+ return true;
+ case FALSE:
+ return false;
+ }
+ }
+ return null;
+ }
+
+ public static Literal reverseSign(Literal lit) throws TypeMismatchException {
+ switch (lit.getLiteralType()) {
+ case DOUBLE:
+ DoubleLiteral dLit = (DoubleLiteral) lit;
+ DoubleLiteral reversedDLit = new DoubleLiteral(-dLit.getValue());
+ return reversedDLit;
+ case FLOAT:
+ FloatLiteral fLit = (FloatLiteral) lit;
+ FloatLiteral reversedFLit = new FloatLiteral(-fLit.getValue());
+ return reversedFLit;
+ case LONG:
+ LongIntegerLiteral lLit = (LongIntegerLiteral) lit;
+ LongIntegerLiteral reversedLLit = new LongIntegerLiteral(-lLit.getValue());
+ return reversedLLit;
+ case INTEGER:
+ IntegerLiteral iLit = (IntegerLiteral) lit;
+ IntegerLiteral reversedILit = new IntegerLiteral(-iLit.getValue());
+ return reversedILit;
+ case NULL:
+ case MISSING:
+ return lit;
+ default:
+ throw new TypeMismatchException(null, convertLiteralTypeTagToATypeTag(lit), ATypeTag.DOUBLE,
+ ATypeTag.FLOAT, ATypeTag.BIGINT, ATypeTag.INTEGER);
+ }
+ }
+
+ public static double getDoubleValue(Literal item) throws TypeMismatchException {
+ if ((item.getLiteralType() == Literal.Type.DOUBLE) || (item.getLiteralType() == Literal.Type.FLOAT)
+ || (item.getLiteralType() == Literal.Type.LONG) || (item.getLiteralType() == Literal.Type.INTEGER)) {
+ return ((Number) item.getValue()).doubleValue();
+ } else {
+ throw new TypeMismatchException(null, convertLiteralTypeTagToATypeTag(item), ATypeTag.DOUBLE,
+ ATypeTag.FLOAT, ATypeTag.BIGINT, ATypeTag.INTEGER);
+ }
+ }
+
+ public static long getLongValue(Literal item) throws TypeMismatchException {
+ if ((item.getLiteralType() == Literal.Type.LONG) || (item.getLiteralType() == Literal.Type.INTEGER)) {
+ return ((Number) item.getValue()).longValue();
+ } else {
+ throw new TypeMismatchException(null, convertLiteralTypeTagToATypeTag(item), ATypeTag.BIGINT,
+ ATypeTag.INTEGER);
+ }
+ }
+
+ private static ATypeTag convertLiteralTypeTagToATypeTag(Literal lit) {
+ switch (lit.getLiteralType()) {
+ case DOUBLE:
+ return ATypeTag.DOUBLE;
+ case FLOAT:
+ return ATypeTag.FLOAT;
+ case LONG:
+ return ATypeTag.BIGINT;
+ case INTEGER:
+ return ATypeTag.INTEGER;
+ case TRUE:
+ case FALSE:
+ return ATypeTag.BOOLEAN;
+ case NULL:
+ return ATypeTag.NULL;
+ case MISSING:
+ return ATypeTag.MISSING;
+ default:
+ return ATypeTag.STRING;
+ }
+ }
+
+ public static void collectDependencies(Expression expression, IQueryRewriter rewriter,
+ List<Triple<DataverseName, String, String>> outDatasetDependencies,
+ List<Triple<DataverseName, String, String>> outSynonymDependencies,
+ List<Triple<DataverseName, String, String>> outFunctionDependencies) throws CompilationException {
+ // Duplicate elimination
+ Set<DatasetFullyQualifiedName> seenDatasets = new HashSet<>();
+ Set<DatasetFullyQualifiedName> seenSynonyms = new HashSet<>();
+ Set<FunctionSignature> seenFunctions = new HashSet<>();
+ List<AbstractCallExpression> functionCalls = new ArrayList<>();
+ rewriter.getFunctionCalls(expression, functionCalls);
+
+ for (AbstractCallExpression functionCall : functionCalls) {
+ switch (functionCall.getKind()) {
+ case CALL_EXPRESSION:
+ FunctionSignature signature = functionCall.getFunctionSignature();
+ if (FunctionUtil.isBuiltinFunctionSignature(signature)) {
+ if (FunctionUtil.isBuiltinDatasetFunction(signature)) {
+ Triple<DatasetFullyQualifiedName, Boolean, DatasetFullyQualifiedName> dsArgs =
+ FunctionUtil.parseDatasetFunctionArguments(functionCall);
+ DatasetFullyQualifiedName synonymReference = dsArgs.third;
+ if (synonymReference != null) {
+ // resolved via synonym -> store synonym name as a dependency
+ if (seenSynonyms.add(synonymReference)) {
+ outSynonymDependencies.add(new Triple<>(synonymReference.getDataverseName(),
+ synonymReference.getDatasetName(), null));
+ }
+ } else {
+ // resolved directly -> store dataset (or view) name as a dependency
+ DatasetFullyQualifiedName datasetReference = dsArgs.first;
+ if (seenDatasets.add(datasetReference)) {
+ outDatasetDependencies.add(new Triple<>(datasetReference.getDataverseName(),
+ datasetReference.getDatasetName(), null));
+ }
+ }
+ }
+ } else {
+ if (seenFunctions.add(signature)) {
+ outFunctionDependencies.add(new Triple<>(signature.getDataverseName(), signature.getName(),
+ Integer.toString(signature.getArity())));
+ }
+ }
+ break;
+ case WINDOW_EXPRESSION:
+ // there cannot be used-defined window functions
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ functionCall.getSourceLocation(), functionCall.getFunctionSignature().toString(false));
+ }
+ }
+ }
+
+ public static boolean hasFunctionOrViewRecursion(Map<FunctionSignature, FunctionDecl> functionDeclMap,
+ Map<DatasetFullyQualifiedName, ViewDecl> viewDeclMap,
+ java.util.function.Function<Collection<AbstractCallExpression>, GatherFunctionCallsVisitor> callVisitorFactory)
+ throws CompilationException {
+ List<AbstractCallExpression> callList = new ArrayList<>();
+ GatherFunctionCallsVisitor callVisitor = callVisitorFactory.apply(callList);
+ MutableGraph<AbstractStatement> callGraph = GraphBuilder.directed().allowsSelfLoops(true).build();
+ for (FunctionDecl from : functionDeclMap.values()) {
+ callList.clear();
+ from.getNormalizedFuncBody().accept(callVisitor, null);
+ for (AbstractCallExpression callExpr : callList) {
+ addToCallGraph(callGraph, from, callExpr, functionDeclMap, viewDeclMap);
+ }
+ }
+ for (ViewDecl from : viewDeclMap.values()) {
+ callList.clear();
+ from.getNormalizedViewBody().accept(callVisitor, null);
+ for (AbstractCallExpression callExpr : callList) {
+ addToCallGraph(callGraph, from, callExpr, functionDeclMap, viewDeclMap);
+ }
+ }
+ return Graphs.hasCycle(callGraph);
+ }
+
+ private static void addToCallGraph(MutableGraph<AbstractStatement> callGraph, AbstractStatement from,
+ AbstractCallExpression callExpr, Map<FunctionSignature, FunctionDecl> functionDeclMap,
+ Map<DatasetFullyQualifiedName, ViewDecl> viewDeclMap) throws CompilationException {
+ if (callExpr.getKind() == Expression.Kind.CALL_EXPRESSION) {
+ FunctionSignature callSignature = callExpr.getFunctionSignature();
+ if (FunctionUtil.isBuiltinFunctionSignature(callSignature)) {
+ if (FunctionUtil.isBuiltinDatasetFunction(callSignature)) {
+ Triple<DatasetFullyQualifiedName, Boolean, DatasetFullyQualifiedName> dsArgs =
+ FunctionUtil.parseDatasetFunctionArguments(callExpr);
+ if (Boolean.TRUE.equals(dsArgs.second)) {
+ DatasetFullyQualifiedName viewName = dsArgs.first;
+ ViewDecl vdTo = viewDeclMap.get(viewName);
+ if (vdTo != null) {
+ callGraph.putEdge(from, vdTo);
+ }
+ }
+ }
+ } else {
+ FunctionDecl fdTo = functionDeclMap.get(callSignature);
+ if (fdTo != null) {
+ callGraph.putEdge(from, fdTo);
+ }
+ }
+ }
+ }
+
+ public static Query createWrappedQuery(Expression expr, SourceLocation sourceLoc) {
+ Query wrappedQuery = new Query(false);
+ wrappedQuery.setSourceLocation(sourceLoc);
+ wrappedQuery.setBody(expr);
+ wrappedQuery.setTopLevel(false);
+ return wrappedQuery;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index ded91af..42a35f3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -23,12 +23,9 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
-import java.util.Set;
import java.util.function.BiFunction;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -36,19 +33,18 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.functions.FunctionConstants;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.IParser;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.IQueryRewriter;
import org.apache.asterix.lang.common.expression.AbstractCallExpression;
-import org.apache.asterix.lang.common.expression.CallExpr;
import org.apache.asterix.lang.common.expression.OrderedListTypeDefinition;
import org.apache.asterix.lang.common.expression.TypeExpression;
import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
import org.apache.asterix.lang.common.expression.UnorderedListTypeDefinition;
import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.BuiltinTypeMap;
import org.apache.asterix.metadata.entities.Dataverse;
@@ -60,7 +56,6 @@
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.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
@@ -68,10 +63,6 @@
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
-import com.google.common.graph.GraphBuilder;
-import com.google.common.graph.Graphs;
-import com.google.common.graph.MutableGraph;
-
public class FunctionUtil {
public static final String IMPORT_PRIVATE_FUNCTIONS = "import-private-functions";
@@ -208,79 +199,22 @@
};
}
- public static void checkFunctionRecursion(Map<FunctionSignature, FunctionDecl> functionDeclMap,
- java.util.function.Function<Collection<AbstractCallExpression>, GatherFunctionCallsVisitor> gfcFactory,
- SourceLocation sourceLoc) throws CompilationException {
- List<AbstractCallExpression> callList = new ArrayList<>();
- GatherFunctionCallsVisitor gfc = gfcFactory.apply(callList);
- MutableGraph<FunctionDecl> graph = GraphBuilder.directed().allowsSelfLoops(true).build();
- for (FunctionDecl fdFrom : functionDeclMap.values()) {
- callList.clear();
- fdFrom.getNormalizedFuncBody().accept(gfc, null);
- for (AbstractCallExpression callExpr : callList) {
- if (callExpr.getKind() == Expression.Kind.CALL_EXPRESSION) {
- FunctionSignature callSignature = callExpr.getFunctionSignature();
- FunctionDecl fdTo = functionDeclMap.get(callSignature);
- if (fdTo != null) {
- graph.putEdge(fdFrom, fdTo);
- }
- }
- }
+ public static List<List<Triple<DataverseName, String, String>>> getFunctionDependencies(FunctionDecl fd,
+ IQueryRewriter rewriter) throws CompilationException {
+ Expression normBody = fd.getNormalizedFuncBody();
+ if (normBody == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fd.getSourceLocation(),
+ fd.getSignature().toString());
}
- if (Graphs.hasCycle(graph)) {
- throw new CompilationException(ErrorCode.ILLEGAL_FUNCTION_RECURSION, sourceLoc);
- }
- }
- public static List<List<Triple<DataverseName, String, String>>> getFunctionDependencies(IQueryRewriter rewriter,
- Expression expression) throws CompilationException {
- List<AbstractCallExpression> functionCalls = new ArrayList<>();
- rewriter.getFunctionCalls(expression, functionCalls);
- // Duplicate elimination
- Set<FunctionSignature> seenFunctions = new HashSet<>();
- Set<Pair<DataverseName, String>> seenDatasets = new HashSet<>();
- Set<Pair<DataverseName, String>> seenSynonyms = new HashSet<>();
- //Get the List of used functions and used datasets
+ // Get the list of used functions and used datasets
List<Triple<DataverseName, String, String>> datasetDependencies = new ArrayList<>();
- List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
- List<Triple<DataverseName, String, String>> typeDependencies = Collections.emptyList();
List<Triple<DataverseName, String, String>> synonymDependencies = new ArrayList<>();
- for (AbstractCallExpression functionCall : functionCalls) {
- switch (functionCall.getKind()) {
- case CALL_EXPRESSION:
- FunctionSignature signature = functionCall.getFunctionSignature();
- if (isBuiltinDatasetFunction(signature)) {
- CallExpr callExpr = (CallExpr) functionCall;
- if (callExpr.getExprList().size() > 2) {
- // resolved via synonym -> store synonym name as a dependency
- Pair<DataverseName, String> synonymReference = parseDatasetFunctionArguments(callExpr, 2);
- if (seenSynonyms.add(synonymReference)) {
- synonymDependencies
- .add(new Triple<>(synonymReference.first, synonymReference.second, null));
- }
- } else {
- // resolved directly -> store dataset name as a dependency
- Pair<DataverseName, String> datasetReference = parseDatasetFunctionArguments(callExpr, 0);
- if (seenDatasets.add(datasetReference)) {
- datasetDependencies
- .add(new Triple<>(datasetReference.first, datasetReference.second, null));
- }
- }
- } else if (BuiltinFunctions.getBuiltinFunctionInfo(signature.createFunctionIdentifier()) == null) {
- if (seenFunctions.add(signature)) {
- functionDependencies.add(new Triple<>(signature.getDataverseName(), signature.getName(),
- Integer.toString(signature.getArity())));
- }
- }
- break;
- case WINDOW_EXPRESSION:
- // there cannot be used-defined window functions
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expression.getSourceLocation(),
- functionCall.getFunctionSignature().toString(false));
- }
- }
+ List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
+ ExpressionUtils.collectDependencies(normBody, rewriter, datasetDependencies, synonymDependencies,
+ functionDependencies);
+
+ List<Triple<DataverseName, String, String>> typeDependencies = Collections.emptyList();
return Function.createDependencies(datasetDependencies, functionDependencies, typeDependencies,
synonymDependencies);
}
@@ -303,27 +237,27 @@
&& Objects.equals(FN_DATASET_NAME, fs.getName());
}
- public static Pair<DataverseName, String> parseDatasetFunctionArguments(CallExpr datasetFn)
- throws CompilationException {
- return parseDatasetFunctionArguments(datasetFn, 0);
+ public static Triple<DatasetFullyQualifiedName, Boolean, DatasetFullyQualifiedName> parseDatasetFunctionArguments(
+ AbstractCallExpression datasetFn) throws CompilationException {
+ List<Expression> argList = datasetFn.getExprList();
+ DatasetFullyQualifiedName datasetOrViewName = parseDatasetFunctionArguments(argList, 0,
+ datasetFn.getSourceLocation(), ExpressionUtils::getStringLiteral);
+ boolean isView = argList.size() > 2 && Boolean.TRUE.equals(ExpressionUtils.getBooleanLiteral(argList.get(2)));
+ DatasetFullyQualifiedName synonymName = argList.size() > 3 ? parseDatasetFunctionArguments(argList, 3,
+ datasetFn.getSourceLocation(), ExpressionUtils::getStringLiteral) : null;
+ return new Triple<>(datasetOrViewName, isView, synonymName);
}
- public static Pair<DataverseName, String> parseDatasetFunctionArguments(CallExpr datasetFn, int startPos)
- throws CompilationException {
- return parseDatasetFunctionArguments(datasetFn.getExprList(), startPos, datasetFn.getSourceLocation(),
- ExpressionUtils::getStringLiteral);
- }
-
- public static Pair<DataverseName, String> parseDatasetFunctionArguments(AbstractFunctionCallExpression datasetFn)
+ public static DatasetFullyQualifiedName parseDatasetFunctionArguments(AbstractFunctionCallExpression datasetFn)
throws CompilationException {
return parseDatasetFunctionArguments(datasetFn.getArguments(), 0, datasetFn.getSourceLocation(),
FunctionUtil::getStringConstant);
}
- private static <T> Pair<DataverseName, String> parseDatasetFunctionArguments(List<T> datasetFnArgs, int startPos,
- SourceLocation sourceLoc, java.util.function.Function<T, String> argExtractFunction)
+ private static <T> DatasetFullyQualifiedName parseDatasetFunctionArguments(List<T> datasetFnArgs, int startPos,
+ SourceLocation sourceLoc, java.util.function.Function<T, String> stringAccessor)
throws CompilationException {
- String dataverseNameArg = argExtractFunction.apply(datasetFnArgs.get(startPos));
+ String dataverseNameArg = stringAccessor.apply(datasetFnArgs.get(startPos));
if (dataverseNameArg == null) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Invalid argument to dataset()");
}
@@ -333,11 +267,11 @@
} catch (AsterixException e) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, e, "Invalid argument to dataset()");
}
- String datasetName = argExtractFunction.apply(datasetFnArgs.get(startPos + 1));
+ String datasetName = stringAccessor.apply(datasetFnArgs.get(startPos + 1));
if (datasetName == null) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Invalid argument to dataset()");
}
- return new Pair<>(dataverseName, datasetName);
+ return new DatasetFullyQualifiedName(dataverseName, datasetName);
}
private static String getStringConstant(Mutable<ILogicalExpression> arg) {
@@ -349,17 +283,6 @@
return (value != null) && Boolean.parseBoolean(value.toLowerCase());
}
- public static Map<FunctionSignature, FunctionDecl> getFunctionMap(List<FunctionDecl> declaredFunctions) {
- if (declaredFunctions == null || declaredFunctions.isEmpty()) {
- return Collections.emptyMap();
- }
- Map<FunctionSignature, FunctionDecl> result = new HashMap<>();
- for (FunctionDecl fd : declaredFunctions) {
- result.put(fd.getSignature(), fd);
- }
- return result;
- }
-
public static FunctionDecl parseStoredFunction(Function function, IParserFactory parserFactory,
IWarningCollector warningCollector, SourceLocation sourceLoc) throws CompilationException {
if (!function.getLanguage().equals(parserFactory.getLanguage())) {
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 28639c1..0b9c895 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
@@ -70,10 +70,10 @@
case LIST_CONSTRUCTOR_EXPRESSION:
return parseList((ListConstructor) expr);
default:
- throw new RuntimeDataException(ErrorCode.PARSE_ERROR, NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE,
- Expression.Kind.LITERAL_EXPRESSION.toString(),
- Expression.Kind.RECORD_CONSTRUCTOR_EXPRESSION.toString(),
- Expression.Kind.LIST_CONSTRUCTOR_EXPRESSION.toString());
+ throw new RuntimeDataException(ErrorCode.PARSE_ERROR,
+ String.format(NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE, Expression.Kind.LITERAL_EXPRESSION,
+ Expression.Kind.RECORD_CONSTRUCTOR_EXPRESSION,
+ Expression.Kind.LIST_CONSTRUCTOR_EXPRESSION));
}
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ViewUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ViewUtil.java
new file mode 100644
index 0000000..cb704b2
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ViewUtil.java
@@ -0,0 +1,192 @@
+/*
+ * 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.lang.common.util;
+
+import static org.apache.asterix.metadata.utils.TypeUtil.getTemporalFormat;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.common.base.IQueryRewriter;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.FieldAccessor;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.metadata.entities.ViewDetails;
+import org.apache.asterix.metadata.utils.TypeUtil;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public final class ViewUtil {
+
+ private ViewUtil() {
+ }
+
+ public static ViewDecl parseStoredView(DatasetFullyQualifiedName viewName, ViewDetails view,
+ IParserFactory parserFactory, IWarningCollector warningCollector, SourceLocation sourceLoc)
+ throws CompilationException {
+ IParser parser = parserFactory.createParser(new StringReader(view.getViewBody()));
+ try {
+ ViewDecl viewDecl = parser.parseViewBody(viewName);
+ viewDecl.setSourceLocation(sourceLoc);
+ if (warningCollector != null) {
+ parser.getWarnings(warningCollector);
+ }
+ return viewDecl;
+ } catch (CompilationException e) {
+ throw new CompilationException(ErrorCode.COMPILATION_BAD_VIEW_DEFINITION, e, sourceLoc, viewName,
+ e.getMessage());
+ }
+ }
+
+ public static List<List<Triple<DataverseName, String, String>>> getViewDependencies(ViewDecl viewDecl,
+ List<ViewDetails.ForeignKey> foreignKeys, IQueryRewriter rewriter) throws CompilationException {
+ Expression normBody = viewDecl.getNormalizedViewBody();
+ if (normBody == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, viewDecl.getSourceLocation(),
+ viewDecl.getViewName().toString());
+ }
+
+ // Get the list of used functions and used datasets
+ List<Triple<DataverseName, String, String>> datasetDependencies = new ArrayList<>();
+ List<Triple<DataverseName, String, String>> synonymDependencies = new ArrayList<>();
+ List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
+ ExpressionUtils.collectDependencies(normBody, rewriter, datasetDependencies, synonymDependencies,
+ functionDependencies);
+
+ if (foreignKeys != null) {
+ DatasetFullyQualifiedName viewName = viewDecl.getViewName();
+ for (ViewDetails.ForeignKey foreignKey : foreignKeys) {
+ DatasetFullyQualifiedName refName = foreignKey.getReferencedDatasetName();
+ boolean isSelfReference = refName.equals(viewName);
+ if (isSelfReference || containsDependency(datasetDependencies, refName)) {
+ continue;
+ }
+ datasetDependencies.add(new Triple<>(refName.getDataverseName(), refName.getDatasetName(), null));
+ }
+ }
+
+ List<Triple<DataverseName, String, String>> typeDependencies = Collections.emptyList();
+ return ViewDetails.createDependencies(datasetDependencies, functionDependencies, typeDependencies,
+ synonymDependencies);
+ }
+
+ private static boolean containsDependency(List<Triple<DataverseName, String, String>> inList,
+ DatasetFullyQualifiedName searchName) {
+ for (Triple<DataverseName, String, String> d : inList) {
+ if (d.first.equals(searchName.getDataverseName()) && d.second.equals(searchName.getDatasetName())) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public static void validateViewItemType(ARecordType recordType, SourceLocation sourceLoc)
+ throws CompilationException {
+ if (recordType.isOpen()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "view type cannot have open fields");
+ }
+ String[] fieldNames = recordType.getFieldNames();
+ IAType[] fieldTypes = recordType.getFieldTypes();
+ for (int i = 0, n = fieldNames.length; i < n; i++) {
+ IAType fieldType = fieldTypes[i];
+ IAType primeType;
+ if (fieldType.getTypeTag() == ATypeTag.UNION) {
+ AUnionType unionType = (AUnionType) fieldType;
+ if (!unionType.isNullableType()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ String.format("Invalid type for field %s. Optional type must allow NULL", fieldNames[i]));
+ }
+ primeType = unionType.getActualType();
+ } else {
+ primeType = fieldType;
+ }
+ if (TypeUtil.getTypeConstructorDefaultNull(primeType, false) == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, "view",
+ primeType.getTypeName());
+ }
+ }
+ }
+
+ public static Expression createTypeConvertExpression(Expression inExpr, IAType targetType,
+ Triple<String, String, String> temporalDataFormat, DatasetFullyQualifiedName viewName,
+ SourceLocation sourceLoc) throws CompilationException {
+ String format = temporalDataFormat != null ? getTemporalFormat(targetType, temporalDataFormat) : null;
+ boolean withFormat = format != null;
+ FunctionIdentifier constrFid = TypeUtil.getTypeConstructorDefaultNull(targetType, withFormat);
+ if (constrFid == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, viewName.toString(),
+ targetType.getTypeName());
+ }
+ List<Expression> convertArgList = new ArrayList<>(2);
+ convertArgList.add(inExpr);
+ if (format != null) {
+ LiteralExpr formatExpr = new LiteralExpr(new StringLiteral(format));
+ formatExpr.setSourceLocation(sourceLoc);
+ convertArgList.add(formatExpr);
+ }
+ CallExpr convertExpr = new CallExpr(new FunctionSignature(constrFid), convertArgList);
+ convertExpr.setSourceLocation(inExpr.getSourceLocation());
+ return convertExpr;
+ }
+
+ public static Expression createNotIsNullExpression(Expression inExpr, SourceLocation sourceLoc) {
+ List<Expression> isNullArgs = new ArrayList<>(1);
+ isNullArgs.add(inExpr);
+ CallExpr isNullExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.IS_NULL), isNullArgs);
+ isNullExpr.setSourceLocation(sourceLoc);
+ List<Expression> notExprArgs = new ArrayList<>(1);
+ notExprArgs.add(isNullExpr);
+ CallExpr notExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT), notExprArgs);
+ notExpr.setSourceLocation(sourceLoc);
+ return notExpr;
+ }
+
+ public static Expression createFieldAccessExpression(VarIdentifier inVar, String fieldName,
+ SourceLocation sourceLoc) {
+ VariableExpr inVarRef = new VariableExpr(inVar);
+ inVarRef.setSourceLocation(sourceLoc);
+ FieldAccessor fa = new FieldAccessor(inVarRef, new Identifier(fieldName));
+ fa.setSourceLocation(sourceLoc);
+ return fa;
+ }
+
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index cba6bb5..52775d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -19,6 +19,7 @@
package org.apache.asterix.lang.common.visitor;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -26,6 +27,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.Expression.Kind;
import org.apache.asterix.lang.common.base.ILangExpression;
@@ -52,6 +54,7 @@
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.InsertStatement;
import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.QuantifiedPair;
import org.apache.asterix.lang.common.struct.VarIdentifier;
@@ -59,6 +62,7 @@
import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -68,12 +72,15 @@
protected final Map<FunctionSignature, FunctionDecl> usedUDFs;
+ protected final Map<DatasetFullyQualifiedName, ViewDecl> usedViews;
+
protected final CloneAndSubstituteVariablesVisitor cloneVisitor;
public AbstractInlineUdfsVisitor(LangRewritingContext context, Map<FunctionSignature, FunctionDecl> usedUDFs,
- CloneAndSubstituteVariablesVisitor cloneVisitor) {
+ Map<DatasetFullyQualifiedName, ViewDecl> usedViews, CloneAndSubstituteVariablesVisitor cloneVisitor) {
this.context = context;
this.usedUDFs = usedUDFs;
+ this.usedViews = usedViews;
this.cloneVisitor = cloneVisitor;
}
@@ -89,18 +96,12 @@
@Override
public Boolean visit(Query q, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(q.getBody());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(q.getBody());
q.setBody(p.second);
return p.first;
}
@Override
- public Boolean visit(FunctionDecl fd, Void arg) throws CompilationException {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fd.getSourceLocation(),
- fd.getSignature().toString());
- }
-
- @Override
public Boolean visit(ListConstructor lc, Void arg) throws CompilationException {
Pair<Boolean, List<Expression>> p = inlineUdfsInExprList(lc.getExprList());
lc.setExprList(p.second);
@@ -111,10 +112,10 @@
public Boolean visit(RecordConstructor rc, Void arg) throws CompilationException {
boolean changed = false;
for (FieldBinding b : rc.getFbList()) {
- Pair<Boolean, Expression> leftExprInlined = inlineUdfsInExpr(b.getLeftExpr());
+ Pair<Boolean, Expression> leftExprInlined = inlineUdfsAndViewsInExpr(b.getLeftExpr());
b.setLeftExpr(leftExprInlined.second);
changed = changed || leftExprInlined.first;
- Pair<Boolean, Expression> rightExprInlined = inlineUdfsInExpr(b.getRightExpr());
+ Pair<Boolean, Expression> rightExprInlined = inlineUdfsAndViewsInExpr(b.getRightExpr());
b.setRightExpr(rightExprInlined.second);
changed = changed || rightExprInlined.first;
}
@@ -127,7 +128,7 @@
callExpr.setExprList(p.second);
boolean changed = p.first;
if (callExpr.hasAggregateFilterExpr()) {
- Pair<Boolean, Expression> be = inlineUdfsInExpr(callExpr.getAggregateFilterExpr());
+ Pair<Boolean, Expression> be = inlineUdfsAndViewsInExpr(callExpr.getAggregateFilterExpr());
callExpr.setAggregateFilterExpr(be.second);
changed |= be.first;
}
@@ -143,25 +144,25 @@
@Override
public Boolean visit(FieldAccessor fa, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(fa.getExpr());
fa.setExpr(p.second);
return p.first;
}
@Override
public Boolean visit(IndexAccessor fa, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(fa.getExpr());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(fa.getExpr());
fa.setExpr(p.second);
return p.first;
}
@Override
public Boolean visit(IfExpr ifexpr, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p1 = inlineUdfsInExpr(ifexpr.getCondExpr());
+ Pair<Boolean, Expression> p1 = inlineUdfsAndViewsInExpr(ifexpr.getCondExpr());
ifexpr.setCondExpr(p1.second);
- Pair<Boolean, Expression> p2 = inlineUdfsInExpr(ifexpr.getThenExpr());
+ Pair<Boolean, Expression> p2 = inlineUdfsAndViewsInExpr(ifexpr.getThenExpr());
ifexpr.setThenExpr(p2.second);
- Pair<Boolean, Expression> p3 = inlineUdfsInExpr(ifexpr.getElseExpr());
+ Pair<Boolean, Expression> p3 = inlineUdfsAndViewsInExpr(ifexpr.getElseExpr());
ifexpr.setElseExpr(p3.second);
return p1.first || p2.first || p3.first;
}
@@ -170,27 +171,27 @@
public Boolean visit(QuantifiedExpression qe, Void arg) throws CompilationException {
boolean changed = false;
for (QuantifiedPair t : qe.getQuantifiedList()) {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(t.getExpr());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(t.getExpr());
t.setExpr(p.second);
if (p.first) {
changed = true;
}
}
- Pair<Boolean, Expression> p2 = inlineUdfsInExpr(qe.getSatisfiesExpr());
+ Pair<Boolean, Expression> p2 = inlineUdfsAndViewsInExpr(qe.getSatisfiesExpr());
qe.setSatisfiesExpr(p2.second);
return changed || p2.first;
}
@Override
public Boolean visit(LetClause lc, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(lc.getBindingExpr());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(lc.getBindingExpr());
lc.setBindingExpr(p.second);
return p.first;
}
@Override
public Boolean visit(WhereClause wc, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(wc.getWhereExpr());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(wc.getWhereExpr());
wc.setWhereExpr(p.second);
return p.first;
}
@@ -235,12 +236,12 @@
public Boolean visit(LimitClause lc, Void arg) throws CompilationException {
boolean changed = false;
if (lc.hasLimitExpr()) {
- Pair<Boolean, Expression> p1 = inlineUdfsInExpr(lc.getLimitExpr());
+ Pair<Boolean, Expression> p1 = inlineUdfsAndViewsInExpr(lc.getLimitExpr());
lc.setLimitExpr(p1.second);
changed = p1.first;
}
if (lc.hasOffset()) {
- Pair<Boolean, Expression> p2 = inlineUdfsInExpr(lc.getOffset());
+ Pair<Boolean, Expression> p2 = inlineUdfsAndViewsInExpr(lc.getOffset());
lc.setOffset(p2.second);
changed |= p2.first;
}
@@ -267,53 +268,102 @@
boolean changed = false;
Expression returnExpression = insert.getReturnExpression();
if (returnExpression != null) {
- Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsInExpr(returnExpression);
+ Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsAndViewsInExpr(returnExpression);
insert.setReturnExpression(rewrittenReturnExpr.second);
changed |= rewrittenReturnExpr.first;
}
- Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsInExpr(insert.getBody());
+ Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsAndViewsInExpr(insert.getBody());
insert.setBody(rewrittenBodyExpression.second);
return changed || rewrittenBodyExpression.first;
}
- protected Pair<Boolean, Expression> inlineUdfsInExpr(Expression expr) throws CompilationException {
+ protected Pair<Boolean, Expression> inlineUdfsAndViewsInExpr(Expression expr) throws CompilationException {
if (expr.getKind() != Kind.CALL_EXPRESSION) {
boolean r = expr.accept(this, null);
return new Pair<>(r, expr);
}
CallExpr f = (CallExpr) expr;
boolean r = expr.accept(this, null);
+
+ List<LetClause> letClauses;
+ VariableSubstitutionEnvironment bodyVarSubst;
+ Expression normBodyExpr;
+
FunctionSignature fs = f.getFunctionSignature();
if (FunctionUtil.isBuiltinFunctionSignature(fs)) {
- return new Pair<>(r, expr);
+ if (!FunctionUtil.isBuiltinDatasetFunction(fs)) {
+ return new Pair<>(r, expr);
+ }
+ Triple<DatasetFullyQualifiedName, Boolean, DatasetFullyQualifiedName> dsArgs =
+ FunctionUtil.parseDatasetFunctionArguments(f);
+ if (!Boolean.TRUE.equals(dsArgs.second)) {
+ // not a view
+ return new Pair<>(r, expr);
+ }
+ DatasetFullyQualifiedName viewName = dsArgs.first;
+ ViewDecl implem = usedViews.get(viewName);
+ if (implem == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_VIEW, f.getSourceLocation(), viewName);
+ }
+ // it's one of the views we want to inline
+ letClauses = Collections.emptyList();
+ bodyVarSubst = new VariableSubstitutionEnvironment();
+ normBodyExpr = implem.getNormalizedViewBody();
+ if (normBodyExpr == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, f.getSourceLocation(),
+ viewName.toString());
+ }
+ } else {
+ FunctionDecl implem = usedUDFs.get(fs);
+ if (implem == null) {
+ //it's an external UDF
+ return new Pair<>(r, expr);
+ }
+ // it's one of the functions we want to inline
+ boolean isVarargs = implem.getSignature().getArity() == FunctionIdentifier.VARARGS;
+ Pair<List<LetClause>, VariableSubstitutionEnvironment> clausesAndSubst =
+ createFunctionParametersSubstitution(implem.getParamList(), isVarargs, f.getExprList(),
+ f.getSourceLocation());
+ letClauses = clausesAndSubst.first;
+ bodyVarSubst = clausesAndSubst.second;
+ normBodyExpr = implem.getNormalizedFuncBody();
+ if (normBodyExpr == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, f.getSourceLocation(),
+ fs.toString());
+ }
}
+
if (f.hasAggregateFilterExpr()) {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_USE_OF_FILTER_CLAUSE, f.getSourceLocation());
}
- FunctionDecl implem = usedUDFs.get(fs);
- if (implem == null) {
- throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, f.getSourceLocation(), fs.toString());
- }
- // it's one of the functions we want to inline
- List<Expression> argList = f.getExprList();
+
+ Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = normBodyExpr.accept(cloneVisitor, bodyVarSubst);
+ Expression resExpr = letClauses.isEmpty() ? (Expression) p2.first
+ : generateQueryExpression(letClauses, (Expression) p2.first);
+ return new Pair<>(true, resExpr);
+ }
+
+ private Pair<List<LetClause>, VariableSubstitutionEnvironment> createFunctionParametersSubstitution(
+ List<VarIdentifier> paramList, boolean isVarargs, List<Expression> argList, SourceLocation sourceLoc)
+ throws CompilationException {
int argCount = argList.size();
List<LetClause> clauses = new ArrayList<>(argCount + 1);
List<Expression> argVars = new ArrayList<>(argCount);
- for (Expression e : f.getExprList()) {
+ for (Expression e : argList) {
// Obs: we could do smth about passing also literals, or let
// variable inlining to take care of this.
VarIdentifier argVar;
if (e.getKind() == Kind.VARIABLE_EXPRESSION) {
argVar = ((VariableExpr) e).getVar();
} else {
- SourceLocation sourceLoc = e.getSourceLocation();
+ SourceLocation argSourceLoc = e.getSourceLocation();
argVar = context.newVariable();
Pair<ILangExpression, VariableSubstitutionEnvironment> p1 =
e.accept(cloneVisitor, new VariableSubstitutionEnvironment());
VariableExpr newVRef1 = new VariableExpr(argVar);
- newVRef1.setSourceLocation(sourceLoc);
+ newVRef1.setSourceLocation(argSourceLoc);
LetClause c = new LetClause(newVRef1, (Expression) p1.first);
- c.setSourceLocation(sourceLoc);
+ c.setSourceLocation(argSourceLoc);
clauses.add(c);
}
@@ -323,49 +373,34 @@
}
VariableSubstitutionEnvironment subst = new VariableSubstitutionEnvironment();
- List<VarIdentifier> paramList = implem.getParamList();
- if (implem.getSignature().getArity() == FunctionIdentifier.VARARGS) {
+ if (isVarargs) {
if (paramList.size() != 1) {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
- paramList.size());
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, paramList.size());
}
VarIdentifier paramVarargs = paramList.get(0);
CallExpr argsListExpr =
new CallExpr(new FunctionSignature(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR), argVars);
- argsListExpr.setSourceLocation(expr.getSourceLocation());
+ argsListExpr.setSourceLocation(sourceLoc);
VarIdentifier argsVar = context.newVariable();
VariableExpr argsVarRef1 = new VariableExpr(argsVar);
- argsVarRef1.setSourceLocation(expr.getSourceLocation());
+ argsVarRef1.setSourceLocation(sourceLoc);
LetClause c = new LetClause(argsVarRef1, argsListExpr);
- c.setSourceLocation(expr.getSourceLocation());
+ c.setSourceLocation(sourceLoc);
clauses.add(c);
VariableExpr argsVarRef2 = new VariableExpr(argsVar);
- argsVarRef2.setSourceLocation(expr.getSourceLocation());
+ argsVarRef2.setSourceLocation(sourceLoc);
subst.addSubstituion(new VariableExpr(paramVarargs), argsVarRef2);
} else {
if (paramList.size() != argCount) {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, expr.getSourceLocation(),
- paramList.size());
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, paramList.size());
}
for (int i = 0; i < argCount; i++) {
subst.addSubstituion(new VariableExpr(paramList.get(i)), argVars.get(i));
}
}
-
- Expression funcBodyNorm = implem.getNormalizedFuncBody();
- if (funcBodyNorm == null) {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, f.getSourceLocation(), fs.toString());
- }
- Pair<ILangExpression, VariableSubstitutionEnvironment> p2 = funcBodyNorm.accept(cloneVisitor, subst);
- Expression resExpr;
- if (clauses.isEmpty()) {
- resExpr = (Expression) p2.first;
- } else {
- resExpr = generateQueryExpression(clauses, (Expression) p2.first);
- }
- return new Pair<>(true, resExpr);
+ return new Pair<>(clauses, subst);
}
protected Pair<Boolean, List<Expression>> inlineUdfsInExprList(List<Expression> exprList)
@@ -373,7 +408,7 @@
List<Expression> newList = new ArrayList<>(exprList.size());
boolean changed = false;
for (Expression e : exprList) {
- Pair<Boolean, Expression> be = inlineUdfsInExpr(e);
+ Pair<Boolean, Expression> be = inlineUdfsAndViewsInExpr(e);
newList.add(be.second);
changed |= be.first;
}
@@ -385,7 +420,7 @@
List<GbyVariableExpressionPair> newList = new ArrayList<>(gbyPairList.size());
boolean changed = false;
for (GbyVariableExpressionPair p : gbyPairList) {
- Pair<Boolean, Expression> be = inlineUdfsInExpr(p.getExpr());
+ Pair<Boolean, Expression> be = inlineUdfsAndViewsInExpr(p.getExpr());
newList.add(new GbyVariableExpressionPair(p.getVar(), be.second));
changed |= be.first;
}
@@ -397,7 +432,7 @@
List<Pair<Expression, Identifier>> newList = new ArrayList<>(fieldList.size());
boolean changed = false;
for (Pair<Expression, Identifier> p : fieldList) {
- Pair<Boolean, Expression> be = inlineUdfsInExpr(p.first);
+ Pair<Boolean, Expression> be = inlineUdfsAndViewsInExpr(p.first);
newList.add(new Pair<>(be.second, p.second));
changed |= be.first;
}
@@ -409,7 +444,7 @@
Map<Expression, VariableExpr> newMap = new HashMap<>();
boolean changed = false;
for (Map.Entry<Expression, VariableExpr> me : varMap.entrySet()) {
- Pair<Boolean, Expression> be = inlineUdfsInExpr(me.getKey());
+ Pair<Boolean, Expression> be = inlineUdfsAndViewsInExpr(me.getKey());
newMap.put(be.second, me.getValue());
changed |= be.first;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
index 4b30c97..d73c264 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/CloneAndSubstituteVariablesVisitor.java
@@ -250,7 +250,8 @@
VariableSubstitutionEnvironment env) throws CompilationException {
List<Expression> exprList =
VariableCloneAndSubstitutionUtil.visitAndCloneExprList(oc.getOrderbyList(), env, this);
- OrderbyClause oc2 = new OrderbyClause(exprList, new ArrayList<>(oc.getModifierList()));
+ OrderbyClause oc2 = new OrderbyClause(exprList, new ArrayList<>(oc.getModifierList()),
+ new ArrayList<>(oc.getNullModifierList()));
oc2.setNumFrames(oc.getNumFrames());
oc2.setNumTuples(oc.getNumTuples());
oc2.setRangeMap(oc.getRangeMap());
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 2650ca3..0ddbeb4 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -39,6 +39,7 @@
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
import org.apache.asterix.lang.common.clause.OrderbyClause;
+import org.apache.asterix.lang.common.clause.OrderbyClause.NullOrderModifier;
import org.apache.asterix.lang.common.clause.OrderbyClause.OrderModifier;
import org.apache.asterix.lang.common.clause.UpdateClause;
import org.apache.asterix.lang.common.clause.WhereClause;
@@ -76,6 +77,7 @@
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
+import org.apache.asterix.lang.common.statement.CreateViewStatement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -104,6 +106,8 @@
import org.apache.asterix.lang.common.statement.TypeDecl;
import org.apache.asterix.lang.common.statement.TypeDropStatement;
import org.apache.asterix.lang.common.statement.UpdateStatement;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.statement.ViewDropStatement;
import org.apache.asterix.lang.common.statement.WriteStatement;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.OperatorType;
@@ -314,7 +318,7 @@
@Override
public Void visit(OrderbyClause oc, Integer step) throws CompilationException {
out.print(skip(step) + "order by ");
- printDelimitedObyExpressions(oc.getOrderbyList(), oc.getModifierList(), step);
+ printDelimitedObyExpressions(oc.getOrderbyList(), oc.getModifierList(), oc.getNullModifierList(), step);
out.println();
return null;
}
@@ -957,6 +961,32 @@
return null;
}
+ @Override
+ public Void visit(CreateViewStatement cvs, Integer step) throws CompilationException {
+ out.print(skip(step) + CREATE + generateOrReplace(cvs.getReplaceIfExists()) + " view ");
+ out.print(generateIfNotExists(cvs.getIfNotExists()));
+ out.print(generateFullName(cvs.getDataverseName(), cvs.getViewName()));
+ out.print(" as ");
+ out.print(cvs.getViewBody());
+ out.println(SEMICOLON);
+ return null;
+ }
+
+ @Override
+ public Void visit(ViewDropStatement vds, Integer step) throws CompilationException {
+ out.print(skip(step) + "drop view ");
+ out.print(generateFullName(vds.getDataverseName(), vds.getViewName()));
+ out.print(generateIfExists(vds.getIfExists()));
+ out.println(SEMICOLON);
+ return null;
+ }
+
+ @Override
+ public Void visit(ViewDecl vd, Integer arg) throws CompilationException {
+ // this statement is internal
+ return null;
+ }
+
protected void printConfiguration(Map<String, String> properties) {
if (properties.size() > 0) {
out.print("(");
@@ -1008,8 +1038,8 @@
}
}
- protected void printDelimitedObyExpressions(List<Expression> list, List<OrderModifier> mlist, Integer step)
- throws CompilationException {
+ protected void printDelimitedObyExpressions(List<Expression> list, List<OrderModifier> mlist,
+ List<NullOrderModifier> nlist, Integer step) throws CompilationException {
int index = 0;
int size = list.size();
for (Expression expr : list) {
@@ -1018,6 +1048,11 @@
if (orderModifier != OrderModifier.ASC) {
out.print(orderModifier.toString().toLowerCase());
}
+ NullOrderModifier nullModifier = nlist.get(index);
+ if (nullModifier != null) {
+ out.print(" nulls ");
+ out.print(nullModifier.toString().toLowerCase());
+ }
if (++index < size) {
out.print(COMMA);
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
index 25f4103..b7cf7af 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
@@ -47,7 +47,6 @@
import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
import org.apache.asterix.lang.common.expression.UnaryExpr;
import org.apache.asterix.lang.common.expression.VariableExpr;
-import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.InsertStatement;
import org.apache.asterix.lang.common.statement.Query;
import org.apache.asterix.lang.common.struct.Identifier;
@@ -242,9 +241,4 @@
}
return null;
}
-
- @Override
- public Void visit(FunctionDecl fd, Void arg) throws CompilationException {
- return null;
- }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
index e756eee..79e90a4 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
@@ -31,6 +31,7 @@
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
import org.apache.asterix.lang.common.clause.OrderbyClause;
+import org.apache.asterix.lang.common.clause.OrderbyClause.NullOrderModifier;
import org.apache.asterix.lang.common.clause.OrderbyClause.OrderModifier;
import org.apache.asterix.lang.common.clause.WhereClause;
import org.apache.asterix.lang.common.expression.CallExpr;
@@ -241,10 +242,14 @@
public Void visit(OrderbyClause oc, Integer step) throws CompilationException {
out.println(skip(step) + "Orderby");
List<OrderModifier> mlist = oc.getModifierList();
+ List<NullOrderModifier> nlist = oc.getNullModifierList();
List<Expression> list = oc.getOrderbyList();
for (int i = 0; i < list.size(); i++) {
list.get(i).accept(this, step + 1);
- out.println(skip(step + 1) + mlist.get(i).toString());
+ OrderModifier orderModifier = mlist.get(i);
+ NullOrderModifier nullOrderModifier = nlist.get(i);
+ out.println(
+ skip(step + 1) + orderModifier + (nullOrderModifier != null ? " NULLS " + nullOrderModifier : ""));
}
out.println();
return null;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 0b4f56b..91fe664 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -37,6 +37,7 @@
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
+import org.apache.asterix.lang.common.statement.CreateViewStatement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -63,6 +64,8 @@
import org.apache.asterix.lang.common.statement.TypeDecl;
import org.apache.asterix.lang.common.statement.TypeDropStatement;
import org.apache.asterix.lang.common.statement.UpdateStatement;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.statement.ViewDropStatement;
import org.apache.asterix.lang.common.statement.WriteStatement;
public abstract class AbstractQueryExpressionVisitor<R, T> implements ILangVisitor<R, T> {
@@ -291,4 +294,19 @@
public R visit(LibraryDropStatement del, T arg) throws CompilationException {
return null;
}
+
+ @Override
+ public R visit(CreateViewStatement cvs, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
+ public R visit(ViewDropStatement vds, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
+ public R visit(ViewDecl vd, T arg) throws CompilationException {
+ return null;
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index e36ec73..c2b1311 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -54,6 +54,7 @@
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
+import org.apache.asterix.lang.common.statement.CreateViewStatement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -80,6 +81,8 @@
import org.apache.asterix.lang.common.statement.TypeDecl;
import org.apache.asterix.lang.common.statement.TypeDropStatement;
import org.apache.asterix.lang.common.statement.UpdateStatement;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.statement.ViewDropStatement;
import org.apache.asterix.lang.common.statement.WriteStatement;
public interface ILangVisitor<R, T> {
@@ -207,4 +210,10 @@
R visit(CompactStatement del, T arg) throws CompilationException;
R visit(ListSliceExpression expression, T arg) throws CompilationException;
+
+ R visit(CreateViewStatement cvs, T arg) throws CompilationException;
+
+ R visit(ViewDropStatement vds, T arg) throws CompilationException;
+
+ R visit(ViewDecl vd, T arg) throws CompilationException;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
index 18b7a24..be3ebda 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
@@ -19,8 +19,11 @@
package org.apache.asterix.lang.sqlpp.clause;
+import java.util.Objects;
+
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.asterix.lang.sqlpp.optype.JoinType;
@@ -30,10 +33,21 @@
private final JoinType joinType;
+ private Literal.Type outerJoinMissingValueType;
+
public JoinClause(JoinType joinType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar,
- Expression conditionExpr) {
+ Expression conditionExpr, Literal.Type outerJoinMissingValueType) {
super(rightExpr, rightVar, rightPosVar, conditionExpr);
this.joinType = joinType;
+ setOuterJoinMissingValueType(outerJoinMissingValueType);
+ }
+
+ public Literal.Type getOuterJoinMissingValueType() {
+ return outerJoinMissingValueType;
+ }
+
+ public void setOuterJoinMissingValueType(Literal.Type outerJoinMissingValueType) {
+ this.outerJoinMissingValueType = validateMissingValueType(joinType, outerJoinMissingValueType);
}
@Override
@@ -52,7 +66,8 @@
@Override
public int hashCode() {
- return 31 * super.hashCode() + joinType.hashCode();
+ return 31 * super.hashCode() + 31 * joinType.hashCode()
+ + (outerJoinMissingValueType != null ? outerJoinMissingValueType.hashCode() : 0);
}
@Override
@@ -64,6 +79,28 @@
return false;
}
JoinClause target = (JoinClause) object;
- return super.equals(target) && joinType.equals(target.getJoinType());
+ return super.equals(target) && joinType.equals(target.getJoinType())
+ && Objects.equals(outerJoinMissingValueType, target.outerJoinMissingValueType);
+ }
+
+ private static Literal.Type validateMissingValueType(JoinType joinType, Literal.Type missingValueType) {
+ switch (joinType) {
+ case INNER:
+ if (missingValueType != null) {
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ return null;
+ case LEFTOUTER:
+ case RIGHTOUTER:
+ switch (Objects.requireNonNull(missingValueType)) {
+ case MISSING:
+ case NULL:
+ return missingValueType;
+ default:
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ default:
+ throw new IllegalStateException(String.valueOf(joinType));
+ }
}
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
index 7f353d6..fbfc5a2 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/Projection.java
@@ -29,16 +29,58 @@
public class Projection extends AbstractClause {
+ public enum Kind {
+ NAMED_EXPR, // expr AS name
+ STAR, // *
+ VAR_STAR, // variable.*
+ EVERY_VAR_STAR // *.* (currently only used in SQL-compatible mode)
+ }
+
+ private Kind kind;
private Expression expr;
private String name;
- private boolean star;
- private boolean varStar;
- public Projection(Expression expr, String name, boolean star, boolean varStar) {
+ public Projection(Kind kind, Expression expr, String name) {
+ validateKind(kind, expr, name);
+ this.kind = kind;
this.expr = expr;
this.name = name;
- this.star = star;
- this.varStar = varStar;
+ }
+
+ private static void validateKind(Kind kind, Expression expr, String name) {
+ switch (kind) {
+ case NAMED_EXPR:
+ if (expr == null) {
+ throw new NullPointerException();
+ }
+ break;
+ case STAR:
+ case EVERY_VAR_STAR:
+ if (expr != null || name != null) {
+ throw new IllegalArgumentException();
+ }
+ break;
+ case VAR_STAR:
+ if (expr == null) {
+ throw new NullPointerException();
+ }
+ if (name != null) {
+ throw new IllegalArgumentException();
+ }
+ break;
+ default:
+ throw new IllegalArgumentException();
+ }
+ }
+
+ @Deprecated
+ public Projection(Expression expr, String name, boolean star, boolean varStar) {
+ this(asKind(star, varStar), expr, name);
+ }
+
+ @Deprecated
+ private static Kind asKind(boolean star, boolean varStar) {
+ return star ? Kind.STAR : varStar ? Kind.VAR_STAR : Kind.NAMED_EXPR;
}
@Override
@@ -59,6 +101,10 @@
this.expr = expr;
}
+ public boolean hasExpression() {
+ return expr != null;
+ }
+
public String getName() {
return name;
}
@@ -71,22 +117,43 @@
return name != null;
}
- public boolean star() {
- return star;
+ public Kind getKind() {
+ return kind;
}
+ public void setKind(Kind kind) {
+ this.kind = kind;
+ }
+
+ @Deprecated
+ public boolean star() {
+ return kind == Kind.STAR;
+ }
+
+ @Deprecated
public boolean varStar() {
- return varStar;
+ return kind == Kind.VAR_STAR;
}
@Override
public String toString() {
- return star ? "*" : (String.valueOf(expr) + (varStar ? ".*" : (hasName() ? " as " + getName() : "")));
+ switch (kind) {
+ case NAMED_EXPR:
+ return expr + (hasName() ? " as " + getName() : "");
+ case STAR:
+ return "*";
+ case VAR_STAR:
+ return expr + ".*";
+ case EVERY_VAR_STAR:
+ return "*.*";
+ default:
+ throw new IllegalStateException();
+ }
}
@Override
public int hashCode() {
- return Objects.hash(expr, varStar, name, star);
+ return Objects.hash(kind, expr, name);
}
@Override
@@ -98,7 +165,6 @@
return false;
}
Projection target = (Projection) object;
- return Objects.equals(expr, target.expr) && Objects.equals(name, target.name) && varStar == target.varStar
- && star == target.star;
+ return kind == target.kind && Objects.equals(expr, target.expr) && Objects.equals(name, target.name);
}
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
index 3b550a3..ea6bac9 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/SelectClause.java
@@ -48,6 +48,16 @@
return ClauseType.SELECT_CLAUSE;
}
+ public void setSelectElement(SelectElement selectElement) {
+ this.selectElement = selectElement;
+ this.selectRegular = null;
+ }
+
+ public void setSelectRegular(SelectRegular selectRegular) {
+ this.selectRegular = selectRegular;
+ this.selectElement = null;
+ }
+
public SelectElement getSelectElement() {
return selectElement;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
index f035be4..054ff26 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
@@ -19,8 +19,11 @@
package org.apache.asterix.lang.sqlpp.clause;
+import java.util.Objects;
+
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.asterix.lang.sqlpp.optype.UnnestType;
@@ -30,9 +33,21 @@
private final UnnestType unnestType;
- public UnnestClause(UnnestType unnestType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar) {
+ private Literal.Type outerUnnestMissingValueType;
+
+ public UnnestClause(UnnestType unnestType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar,
+ Literal.Type outerUnnestMissingValueType) {
super(rightExpr, rightVar, rightPosVar);
this.unnestType = unnestType;
+ setOuterUnnestMissingValueType(outerUnnestMissingValueType);
+ }
+
+ public Literal.Type getOuterUnnestMissingValueType() {
+ return outerUnnestMissingValueType;
+ }
+
+ public void setOuterUnnestMissingValueType(Literal.Type outerUnnestMissingValueType) {
+ this.outerUnnestMissingValueType = validateMissingValueType(unnestType, outerUnnestMissingValueType);
}
@Override
@@ -51,7 +66,8 @@
@Override
public int hashCode() {
- return 31 * super.hashCode() + unnestType.hashCode();
+ return 31 * super.hashCode() + 31 * unnestType.hashCode()
+ + +(outerUnnestMissingValueType != null ? outerUnnestMissingValueType.hashCode() : 0);
}
@Override
@@ -63,6 +79,27 @@
return false;
}
UnnestClause target = (UnnestClause) object;
- return super.equals(target) && unnestType.equals(target.getUnnestType());
+ return super.equals(target) && unnestType.equals(target.getUnnestType())
+ && Objects.equals(outerUnnestMissingValueType, target.outerUnnestMissingValueType);
+ }
+
+ private static Literal.Type validateMissingValueType(UnnestType unnestType, Literal.Type missingValueType) {
+ switch (unnestType) {
+ case INNER:
+ if (missingValueType != null) {
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ return null;
+ case LEFTOUTER:
+ switch (Objects.requireNonNull(missingValueType)) {
+ case MISSING:
+ case NULL:
+ return missingValueType;
+ default:
+ throw new IllegalArgumentException(String.valueOf(missingValueType));
+ }
+ default:
+ throw new IllegalStateException(String.valueOf(unnestType));
+ }
}
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/SelectExpression.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/SelectExpression.java
index 62b8734..2a3ec4e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/SelectExpression.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/SelectExpression.java
@@ -68,10 +68,18 @@
return selectSetOperation;
}
+ public void setOrderbyClause(OrderbyClause orderbyClause) {
+ this.orderbyClause = orderbyClause;
+ }
+
public OrderbyClause getOrderbyClause() {
return orderbyClause;
}
+ public void setLimitClause(LimitClause limitClause) {
+ this.limitClause = limitClause;
+ }
+
public LimitClause getLimitClause() {
return limitClause;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java
index 417cae6..85b9578 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/WindowExpression.java
@@ -40,6 +40,7 @@
private List<Expression> partitionList;
private List<Expression> orderbyList;
private List<OrderbyClause.OrderModifier> orderbyModifierList;
+ private List<OrderbyClause.NullOrderModifier> orderbyNullModifierList;
private FrameMode frameMode;
private FrameBoundaryKind frameStartKind;
@@ -56,7 +57,8 @@
public WindowExpression(FunctionSignature functionSignature, List<Expression> exprList, Expression aggFilterExpr,
List<Expression> partitionList, List<Expression> orderbyList,
- List<OrderbyClause.OrderModifier> orderbyModifierList, FrameMode frameMode,
+ List<OrderbyClause.OrderModifier> orderbyModifierList,
+ List<OrderbyClause.NullOrderModifier> orderbyNullModifierList, FrameMode frameMode,
FrameBoundaryKind frameStartKind, Expression frameStartExpr, FrameBoundaryKind frameEndKind,
Expression frameEndExpr, FrameExclusionKind frameExclusionKind, VariableExpr windowVar,
List<Pair<Expression, Identifier>> windowFieldList, Boolean ignoreNulls, Boolean fromLast) {
@@ -64,6 +66,7 @@
this.partitionList = partitionList;
this.orderbyList = orderbyList;
this.orderbyModifierList = orderbyModifierList;
+ this.orderbyNullModifierList = orderbyNullModifierList;
this.frameMode = frameMode;
this.frameStartKind = frameStartKind;
this.frameStartExpr = frameStartExpr;
@@ -113,6 +116,14 @@
this.orderbyModifierList = orderbyModifierList;
}
+ public List<OrderbyClause.NullOrderModifier> getOrderbyNullModifierList() {
+ return orderbyNullModifierList;
+ }
+
+ public void setOrderbyNullModifierList(List<OrderbyClause.NullOrderModifier> orderbyNullModifierList) {
+ this.orderbyNullModifierList = orderbyNullModifierList;
+ }
+
public boolean hasFrameDefinition() {
return frameMode != null;
}
@@ -292,7 +303,12 @@
if (i > 0) {
sb.append(',');
}
- sb.append(orderbyList.get(i)).append(' ').append(orderbyModifierList.get(i));
+ OrderbyClause.OrderModifier orderModifier = orderbyModifierList.get(i);
+ OrderbyClause.NullOrderModifier nullOrderModifier = orderbyNullModifierList.get(i);
+ sb.append(orderbyList.get(i)).append(' ').append(orderModifier);
+ if (nullOrderModifier != null) {
+ sb.append(" NULLS ").append(nullOrderModifier);
+ }
}
}
if (hasFrameDefinition()) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
index ea199ea..925140b 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
@@ -41,6 +41,7 @@
LIST_VAL_FILE_HINT("list-val-file"),
RANGE_HINT("range"),
SKIP_SECONDARY_INDEX_SEARCH_HINT("skip-index"),
+ SPATIAL_JOIN_HINT("spatial-partitioning"),
USE_SECONDARY_INDEX_SEARCH_HINT("use-index"),
VAL_FILE_HINT("val-files"),
VAL_FILE_SAME_INDEX_HINT("val-file-same-idx"),
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 8dd206f..557f17d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -18,15 +18,49 @@
*/
package org.apache.asterix.lang.sqlpp.rewrites;
+import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.lang.common.base.AbstractClause;
+import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.IReturningStatement;
+import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.clause.WhereClause;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.FieldBinding;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.StringLiteral;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.ViewUtil;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
+import org.apache.asterix.lang.sqlpp.clause.SelectElement;
+import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+/**
+ * This rewriter is used to rewrite body expression of user defined functions and views
+ */
class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
public SqlppFunctionBodyRewriter(IParserFactory parserFactory) {
@@ -35,15 +69,15 @@
@Override
public void rewrite(LangRewritingContext context, IReturningStatement topStatement, boolean allowNonStoredUdfCalls,
- boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException {
- if (inlineUdfs) {
- // When rewriting function body we do not inline UDFs into it.
+ boolean inlineUdfsAndViews, Collection<VarIdentifier> externalVars) throws CompilationException {
+ if (inlineUdfsAndViews) {
+ // When rewriting function or view body we do not inline UDFs or views into it.
// The main query rewriter will inline everything later, when it processes the query
- throw new CompilationException(ErrorCode.ILLEGAL_STATE, topStatement.getSourceLocation(), "inlineUdfs");
+ throw new CompilationException(ErrorCode.ILLEGAL_STATE, topStatement.getSourceLocation(), "");
}
// Sets up parameters.
- setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfs);
+ setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfsAndViews);
// Resolves function calls
resolveFunctionCalls();
@@ -97,4 +131,101 @@
// Rewrites RIGHT OUTER JOINs into LEFT OUTER JOINs if possible
rewriteRightJoins();
}
+
+ static Expression castViewBodyAsType(LangRewritingContext context, Expression bodyExpr, IAType itemType,
+ Triple<String, String, String> temporalDataFormat, DatasetFullyQualifiedName viewName,
+ SourceLocation sourceLoc) throws CompilationException {
+ if (itemType.getTypeTag() != ATypeTag.OBJECT) {
+ throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, viewName,
+ itemType.getTypeName());
+ }
+ ARecordType recordType = (ARecordType) itemType;
+ if (recordType.isOpen()) {
+ throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, viewName,
+ itemType.getTypeName());
+ }
+ String[] fieldNames = recordType.getFieldNames();
+ IAType[] fieldTypes = recordType.getFieldTypes();
+ int n = fieldNames.length;
+ if (n == 0) {
+ throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, viewName,
+ itemType.getTypeName());
+ }
+ List<FieldBinding> projections = new ArrayList<>(n);
+ List<AbstractClause> letWhereClauseList = new ArrayList<>(n + 1);
+ List<Expression> filters = null;
+ VarIdentifier fromVar = context.newVariable();
+ for (int i = 0; i < n; i++) {
+ String fieldName = fieldNames[i];
+ IAType fieldType = fieldTypes[i];
+ IAType primeType;
+ boolean fieldTypeNullable;
+ if (fieldType.getTypeTag() == ATypeTag.UNION) {
+ AUnionType unionType = (AUnionType) fieldType;
+ fieldTypeNullable = unionType.isNullableType();
+ if (!fieldTypeNullable) {
+ throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, sourceLoc, viewName,
+ unionType.toString());
+ }
+ primeType = unionType.getActualType();
+ } else {
+ fieldTypeNullable = false;
+ primeType = fieldType;
+ }
+ Expression expr = ViewUtil.createFieldAccessExpression(fromVar, fieldName, sourceLoc);
+ Expression projectExpr =
+ ViewUtil.createTypeConvertExpression(expr, primeType, temporalDataFormat, viewName, sourceLoc);
+ VarIdentifier projectVar = context.newVariable();
+ VariableExpr projectVarRef1 = new VariableExpr(projectVar);
+ projectVarRef1.setSourceLocation(sourceLoc);
+ LetClause letClause = new LetClause(projectVarRef1, projectExpr);
+ letWhereClauseList.add(letClause);
+ VariableExpr projectVarRef2 = new VariableExpr(projectVar);
+ projectVarRef2.setSourceLocation(sourceLoc);
+ projections.add(new FieldBinding(new LiteralExpr(new StringLiteral(fieldName)), projectVarRef2));
+
+ if (!fieldTypeNullable) {
+ VariableExpr projectVarRef3 = new VariableExpr(projectVar);
+ projectVarRef3.setSourceLocation(sourceLoc);
+ Expression notIsNullExpr = ViewUtil.createNotIsNullExpression(projectVarRef3, sourceLoc);
+ if (filters == null) {
+ filters = new ArrayList<>();
+ }
+ filters.add(notIsNullExpr);
+ }
+ }
+
+ VariableExpr fromVarRef = new VariableExpr(fromVar);
+ fromVarRef.setSourceLocation(sourceLoc);
+ FromClause fromClause =
+ new FromClause(Collections.singletonList(new FromTerm(bodyExpr, fromVarRef, null, null)));
+ fromClause.setSourceLocation(sourceLoc);
+
+ if (filters != null && !filters.isEmpty()) {
+ Expression whereExpr;
+ if (filters.size() == 1) {
+ whereExpr = filters.get(0);
+ } else {
+ CallExpr andExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.AND), filters);
+ andExpr.setSourceLocation(sourceLoc);
+ whereExpr = andExpr;
+ }
+ WhereClause whereClause = new WhereClause(whereExpr);
+ whereClause.setSourceLocation(sourceLoc);
+ letWhereClauseList.add(whereClause);
+ }
+
+ RecordConstructor recordConstr = new RecordConstructor(projections);
+ recordConstr.setSourceLocation(sourceLoc);
+
+ SelectClause selectClause = new SelectClause(new SelectElement(recordConstr), null, false);
+ selectClause.setSourceLocation(sourceLoc);
+ SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, letWhereClauseList, null, null);
+ selectBlock.setSourceLocation(sourceLoc);
+ SelectSetOperation selectSetOperation = new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
+ selectSetOperation.setSourceLocation(sourceLoc);
+ SelectExpression selectExpression = new SelectExpression(null, selectSetOperation, null, null, true);
+ selectExpression.setSourceLocation(sourceLoc);
+ return selectExpression;
+ }
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriterFactory.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriterFactory.java
deleted file mode 100644
index bf27464..0000000
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriterFactory.java
+++ /dev/null
@@ -1,44 +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.
- */
-package org.apache.asterix.lang.sqlpp.rewrites;
-
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.base.IQueryRewriter;
-import org.apache.asterix.lang.common.base.IRewriterFactory;
-import org.apache.asterix.lang.common.base.IStatementRewriter;
-
-class SqlppFunctionBodyRewriterFactory implements IRewriterFactory {
-
- private final IParserFactory parserFactory;
-
- public SqlppFunctionBodyRewriterFactory(IParserFactory parserFactory) {
- this.parserFactory = parserFactory;
- }
-
- @Override
- public IQueryRewriter createQueryRewriter() {
- return new SqlppFunctionBodyRewriter(parserFactory);
- }
-
- @Override
- public IStatementRewriter createStatementRewriter() {
- throw new IllegalStateException("There could not be non-query statements inside a function definition.");
- }
-
-}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index a3ff1c6..2f4fcc8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -24,30 +24,42 @@
import java.util.Deque;
import java.util.HashSet;
import java.util.LinkedHashMap;
+import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractExpression;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.IParserFactory;
import org.apache.asterix.lang.common.base.IQueryRewriter;
import org.apache.asterix.lang.common.base.IReturningStatement;
import org.apache.asterix.lang.common.expression.AbstractCallExpression;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.FieldAccessor;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.MissingLiteral;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.ViewUtil;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.GenerateColumnNameVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineColumnAliasVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineWithExpressionVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.OperatorExpressionVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SetOperationVisitor;
+import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlCompatRewriteVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseAggregateExtractionVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseExpressionVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppFunctionCallResolverVisitor;
@@ -65,10 +77,20 @@
import org.apache.asterix.lang.sqlpp.rewrites.visitor.VariableCheckAndRewriteVisitor;
import org.apache.asterix.lang.sqlpp.util.SqlppAstPrintUtil;
import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Dataverse;
import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.ViewDetails;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.TypeUtil;
+import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.util.LogRedactionUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -79,14 +101,18 @@
public static final String INLINE_WITH_OPTION = "inline_with";
private static final boolean INLINE_WITH_OPTION_DEFAULT = true;
+
+ public static final String SQL_COMPAT_OPTION = "sql_compat";
+ private static final boolean SQL_COMPAT_OPTION_DEFAULT = false;
+
private final IParserFactory parserFactory;
- private SqlppFunctionBodyRewriter functionBodyRewriter;
+ private SqlppFunctionBodyRewriter functionAndViewBodyRewriter;
private IReturningStatement topStatement;
private LangRewritingContext context;
private MetadataProvider metadataProvider;
private Collection<VarIdentifier> externalVars;
private boolean allowNonStoredUdfCalls;
- private boolean inlineUdfs;
+ private boolean inlineUdfsAndViews;
private boolean isLogEnabled;
public SqlppQueryRewriter(IParserFactory parserFactory) {
@@ -94,24 +120,24 @@
}
protected void setup(LangRewritingContext context, IReturningStatement topStatement,
- Collection<VarIdentifier> externalVars, boolean allowNonStoredUdfCalls, boolean inlineUdfs)
+ Collection<VarIdentifier> externalVars, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews)
throws CompilationException {
this.context = context;
this.metadataProvider = context.getMetadataProvider();
this.topStatement = topStatement;
this.externalVars = externalVars != null ? externalVars : Collections.emptyList();
this.allowNonStoredUdfCalls = allowNonStoredUdfCalls;
- this.inlineUdfs = inlineUdfs;
+ this.inlineUdfsAndViews = inlineUdfsAndViews;
this.isLogEnabled = LOGGER.isTraceEnabled();
logExpression("Starting AST rewrites on", "");
}
@Override
public void rewrite(LangRewritingContext context, IReturningStatement topStatement, boolean allowNonStoredUdfCalls,
- boolean inlineUdfs, Collection<VarIdentifier> externalVars) throws CompilationException {
+ boolean inlineUdfsAndViews, Collection<VarIdentifier> externalVars) throws CompilationException {
// Sets up parameters.
- setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfs);
+ setup(context, topStatement, externalVars, allowNonStoredUdfCalls, inlineUdfsAndViews);
// Resolves function calls
resolveFunctionCalls();
@@ -119,6 +145,11 @@
// Generates column names.
generateColumnNames();
+ // SQL-compat mode rewrites
+ // Must run after generateColumnNames() because it might need to generate new column names
+ // for the new projections that it introduces
+ rewriteSqlCompat();
+
// Substitutes group-by key expressions.
substituteGroupbyKeyExpression();
@@ -162,8 +193,8 @@
// Rewrites RIGHT OUTER JOINs into LEFT OUTER JOINs if possible
rewriteRightJoins();
- // Inlines functions.
- loadAndInlineDeclaredUdfs();
+ // Inlines functions and views
+ loadAndInlineUdfsAndViews();
// Rewrites SQL++ core aggregate function names into internal names
rewriteSpecialFunctionNames();
@@ -193,6 +224,15 @@
rewriteTopExpr(listInputFunctionVisitor, null);
}
+ protected void rewriteSqlCompat() throws CompilationException {
+ boolean sqlCompatMode = metadataProvider.getBooleanProperty(SQL_COMPAT_OPTION, SQL_COMPAT_OPTION_DEFAULT);
+ if (!sqlCompatMode) {
+ return;
+ }
+ SqlCompatRewriteVisitor visitor = new SqlCompatRewriteVisitor(context);
+ rewriteTopExpr(visitor, null);
+ }
+
protected void resolveFunctionCalls() throws CompilationException {
SqlppFunctionCallResolverVisitor visitor =
new SqlppFunctionCallResolverVisitor(context, allowNonStoredUdfCalls);
@@ -289,12 +329,21 @@
rewriteTopExpr(visitor, null);
}
- protected void loadAndInlineDeclaredUdfs() throws CompilationException {
- Map<FunctionSignature, FunctionDecl> udfs = fetchUserDefinedSqlppFunctions(topStatement);
- FunctionUtil.checkFunctionRecursion(udfs, SqlppGatherFunctionCallsVisitor::new,
- topStatement.getSourceLocation());
- if (!udfs.isEmpty() && inlineUdfs) {
- SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context, udfs);
+ protected void loadAndInlineUdfsAndViews() throws CompilationException {
+ Pair<Map<FunctionSignature, FunctionDecl>, Map<DatasetFullyQualifiedName, ViewDecl>> udfAndViewDecls =
+ loadUdfsAndViews(topStatement);
+ Map<FunctionSignature, FunctionDecl> udfs = udfAndViewDecls.first;
+ Map<DatasetFullyQualifiedName, ViewDecl> views = udfAndViewDecls.second;
+ if (udfs.isEmpty() && views.isEmpty()) {
+ // nothing to do
+ return;
+ }
+ if (ExpressionUtils.hasFunctionOrViewRecursion(udfs, views, SqlppGatherFunctionCallsVisitor::new)) {
+ throw new CompilationException(ErrorCode.ILLEGAL_FUNCTION_OR_VIEW_RECURSION,
+ topStatement.getSourceLocation());
+ }
+ if (inlineUdfsAndViews) {
+ SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context, udfs, views);
while (rewriteTopExpr(visitor, null)) {
// loop until no more changes
}
@@ -332,14 +381,24 @@
return extVars;
}
- private Map<FunctionSignature, FunctionDecl> fetchUserDefinedSqlppFunctions(IReturningStatement topExpr)
- throws CompilationException {
- Map<FunctionSignature, FunctionDecl> udfs = new LinkedHashMap<>();
+ @Override
+ public VarIdentifier toExternalVariableName(String statementParameterName) {
+ return SqlppVariableUtil.toExternalVariableIdentifier(statementParameterName);
+ }
+ @Override
+ public String toFunctionParameterName(VarIdentifier paramVar) {
+ return SqlppVariableUtil.toUserDefinedName(paramVar.getValue());
+ }
+
+ private Pair<Map<FunctionSignature, FunctionDecl>, Map<DatasetFullyQualifiedName, ViewDecl>> loadUdfsAndViews(
+ IReturningStatement topExpr) throws CompilationException {
+ Map<FunctionSignature, FunctionDecl> udfs = new LinkedHashMap<>();
+ Map<DatasetFullyQualifiedName, ViewDecl> views = new LinkedHashMap<>();
Deque<AbstractCallExpression> workQueue = new ArrayDeque<>();
- SqlppGatherFunctionCallsVisitor gfc = new SqlppGatherFunctionCallsVisitor(workQueue);
+ SqlppGatherFunctionCallsVisitor callVisitor = new SqlppGatherFunctionCallsVisitor(workQueue);
for (Expression expr : topExpr.getDirectlyEnclosedExpressions()) {
- expr.accept(gfc, null);
+ expr.accept(callVisitor, null);
}
AbstractCallExpression fnCall;
while ((fnCall = workQueue.poll()) != null) {
@@ -351,31 +410,28 @@
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fnCall.getSourceLocation(),
fs);
}
- if (FunctionUtil.isBuiltinFunctionSignature(fs) || udfs.containsKey(fs)) {
- continue;
+ if (FunctionUtil.isBuiltinFunctionSignature(fs)) {
+ if (FunctionUtil.isBuiltinDatasetFunction(fs)) {
+ Triple<DatasetFullyQualifiedName, Boolean, DatasetFullyQualifiedName> dsArgs =
+ FunctionUtil.parseDatasetFunctionArguments(fnCall);
+ if (Boolean.TRUE.equals(dsArgs.second)) {
+ DatasetFullyQualifiedName viewName = dsArgs.first;
+ if (!views.containsKey(viewName)) {
+ ViewDecl viewDecl = fetchViewDecl(viewName, fnCall.getSourceLocation());
+ views.put(viewName, viewDecl);
+ viewDecl.getNormalizedViewBody().accept(callVisitor, null);
+ }
+ }
+ }
+ } else {
+ if (!udfs.containsKey(fs)) {
+ FunctionDecl fd = fetchFunctionDecl(fs, fnCall.getSourceLocation());
+ if (fd != null) {
+ udfs.put(fs, fd);
+ fd.getNormalizedFuncBody().accept(callVisitor, null);
+ }
+ }
}
- FunctionDecl fd = context.getDeclaredFunctions().get(fs);
- if (fd == null) {
- Function function;
- try {
- function = metadataProvider.lookupUserDefinedFunction(fs);
- } catch (AlgebricksException e) {
- throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, fnCall.getSourceLocation(),
- fs.toString());
- }
- if (function == null) {
- throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, fnCall.getSourceLocation(),
- fs.toString());
- }
- if (function.isExternal()) {
- continue;
- }
- fd = FunctionUtil.parseStoredFunction(function, parserFactory, context.getWarningCollector(),
- fnCall.getSourceLocation());
- }
- prepareFunction(fd);
- udfs.put(fs, fd);
- fd.getNormalizedFuncBody().accept(gfc, null);
break;
case WINDOW_EXPRESSION:
// there cannot be used-defined window functions
@@ -385,54 +441,170 @@
fnCall.getFunctionSignature().toString(false));
}
}
- return udfs;
+ return new Pair<>(udfs, views);
}
- private void prepareFunction(FunctionDecl fd) throws CompilationException {
- Expression fnNormBody = fd.getNormalizedFuncBody();
- if (fnNormBody == null) {
- fnNormBody = rewriteFunctionBody(fd);
- fd.setNormalizedFuncBody(fnNormBody);
+ private FunctionDecl fetchFunctionDecl(FunctionSignature fs, SourceLocation sourceLoc) throws CompilationException {
+ FunctionDecl fd = context.getDeclaredFunctions().get(fs);
+ if (fd == null) {
+ Function function;
+ try {
+ function = metadataProvider.lookupUserDefinedFunction(fs);
+ } catch (AlgebricksException e) {
+ throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc, fs.toString());
+ }
+ if (function == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_FUNCTION, sourceLoc, fs.toString());
+ }
+ if (function.isExternal()) {
+ return null;
+ }
+ fd = FunctionUtil.parseStoredFunction(function, parserFactory, context.getWarningCollector(), sourceLoc);
}
+ Expression normBody = fd.getNormalizedFuncBody();
+ if (normBody == null) {
+ normBody = rewriteFunctionBody(fd);
+ fd.setNormalizedFuncBody(normBody);
+ }
+ return fd;
+ }
+
+ private ViewDecl fetchViewDecl(DatasetFullyQualifiedName viewName, SourceLocation sourceLoc)
+ throws CompilationException {
+ IAType viewItemType = null;
+ Boolean defaultNull = false;
+ Triple<String, String, String> temporalDataFormat = null;
+ ViewDecl viewDecl = context.getDeclaredViews().get(viewName);
+ if (viewDecl == null) {
+ Dataset dataset;
+ try {
+ dataset = metadataProvider.findDataset(viewName.getDataverseName(), viewName.getDatasetName(), true);
+ } catch (AlgebricksException e) {
+ throw new CompilationException(ErrorCode.UNKNOWN_VIEW, e, sourceLoc, viewName);
+ }
+ if (dataset == null || DatasetUtil.isNotView(dataset)) {
+ throw new CompilationException(ErrorCode.UNKNOWN_VIEW, sourceLoc, viewName);
+ }
+ ViewDetails viewDetails = (ViewDetails) dataset.getDatasetDetails();
+ viewDecl = ViewUtil.parseStoredView(viewName, viewDetails, parserFactory, context.getWarningCollector(),
+ sourceLoc);
+ DataverseName itemTypeDataverseName = dataset.getItemTypeDataverseName();
+ String itemTypeName = dataset.getItemTypeName();
+ boolean isAnyType =
+ MetadataBuiltinEntities.ANY_OBJECT_DATATYPE.getDataverseName().equals(itemTypeDataverseName)
+ && MetadataBuiltinEntities.ANY_OBJECT_DATATYPE.getDatatypeName().equals(itemTypeName);
+ if (!isAnyType) {
+ try {
+ viewItemType = metadataProvider.findType(itemTypeDataverseName, itemTypeName);
+ } catch (AlgebricksException e) {
+ throw new CompilationException(ErrorCode.UNKNOWN_TYPE,
+ TypeUtil.getFullyQualifiedDisplayName(itemTypeDataverseName, itemTypeName));
+ }
+ defaultNull = viewDetails.getDefaultNull();
+ temporalDataFormat = new Triple<>(viewDetails.getDatetimeFormat(), viewDetails.getDateFormat(),
+ viewDetails.getTimeFormat());
+ }
+ }
+ Expression normBody = viewDecl.getNormalizedViewBody();
+ if (normBody == null) {
+ normBody = rewriteViewBody(viewDecl, viewItemType, defaultNull, temporalDataFormat);
+ viewDecl.setNormalizedViewBody(normBody);
+ }
+ return viewDecl;
}
private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
- DataverseName fnDataverseName = fnDecl.getSignature().getDataverseName();
+ FunctionSignature fs = fnDecl.getSignature();
+ return rewriteFunctionOrViewBody(fs.getDataverseName(), fs, fnDecl.getFuncBody(), fnDecl.getParamList(),
+ !fnDecl.isStored(), fnDecl.getSourceLocation());
+ }
+
+ private Expression rewriteViewBody(ViewDecl viewDecl, IAType viewItemType, Boolean defaultNull,
+ Triple<String, String, String> temporalDataFormat) throws CompilationException {
+ DatasetFullyQualifiedName viewName = viewDecl.getViewName();
+ SourceLocation sourceLoc = viewDecl.getSourceLocation();
+ Expression rewrittenBodyExpr = rewriteFunctionOrViewBody(viewName.getDataverseName(), viewName,
+ viewDecl.getViewBody(), Collections.emptyList(), false, sourceLoc);
+ if (viewItemType != null) {
+ if (!Boolean.TRUE.equals(defaultNull)) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ "Default Null is required");
+ }
+ rewrittenBodyExpr = SqlppFunctionBodyRewriter.castViewBodyAsType(context, rewrittenBodyExpr, viewItemType,
+ temporalDataFormat, viewName, sourceLoc);
+ }
+ return rewrittenBodyExpr;
+ }
+
+ private Expression rewriteFunctionOrViewBody(DataverseName entityDataverseName, Object entityDisplayName,
+ Expression bodyExpr, List<VarIdentifier> externalVars, boolean allowNonStoredUdfCalls,
+ SourceLocation sourceLoc) throws CompilationException {
Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
- Dataverse fnDataverse;
- if (fnDataverseName == null || fnDataverseName.equals(defaultDataverse.getDataverseName())) {
- fnDataverse = defaultDataverse;
+ Dataverse targetDataverse;
+ if (entityDataverseName == null || entityDataverseName.equals(defaultDataverse.getDataverseName())) {
+ targetDataverse = defaultDataverse;
} else {
try {
- fnDataverse = metadataProvider.findDataverse(fnDataverseName);
+ targetDataverse = metadataProvider.findDataverse(entityDataverseName);
} catch (AlgebricksException e) {
- throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, fnDecl.getSourceLocation(),
- fnDataverseName);
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, entityDataverseName);
}
}
- metadataProvider.setDefaultDataverse(fnDataverse);
+ metadataProvider.setDefaultDataverse(targetDataverse);
try {
- Query wrappedQuery = new Query(false);
- wrappedQuery.setSourceLocation(fnDecl.getSourceLocation());
- wrappedQuery.setBody(fnDecl.getFuncBody());
- wrappedQuery.setTopLevel(false);
- boolean allowNonStoredUdfCalls = !fnDecl.isStored();
- getFunctionBodyRewriter().rewrite(context, wrappedQuery, allowNonStoredUdfCalls, false,
- fnDecl.getParamList());
+ Query wrappedQuery = ExpressionUtils.createWrappedQuery(bodyExpr, sourceLoc);
+ getFunctionAndViewBodyRewriter().rewrite(context, wrappedQuery, allowNonStoredUdfCalls, false,
+ externalVars);
return wrappedQuery.getBody();
} catch (CompilationException e) {
- throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e, fnDecl.getSignature(),
- e.getMessage());
+ throw new CompilationException(ErrorCode.COMPILATION_BAD_FUNCTION_DEFINITION, e,
+ entityDisplayName.toString(), e.getMessage());
} finally {
metadataProvider.setDefaultDataverse(defaultDataverse);
}
}
- protected SqlppFunctionBodyRewriter getFunctionBodyRewriter() {
- if (functionBodyRewriter == null) {
- functionBodyRewriter = new SqlppFunctionBodyRewriter(parserFactory);
+ protected SqlppFunctionBodyRewriter getFunctionAndViewBodyRewriter() {
+ if (functionAndViewBodyRewriter == null) {
+ functionAndViewBodyRewriter = new SqlppFunctionBodyRewriter(parserFactory);
}
- return functionBodyRewriter;
+ return functionAndViewBodyRewriter;
+ }
+
+ @Override
+ public Query createFunctionAccessorQuery(FunctionDecl functionDecl) {
+ // dataverse_name.function_name(MISSING, ... MISSING)
+ FunctionSignature functionSignature = functionDecl.getSignature();
+ int arity = functionSignature.getArity();
+ List<Expression> args = arity == FunctionIdentifier.VARARGS ? Collections.emptyList()
+ : Collections.nCopies(arity, new LiteralExpr(MissingLiteral.INSTANCE));
+ CallExpr fcall = new CallExpr(functionSignature, args);
+ fcall.setSourceLocation(functionDecl.getSourceLocation());
+ return ExpressionUtils.createWrappedQuery(fcall, functionDecl.getSourceLocation());
+ }
+
+ @Override
+ public Query createViewAccessorQuery(ViewDecl viewDecl) {
+ // dataverse_name.view_name
+ DataverseName dataverseName = viewDecl.getViewName().getDataverseName();
+ String viewName = viewDecl.getViewName().getDatasetName();
+ Expression vAccessExpr = createDatasetAccessExpression(dataverseName, viewName, viewDecl.getSourceLocation());
+ return ExpressionUtils.createWrappedQuery(vAccessExpr, viewDecl.getSourceLocation());
+ }
+
+ private static Expression createDatasetAccessExpression(DataverseName dataverseName, String datasetName,
+ SourceLocation sourceLoc) {
+ AbstractExpression resultExpr = null;
+ List<String> dataverseNameParts = dataverseName.getParts();
+ for (int i = 0, n = dataverseNameParts.size(); i < n; i++) {
+ String part = dataverseNameParts.get(i);
+ resultExpr = i == 0 ? new VariableExpr(new VarIdentifier(SqlppVariableUtil.toInternalVariableName(part)))
+ : new FieldAccessor(resultExpr, new Identifier(part));
+ resultExpr.setSourceLocation(sourceLoc);
+ }
+ resultExpr = new FieldAccessor(resultExpr, new Identifier(datasetName));
+ resultExpr.setSourceLocation(sourceLoc);
+ return resultExpr;
}
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
index a9caacb..f9b0e7a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
@@ -21,8 +21,6 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.IStatementRewriter;
import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.struct.VarIdentifier;
-import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
import org.apache.asterix.lang.sqlpp.visitor.SqlppDeleteRewriteVisitor;
import org.apache.asterix.lang.sqlpp.visitor.SqlppSynonymRewriteVisitor;
import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -49,14 +47,4 @@
stmt.accept(SqlppDeleteRewriteVisitor.INSTANCE, metadataProvider);
}
}
-
- @Override
- public String toExternalVariableName(String statementParameterName) {
- return SqlppVariableUtil.toExternalVariableName(statementParameterName);
- }
-
- @Override
- public String toFunctionParameterName(VarIdentifier paramVar) {
- return SqlppVariableUtil.toUserDefinedName(paramVar.getValue());
- }
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
index 3067641..e488e9d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/GenerateColumnNameVisitor.java
@@ -67,7 +67,7 @@
@Override
public Expression visit(Projection projection, ILangExpression arg) throws CompilationException {
- if (!projection.star() && !projection.varStar() && projection.getName() == null) {
+ if (projection.getKind() == Projection.Kind.NAMED_EXPR && !projection.hasName()) {
projection.setName(SqlppVariableUtil.variableNameToDisplayedFieldName(context.newVariable().getValue()));
}
return super.visit(projection, arg);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
index 416f49b..53cd6de 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/InlineColumnAliasVisitor.java
@@ -151,7 +151,7 @@
private Map<Expression, ColumnAliasBinding> mapProjections(List<Projection> projections) {
Map<Expression, ColumnAliasBinding> exprMap = new HashMap<>();
for (Projection projection : projections) {
- if (!projection.star() && !projection.varStar()) {
+ if (projection.getKind() == Projection.Kind.NAMED_EXPR) {
String varName = SqlppVariableUtil.toInternalVariableName(projection.getName());
exprMap.put(new VariableExpr(new VarIdentifier(varName)), ColumnAliasBinding.of(projection));
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index c47634e..93a14ca 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -29,13 +29,17 @@
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
import org.apache.asterix.lang.common.expression.OperatorExpr;
import org.apache.asterix.lang.common.expression.QuantifiedExpression;
import org.apache.asterix.lang.common.expression.QuantifiedExpression.Quantifier;
import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.FalseLiteral;
+import org.apache.asterix.lang.common.literal.TrueLiteral;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.struct.OperatorType;
import org.apache.asterix.lang.common.struct.QuantifiedPair;
+import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScopingVisitor;
@@ -71,8 +75,9 @@
case BETWEEN:
case NOT_BETWEEN:
return processBetweenOperator(operatorExpr, opType);
- default:
- break;
+ case DISTINCT:
+ case NOT_DISTINCT:
+ return processDistinctOperator(operatorExpr, opType);
}
return operatorExpr;
}
@@ -144,18 +149,83 @@
Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
Expression rightComparison = createOperatorExpression(OperatorType.LE, targetCopy, right,
operatorExpr.getHints(), operatorExpr.getSourceLocation());
- OperatorExpr andExpr = new OperatorExpr();
- andExpr.addOperand(leftComparison);
- andExpr.addOperand(rightComparison);
- andExpr.addOperator(OperatorType.AND);
- andExpr.setSourceLocation(operatorExpr.getSourceLocation());
- if (opType == OperatorType.BETWEEN) {
- return andExpr;
- } else {
- CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
- new ArrayList<>(Collections.singletonList(andExpr)));
- callExpr.setSourceLocation(operatorExpr.getSourceLocation());
- return callExpr;
+ Expression andExpr = createOperatorExpression(OperatorType.AND, leftComparison, rightComparison, null,
+ operatorExpr.getSourceLocation());
+ switch (opType) {
+ case BETWEEN:
+ return andExpr;
+ case NOT_BETWEEN:
+ CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
+ new ArrayList<>(Collections.singletonList(andExpr)));
+ callExpr.setSourceLocation(operatorExpr.getSourceLocation());
+ return callExpr;
+ default:
+ throw new IllegalArgumentException(String.valueOf(opType));
+ }
+ }
+
+ private Expression processDistinctOperator(OperatorExpr operatorExpr, OperatorType opType)
+ throws CompilationException {
+
+ // lhs IS NOT DISTINCT FROM rhs =>
+ // CASE
+ // WHEN (lhs = rhs) OR (lhs IS NULL AND rhs IS NULL) OR (lhs IS MISSING AND rhs IS MISSING)
+ // THEN TRUE
+ // ELSE FALSE
+ // END
+ //
+ // lhs IS DISTINCT FROM rhs => NOT ( lhs IS NOT DISTINCT FROM rhs )
+
+ Expression lhs = operatorExpr.getExprList().get(0);
+ Expression rhs = operatorExpr.getExprList().get(1);
+
+ Expression lhsEqRhs = createOperatorExpression(OperatorType.EQ, lhs, rhs, operatorExpr.getHints(),
+ operatorExpr.getSourceLocation());
+
+ CallExpr lhsIsNull = new CallExpr(new FunctionSignature(BuiltinFunctions.IS_NULL),
+ new ArrayList<>(Collections.singletonList((Expression) SqlppRewriteUtil.deepCopy(lhs))));
+ lhsIsNull.setSourceLocation(operatorExpr.getSourceLocation());
+
+ CallExpr rhsIsNull = new CallExpr(new FunctionSignature(BuiltinFunctions.IS_NULL),
+ new ArrayList<>(Collections.singletonList((Expression) SqlppRewriteUtil.deepCopy(rhs))));
+ rhsIsNull.setSourceLocation(operatorExpr.getSourceLocation());
+
+ CallExpr lhsIsMissing = new CallExpr(new FunctionSignature(BuiltinFunctions.IS_MISSING),
+ new ArrayList<>(Collections.singletonList((Expression) SqlppRewriteUtil.deepCopy(lhs))));
+ lhsIsMissing.setSourceLocation(operatorExpr.getSourceLocation());
+
+ CallExpr rhsIsMissing = new CallExpr(new FunctionSignature(BuiltinFunctions.IS_MISSING),
+ new ArrayList<>(Collections.singletonList((Expression) SqlppRewriteUtil.deepCopy(rhs))));
+ rhsIsMissing.setSourceLocation(operatorExpr.getSourceLocation());
+
+ Expression bothAreNull = createOperatorExpression(OperatorType.AND, lhsIsNull, rhsIsNull, null,
+ operatorExpr.getSourceLocation());
+
+ Expression bothAreMissing = createOperatorExpression(OperatorType.AND, lhsIsMissing, rhsIsMissing, null,
+ operatorExpr.getSourceLocation());
+
+ Expression bothAreNullOrMissing = createOperatorExpression(OperatorType.OR, bothAreNull, bothAreMissing, null,
+ operatorExpr.getSourceLocation());
+
+ Expression eqOrNullOrMissing = createOperatorExpression(OperatorType.OR, lhsEqRhs, bothAreNullOrMissing, null,
+ operatorExpr.getSourceLocation());
+
+ CaseExpression caseExpr = new CaseExpression(new LiteralExpr(TrueLiteral.INSTANCE),
+ new ArrayList<>(Collections.singletonList(eqOrNullOrMissing)),
+ new ArrayList<>(Collections.singletonList(new LiteralExpr(TrueLiteral.INSTANCE))),
+ new LiteralExpr(FalseLiteral.INSTANCE));
+ caseExpr.setSourceLocation(operatorExpr.getSourceLocation());
+
+ switch (opType) {
+ case NOT_DISTINCT:
+ return caseExpr;
+ case DISTINCT:
+ CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.NOT),
+ new ArrayList<>(Collections.singletonList(caseExpr)));
+ callExpr.setSourceLocation(operatorExpr.getSourceLocation());
+ return callExpr;
+ default:
+ throw new IllegalArgumentException(String.valueOf(opType));
}
}
@@ -173,5 +243,4 @@
}
return comparison;
}
-
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
index da8b5f2..1266d9a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SetOperationVisitor.java
@@ -96,6 +96,7 @@
SelectExpression newSelectExpression = new SelectExpression(selectExpression.getLetList(),
newSelectSetOperation, orderBy, limit, selectExpression.isSubquery());
newSelectExpression.setSourceLocation(sourceLoc);
+ newSelectExpression.addHints(selectExpression.getHints());
return super.visit(newSelectExpression, arg);
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
new file mode 100644
index 0000000..9b91bc0
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
@@ -0,0 +1,772 @@
+/*
+ * 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.lang.sqlpp.rewrites.visitor;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.lang.common.base.AbstractClause;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.Literal;
+import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.clause.LimitClause;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.FieldAccessor;
+import org.apache.asterix.lang.common.expression.FieldBinding;
+import org.apache.asterix.lang.common.expression.IndexAccessor;
+import org.apache.asterix.lang.common.expression.ListConstructor;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.OperatorExpr;
+import org.apache.asterix.lang.common.expression.QuantifiedExpression;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.expression.UnaryExpr;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.IntegerLiteral;
+import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
+import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.OperatorType;
+import org.apache.asterix.lang.common.struct.QuantifiedPair;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.clause.JoinClause;
+import org.apache.asterix.lang.sqlpp.clause.Projection;
+import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
+import org.apache.asterix.lang.sqlpp.clause.SelectElement;
+import org.apache.asterix.lang.sqlpp.clause.SelectRegular;
+import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
+import org.apache.asterix.lang.sqlpp.clause.UnnestClause;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.optype.SetOpType;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
+import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
+import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
+import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
+import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * Applies initial rewritings for "SQL-compatible" evaluation mode
+ * <ol>
+ * <li>Rewrites {@code SELECT *} into {@code SELECT *.*}
+ * <li>Rewrites {@code NOT? IN expr} into {@code NOT? IN to_array(expr)} if {@code expr} can return a non-list
+ * </ol>
+ * <p/>
+ * Also applies subquery coercion rewritings as follows:
+ * <ol>
+ * <li> FROM/JOIN/UNNEST (subquery) --> no subquery coercion
+ * <li> WITH/LET v = (subquery) --> no subquery coercion
+ * <li> SOME/EVERY v IN (subquery) --> no subquery coercion
+ * <li> [NOT] EXISTS (subquery) --> no subquery coercion
+ * <li> WHERE (x,y) = (subquery) --> coerce the subquery into a single array
+ * <li> WHERE x IN (subquery) --> coerce the subquery into a collection of values
+ * <li> WHERE (x,y) IN (subquery) --> coerce the subquery into a collection of arrays
+ * <li> otherwise --> coerce the subquery into a single value
+ * </ol>
+ */
+public final class SqlCompatRewriteVisitor extends AbstractSqlppSimpleExpressionVisitor {
+
+ private final LangRewritingContext context;
+
+ private final SelectExpressionAnalyzer selectExprAnalyzer = new SelectExpressionAnalyzer();
+
+ public SqlCompatRewriteVisitor(LangRewritingContext context) {
+ this.context = context;
+ }
+
+ @Override
+ public Expression visit(Projection projection, ILangExpression arg) throws CompilationException {
+ if (projection.getKind() == Projection.Kind.STAR) {
+ projection.setKind(Projection.Kind.EVERY_VAR_STAR);
+ }
+ return super.visit(projection, arg);
+ }
+
+ @Override
+ public Expression visit(FromTerm fromTerm, ILangExpression arg) throws CompilationException {
+ Expression expr = fromTerm.getLeftExpression();
+ if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateSubqueryNoCoercion((SelectExpression) expr);
+ }
+ return super.visit(fromTerm, arg);
+ }
+
+ @Override
+ public Expression visit(JoinClause joinClause, ILangExpression arg) throws CompilationException {
+ Expression expr = joinClause.getRightExpression();
+ if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateSubqueryNoCoercion((SelectExpression) expr);
+ }
+ switch (joinClause.getJoinType()) {
+ case LEFTOUTER:
+ case RIGHTOUTER:
+ joinClause.setOuterJoinMissingValueType(Literal.Type.NULL);
+ break;
+ case INNER:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, joinClause.getSourceLocation(),
+ String.valueOf(joinClause.getJoinType()));
+ }
+ return super.visit(joinClause, arg);
+ }
+
+ @Override
+ public Expression visit(UnnestClause unnestClause, ILangExpression arg) throws CompilationException {
+ Expression expr = unnestClause.getRightExpression();
+ if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateSubqueryNoCoercion((SelectExpression) expr);
+ }
+ // keep UNNEST clause aligned with JOIN clause when it comes to producing NULL values
+ switch (unnestClause.getUnnestType()) {
+ case LEFTOUTER:
+ unnestClause.setOuterUnnestMissingValueType(Literal.Type.NULL);
+ break;
+ case INNER:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, unnestClause.getSourceLocation(),
+ String.valueOf(unnestClause.getUnnestType()));
+ }
+ return super.visit(unnestClause, arg);
+ }
+
+ @Override
+ public Expression visit(LetClause letClause, ILangExpression arg) throws CompilationException {
+ Expression expr = letClause.getBindingExpr();
+ if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateSubqueryNoCoercion((SelectExpression) expr);
+ }
+ return super.visit(letClause, arg);
+ }
+
+ @Override
+ public Expression visit(QuantifiedExpression qe, ILangExpression arg) throws CompilationException {
+ for (QuantifiedPair pair : qe.getQuantifiedList()) {
+ Expression expr = pair.getExpr();
+ if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateSubqueryNoCoercion((SelectExpression) expr);
+ }
+ }
+ return super.visit(qe, arg);
+ }
+
+ @Override
+ public Expression visit(OperatorExpr opExpr, ILangExpression arg) throws CompilationException {
+ List<OperatorType> opTypeList = opExpr.getOpList();
+ if (opTypeList.size() == 1) {
+ OperatorType opType = opTypeList.get(0);
+ if (OperatorExpr.opIsComparison(opType)) {
+ List<Expression> argList = opExpr.getExprList();
+ Expression lhs = argList.get(0);
+ Expression rhs = argList.get(1);
+ if (lhs.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateComparisonOpSubquery((SelectExpression) lhs, rhs);
+ }
+ if (rhs.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateComparisonOpSubquery((SelectExpression) rhs, lhs);
+ }
+ } else if (opType == OperatorType.IN || opType == OperatorType.NOT_IN) {
+ List<Expression> argList = opExpr.getExprList();
+ Expression lhs = argList.get(0);
+ Expression rhs = argList.get(1);
+ switch (rhs.getKind()) {
+ case SELECT_EXPRESSION:
+ annotateInOpSubquery((SelectExpression) rhs, lhs);
+ break;
+ case LIST_CONSTRUCTOR_EXPRESSION:
+ case LIST_SLICE_EXPRESSION:
+ // NOT? IN [] -> keep as is
+ break;
+ default:
+ // NOT? IN expr -> NOT? IN to_array(expr)
+ List<Expression> newExprList = new ArrayList<>(2);
+ newExprList.add(lhs);
+ newExprList.add(createCallExpr(BuiltinFunctions.TO_ARRAY, rhs, opExpr.getSourceLocation()));
+ opExpr.setExprList(newExprList);
+ break;
+ }
+ }
+ }
+ return super.visit(opExpr, arg);
+ }
+
+ @Override
+ public Expression visit(UnaryExpr u, ILangExpression arg) throws CompilationException {
+ switch (u.getExprType()) {
+ case EXISTS:
+ case NOT_EXISTS:
+ Expression expr = u.getExpr();
+ if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
+ annotateSubqueryNoCoercion((SelectExpression) expr);
+ }
+ break;
+ }
+ return super.visit(u, arg);
+ }
+
+ @Override
+ public Expression visit(SelectExpression selectExpr, ILangExpression arg) throws CompilationException {
+ SqlCompatSelectExpressionCoercionAnnotation selectExprAnn = null;
+ if (selectExpr.isSubquery()) {
+ selectExprAnn = selectExpr.findHint(SqlCompatSelectExpressionCoercionAnnotation.class);
+ if (selectExprAnn == null) {
+ // all other cases --> coerce the subquery into a scalar value
+ selectExprAnn = annotateSubquery(selectExpr, SqlCompatSelectCoercionKind.SCALAR,
+ SqlCompatSelectCoercionKind.SCALAR);
+ }
+ }
+ Expression newExpr = super.visit(selectExpr, arg);
+ if (selectExprAnn != null) {
+ newExpr = rewriteSelectExpression(newExpr, selectExprAnn);
+ }
+ return newExpr;
+ }
+
+ @Override
+ public Expression visit(SelectBlock selectBlock, ILangExpression arg) throws CompilationException {
+ super.visit(selectBlock, arg);
+ SelectExpression selectExpr = (SelectExpression) arg;
+ SqlCompatSelectExpressionCoercionAnnotation selectExprAnn =
+ selectExpr.findHint(SqlCompatSelectExpressionCoercionAnnotation.class);
+ if (selectExprAnn != null) {
+ rewriteSelectBlock(selectBlock, selectExprAnn);
+ }
+ return null;
+ }
+
+ private void annotateSubqueryNoCoercion(SelectExpression subqueryExpr) {
+ // FROM/JOIN/UNNEST/LET (subquery) -> do NOT coerce the subquery
+ subqueryExpr.addHint(SqlCompatSelectExpressionCoercionAnnotation.NONE_NONE);
+ }
+
+ private void annotateComparisonOpSubquery(SelectExpression subqueryExpr, Expression otherArg)
+ throws CompilationException {
+ // (x,y) = (subquery) -> coerce the subquery into a single array
+ // x = (subquery) -> coerce the subquery into a scalar value
+ annotateSubquery(subqueryExpr, SqlCompatSelectCoercionKind.SCALAR,
+ getSelectBlockAnnotationForOpSubquery(otherArg));
+ }
+
+ private void annotateInOpSubquery(SelectExpression subqueryExpr, Expression otherArg) throws CompilationException {
+ // (x,y) in (subquery) -> coerce the subquery into a collection of arrays
+ // x in (subquery) -> coerce the subquery into a collection of scalar values
+ annotateSubquery(subqueryExpr, SqlCompatSelectCoercionKind.NONE,
+ getSelectBlockAnnotationForOpSubquery(otherArg));
+ }
+
+ private static SqlCompatSelectCoercionKind getSelectBlockAnnotationForOpSubquery(Expression otherArg)
+ throws CompilationException {
+ if (otherArg.getKind() == Expression.Kind.LIST_CONSTRUCTOR_EXPRESSION) {
+ ListConstructor lc = (ListConstructor) otherArg;
+ switch (lc.getType()) {
+ case ORDERED_LIST_CONSTRUCTOR:
+ return SqlCompatSelectCoercionKind.ARRAY;
+ case UNORDERED_LIST_CONSTRUCTOR:
+ return SqlCompatSelectCoercionKind.MULTISET;
+ default:
+ throw new CompilationException(ErrorCode.ILLEGAL_STATE, otherArg.getSourceLocation(), "");
+ }
+ } else {
+ return SqlCompatSelectCoercionKind.SCALAR;
+ }
+ }
+
+ private SqlCompatSelectExpressionCoercionAnnotation annotateSubquery(SelectExpression subqueryExpr,
+ SqlCompatSelectCoercionKind cardinalityCoercion, SqlCompatSelectCoercionKind typeCoercion)
+ throws CompilationException {
+ selectExprAnalyzer.analyze(subqueryExpr.getSelectSetOperation(), true);
+ if (selectExprAnalyzer.subqueryExists) {
+ throw new CompilationException(ErrorCode.COMPILATION_SUBQUERY_COERCION_ERROR,
+ subqueryExpr.getSourceLocation(), "");
+ }
+ if (selectExprAnalyzer.selectRegularExists) {
+ if (selectExprAnalyzer.selectElementExists) {
+ throw new CompilationException(ErrorCode.COMPILATION_SUBQUERY_COERCION_ERROR,
+ subqueryExpr.getSourceLocation(), "Both SELECT and SELECT VALUE are present");
+ }
+ String typeCoercionFieldName = typeCoercion == SqlCompatSelectCoercionKind.NONE ? null
+ : selectExprAnalyzer.generateFieldName(context);
+ SqlCompatSelectExpressionCoercionAnnotation ann = new SqlCompatSelectExpressionCoercionAnnotation(
+ cardinalityCoercion, typeCoercion, typeCoercionFieldName);
+ subqueryExpr.addHint(ann);
+ return ann;
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public Expression visit(SelectSetOperation setOp, ILangExpression arg) throws CompilationException {
+ // let subquery coercion rewriting run first
+ super.visit(setOp, arg);
+
+ if (setOp.hasRightInputs()) {
+ // SetOp (UNION ALL) rewriting
+ selectExprAnalyzer.analyze(setOp, false);
+ if (selectExprAnalyzer.subqueryExists) {
+ throw new CompilationException(ErrorCode.COMPILATION_SET_OPERATION_ERROR, setOp.getSourceLocation(),
+ setOp.getRightInputs().get(0).getSetOpType().toString(), "");
+ }
+ if (selectExprAnalyzer.selectRegularExists) {
+ if (selectExprAnalyzer.selectElementExists) {
+ throw new CompilationException(ErrorCode.COMPILATION_SET_OPERATION_ERROR, setOp.getSourceLocation(),
+ setOp.getRightInputs().get(0).getSetOpType().toString(),
+ "Both SELECT and SELECT VALUE are present");
+ }
+ rewriteSelectSetOp(setOp);
+ }
+ }
+
+ return null;
+ }
+
+ private void rewriteSelectSetOp(SelectSetOperation setOp) throws CompilationException {
+ /*
+ * SELECT a, b, c FROM ...
+ * UNION ALL
+ * SELECT d, e, f FROM ....
+ * -->
+ * SELECT a, b, c
+ * UNION ALL
+ * SELECT v.d AS a, v.e AS b, v.f AS c FROM ( SELECT d, e, f FROM ... )
+ */
+ SelectBlock leftSelectBlock = setOp.getLeftInput().getSelectBlock();
+ boolean ok = leftSelectBlock != null && setOp.hasRightInputs();
+ if (!ok) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, setOp.getSourceLocation(), "");
+ }
+ // collect all field names from the left input
+ List<String> leftFieldNames = collectFieldNames(leftSelectBlock, setOp.getRightInputs().get(0).getSetOpType());
+ for (SetOperationRight rightInput : setOp.getRightInputs()) {
+ // rewrite right inputs
+ rewriteSelectSetOpRightInput(rightInput, leftFieldNames, setOp.getSourceLocation());
+ }
+ }
+
+ private void rewriteSelectSetOpRightInput(SetOperationRight setOpRight, List<String> outputFieldNames,
+ SourceLocation sourceLoc) throws CompilationException {
+ SetOperationInput setOpRightInput = setOpRight.getSetOperationRightInput();
+ SelectBlock rightSelectBlock = setOpRightInput.getSelectBlock();
+ if (rightSelectBlock == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+ }
+ List<String> inputFieldNames = collectFieldNames(rightSelectBlock, setOpRight.getSetOpType());
+ int nFields = inputFieldNames.size();
+ if (nFields != outputFieldNames.size()) {
+ throw new CompilationException(ErrorCode.COMPILATION_SET_OPERATION_ERROR, sourceLoc,
+ setOpRight.getSetOpType().toString(), "Unequal number of input fields");
+ }
+
+ SelectSetOperation setOp1 = new SelectSetOperation(new SetOperationInput(rightSelectBlock, null), null);
+ setOp1.setSourceLocation(sourceLoc);
+ SelectExpression selectExpr1 = new SelectExpression(null, setOp1, null, null, true);
+ selectExpr1.setSourceLocation(sourceLoc);
+
+ VarIdentifier v1 = context.newVariable();
+ VariableExpr v1Expr1 = new VariableExpr(v1);
+ v1Expr1.setSourceLocation(sourceLoc);
+
+ FromTerm fromTerm1 = new FromTerm(selectExpr1, v1Expr1, null, null);
+ fromTerm1.setSourceLocation(sourceLoc);
+ List<FromTerm> fromTermList1 = new ArrayList<>(1);
+ fromTermList1.add(fromTerm1);
+ FromClause fromClause1 = new FromClause(fromTermList1);
+ fromClause1.setSourceLocation(sourceLoc);
+
+ List<FieldBinding> fb1List = new ArrayList<>(nFields);
+ for (int i = 0; i < nFields; i++) {
+ VariableExpr v1Expr2 = new VariableExpr(v1);
+ v1Expr2.setSourceLocation(sourceLoc);
+ FieldAccessor fa1 = new FieldAccessor(v1Expr2, new Identifier(inputFieldNames.get(i)));
+ fa1.setSourceLocation(sourceLoc);
+ LiteralExpr lit1 = new LiteralExpr(new StringLiteral(outputFieldNames.get(i)));
+ lit1.setSourceLocation(sourceLoc);
+ fb1List.add(new FieldBinding(lit1, fa1));
+ }
+ RecordConstructor rc1 = new RecordConstructor(fb1List);
+ rc1.setSourceLocation(sourceLoc);
+ SelectClause selectClause1 = new SelectClause(new SelectElement(rc1), null, false);
+ selectClause1.setSourceLocation(sourceLoc);
+ SelectBlock newRightSelectBlock = new SelectBlock(selectClause1, fromClause1, null, null, null);
+ newRightSelectBlock.setSourceLocation(sourceLoc);
+ setOpRightInput.setSelectBlock(newRightSelectBlock);
+ }
+
+ private List<String> collectFieldNames(SelectBlock selectBlock, SetOpType setOpType) throws CompilationException {
+ SelectRegular selectRegular = selectBlock.getSelectClause().getSelectRegular();
+ if (selectRegular == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, selectBlock.getSourceLocation(), "");
+ }
+ List<Projection> projectionList = selectRegular.getProjections();
+ List<String> fieldNames = new ArrayList<>(projectionList.size());
+ for (Projection projection : projectionList) {
+ if (projection.getKind() != Projection.Kind.NAMED_EXPR) {
+ throw new CompilationException(ErrorCode.COMPILATION_SET_OPERATION_ERROR,
+ projection.getSourceLocation(), setOpType.toString(), "Unsupported projection kind");
+ }
+ fieldNames.add(projection.getName());
+ }
+ return fieldNames;
+ }
+
+ private static CallExpr createCallExpr(FunctionIdentifier fid, Expression inExpr, SourceLocation sourceLoc) {
+ List<Expression> argList = new ArrayList<>(1);
+ argList.add(inExpr);
+ CallExpr callExpr = new CallExpr(new FunctionSignature(fid), argList);
+ callExpr.setSourceLocation(sourceLoc);
+ return callExpr;
+ }
+
+ private static final class SelectExpressionAnalyzer {
+
+ private boolean subqueryExists;
+ private boolean selectRegularExists;
+ private boolean selectElementExists;
+ private boolean computeSelectRegularAllFields;
+ private final Set<String> selectRegularAllFields = new HashSet<>();
+
+ private void reset(boolean computeSelectRegularAllFields) {
+ subqueryExists = false;
+ selectRegularExists = false;
+ selectElementExists = false;
+ selectRegularAllFields.clear();
+ this.computeSelectRegularAllFields = computeSelectRegularAllFields;
+ }
+
+ private void analyze(SelectSetOperation setOp, boolean computeSelectRegularAllFields)
+ throws CompilationException {
+ reset(computeSelectRegularAllFields);
+ analyzeSelectSetOpInput(setOp.getLeftInput());
+ if (setOp.hasRightInputs()) {
+ for (SetOperationRight rhs : setOp.getRightInputs()) {
+ analyzeSelectSetOpInput(rhs.getSetOperationRightInput());
+ }
+ }
+ }
+
+ private void analyzeSelectSetOpInput(SetOperationInput setOpInput) throws CompilationException {
+ if (setOpInput.selectBlock()) {
+ SelectBlock selectBlock = setOpInput.getSelectBlock();
+ SelectClause selectClause = selectBlock.getSelectClause();
+ if (selectClause.selectRegular()) {
+ selectRegularExists = true;
+ if (computeSelectRegularAllFields) {
+ for (Projection projection : selectClause.getSelectRegular().getProjections()) {
+ if (projection.getKind() == Projection.Kind.NAMED_EXPR) {
+ selectRegularAllFields.add(projection.getName());
+ }
+ }
+ }
+ } else if (selectClause.selectElement()) {
+ selectElementExists = true;
+ }
+ } else if (setOpInput.subquery()) {
+ subqueryExists = true;
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, "");
+ }
+ }
+
+ private String generateFieldName(LangRewritingContext ctx) {
+ String fieldName;
+ do {
+ fieldName = SqlppVariableUtil.variableNameToDisplayedFieldName(ctx.newVariable().getValue());
+ } while (selectRegularAllFields.contains(fieldName));
+ return fieldName;
+ }
+ }
+
+ private static class SqlCompatSelectExpressionCoercionAnnotation implements IExpressionAnnotation {
+
+ static final SqlCompatSelectExpressionCoercionAnnotation NONE_NONE =
+ new SqlCompatSelectExpressionCoercionAnnotation(SqlCompatSelectCoercionKind.NONE,
+ SqlCompatSelectCoercionKind.NONE, null);
+
+ final SqlCompatSelectCoercionKind cardinalityCoercion;
+
+ final SqlCompatSelectCoercionKind typeCoercion;
+
+ final String typeCoercionFieldName;
+
+ SqlCompatSelectExpressionCoercionAnnotation(SqlCompatSelectCoercionKind cardinalityCoercion,
+ SqlCompatSelectCoercionKind typeCoercion, String typeCoercionFieldName) {
+ this.cardinalityCoercion = Objects.requireNonNull(cardinalityCoercion);
+ this.typeCoercion = Objects.requireNonNull(typeCoercion);
+ this.typeCoercionFieldName = typeCoercionFieldName;
+ }
+ }
+
+ private enum SqlCompatSelectCoercionKind {
+
+ /**
+ * Indicates that no transformation is needed.
+ * Applicable to both type and cardinality coercion.
+ */
+ NONE,
+
+ /**
+ * Used to specify type and cardinality coercion.
+ * <p>
+ * When used for type coercion
+ * indicates that the output record of the {@link SelectBlock} must be transformed
+ * into a scalar value if that output record has 1 field, or transformed into MISSING value otherwise.
+ * <p>
+ * When used for cardinality coercion
+ * indicates that the result of the {@link SelectExpression}
+ * must be coerced into a single item if its cardinality is 1 or to MISSING otherwise.
+ */
+ SCALAR,
+
+ /**
+ * Only used to specify type coercion.
+ * <p>
+ * Indicates that the output record of the {@link SelectBlock} must be transformed
+ * into an array
+ */
+ ARRAY,
+
+ /**
+ * Only used to specify type coercion.
+ * <p>
+ * Indicates that the output record of the {@link SelectBlock} must be transformed
+ * into a multiset
+ */
+ MULTISET,
+ }
+
+ private void rewriteSelectBlock(SelectBlock selectBlock, SqlCompatSelectExpressionCoercionAnnotation ann)
+ throws CompilationException {
+ SelectClause selectClause = selectBlock.getSelectClause();
+ List<Projection> projectList = selectClause.getSelectRegular().getProjections();
+ SqlCompatSelectCoercionKind typeCoercion = ann.typeCoercion;
+ switch (typeCoercion) {
+ case SCALAR:
+ /*
+ * SELECT x -> SELECT x, x AS $new_unique_field
+ * SELECT x, y -> ERROR
+ * SELECT * -> ERROR
+ */
+ if (projectList.size() > 1) {
+ throw new CompilationException(ErrorCode.COMPILATION_SUBQUERY_COERCION_ERROR,
+ projectList.get(1).getSourceLocation(), "Subquery returns more than one field");
+ }
+ Projection projection = projectList.get(0);
+ if (projection.getKind() != Projection.Kind.NAMED_EXPR) {
+ throw new CompilationException(ErrorCode.COMPILATION_SUBQUERY_COERCION_ERROR,
+ projection.getSourceLocation(), "Unsupported projection kind");
+ }
+ Projection typeCoercionProj = new Projection(Projection.Kind.NAMED_EXPR,
+ (Expression) SqlppRewriteUtil.deepCopy(projection.getExpression()), ann.typeCoercionFieldName);
+ projectList.add(typeCoercionProj);
+ break;
+ case ARRAY:
+ case MULTISET:
+ /*
+ * SELECT x -> SELECT x, [x] AS $new_unique_field -- for ARRAY case
+ * (or SELECT VALUE x, {{x}} AS $new_unique_field) -- for MULTISET case
+ * SELECT x, y -> SELECT x, y, [x, y] AS $new_unique_field -- for ARRAY case
+ * (or SELECT x, y, {{x, y}} AS $new_unique_field) -- for MULTISET case
+ * SELECT * -> ERROR
+ */
+ List<Expression> exprList = new ArrayList<>(projectList.size());
+ for (Projection p : projectList) {
+ if (p.getKind() != Projection.Kind.NAMED_EXPR) {
+ throw new CompilationException(ErrorCode.COMPILATION_SUBQUERY_COERCION_ERROR,
+ p.getSourceLocation(), "Unsupported projection kind");
+ }
+ exprList.add((Expression) SqlppRewriteUtil.deepCopy(p.getExpression()));
+ }
+ ListConstructor.Type listType = typeCoercion == SqlCompatSelectCoercionKind.ARRAY
+ ? ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR
+ : ListConstructor.Type.UNORDERED_LIST_CONSTRUCTOR;
+ ListConstructor listExpr = new ListConstructor(listType, exprList);
+ listExpr.setSourceLocation(selectClause.getSourceLocation());
+ typeCoercionProj = new Projection(Projection.Kind.NAMED_EXPR, listExpr, ann.typeCoercionFieldName);
+ projectList.add(typeCoercionProj);
+ break;
+ case NONE:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.ILLEGAL_STATE, selectBlock.getSourceLocation(),
+ ann.toString());
+ }
+ }
+
+ private Expression rewriteSelectExpression(Expression inExpr, SqlCompatSelectExpressionCoercionAnnotation ann)
+ throws CompilationException {
+ SourceLocation sourceLoc = inExpr.getSourceLocation();
+
+ if (ann.typeCoercion != SqlCompatSelectCoercionKind.NONE) {
+ /*
+ * inExpr = SELECT ..., type_coercion_expr AS $new_unique_field
+ * -->
+ * inExpr = SELECT VALUE v1.$new_unique_field FROM (SELECT ..., type_coercion_expr AS $new_unique_field) v1
+ */
+ VarIdentifier v1 = context.newVariable();
+ VariableExpr v1Ref1 = new VariableExpr(v1);
+ v1Ref1.setSourceLocation(sourceLoc);
+ FromTerm ft1 = new FromTerm(inExpr, v1Ref1, null, null);
+ ft1.setSourceLocation(sourceLoc);
+ List<FromTerm> fc1Terms = new ArrayList<>(1);
+ fc1Terms.add(ft1);
+ FromClause fc1 = new FromClause(fc1Terms);
+ fc1.setSourceLocation(sourceLoc);
+ VariableExpr v1Ref2 = new VariableExpr(v1);
+ v1Ref2.setSourceLocation(sourceLoc);
+ FieldAccessor fa = new FieldAccessor(v1Ref2, new Identifier(ann.typeCoercionFieldName));
+ fa.setSourceLocation(sourceLoc);
+ SelectElement sv1 = new SelectElement(fa);
+ sv1.setSourceLocation(sourceLoc);
+ SelectClause sc1 = new SelectClause(sv1, null, false);
+ sc1.setSourceLocation(sourceLoc);
+ SelectBlock sb1 = new SelectBlock(sc1, fc1, null, null, null);
+ sv1.setSourceLocation(sourceLoc);
+ SelectSetOperation sop1 = new SelectSetOperation(new SetOperationInput(sb1, null), null);
+ sop1.setSourceLocation(sourceLoc);
+ SelectExpression se1 = new SelectExpression(null, sop1, null, null, true);
+ se1.setSourceLocation(sourceLoc);
+
+ inExpr = se1;
+ }
+
+ SqlCompatSelectCoercionKind cardinalityCoercion = ann.cardinalityCoercion;
+ switch (cardinalityCoercion) {
+ case SCALAR:
+ /*
+ * inExpr = (SELECT ...)
+ * ->
+ * STRICT_FIRST_ELEMENT
+ * (
+ * SELECT VALUE v2[(LEN(v2)-1)*2]
+ * LET v2 = (SELECT VALUE v1 FROM (inExpr) v1 LIMIT 2)
+ * )
+ */
+
+ /*
+ * E1: SELECT VALUE v1 FROM (inExpr) v1 LIMIT 2
+ */
+ VarIdentifier v1 = context.newVariable();
+ VariableExpr v1Ref1 = new VariableExpr(v1);
+ v1Ref1.setSourceLocation(sourceLoc);
+ FromTerm ft1 = new FromTerm(inExpr, v1Ref1, null, null);
+ ft1.setSourceLocation(sourceLoc);
+ List<FromTerm> fc1Terms = new ArrayList<>(1);
+ fc1Terms.add(ft1);
+ FromClause fc1 = new FromClause(fc1Terms);
+ fc1.setSourceLocation(sourceLoc);
+ VariableExpr v1Ref2 = new VariableExpr(v1);
+ v1Ref2.setSourceLocation(sourceLoc);
+ SelectElement sv1 = new SelectElement(v1Ref2);
+ sv1.setSourceLocation(sourceLoc);
+ SelectClause sc1 = new SelectClause(sv1, null, false);
+ sc1.setSourceLocation(sourceLoc);
+ SelectBlock sb1 = new SelectBlock(sc1, fc1, null, null, null);
+ sv1.setSourceLocation(sourceLoc);
+ SelectSetOperation sop1 = new SelectSetOperation(new SetOperationInput(sb1, null), null);
+ sop1.setSourceLocation(sourceLoc);
+ LimitClause lc1 = new LimitClause(new LiteralExpr(new IntegerLiteral(2)), null);
+ lc1.setSourceLocation(sourceLoc);
+ SelectExpression se1 = new SelectExpression(null, sop1, null, lc1, true);
+ se1.setSourceLocation(sourceLoc);
+
+ /*
+ * E2:
+ * SELECT VALUE v2[(LEN(v2)-1)*2]
+ * LET v2 = (..E1..)
+ *
+ * E2 returns {{ item }} if LEN(E1) == 1, otherwise it returns {{ MISSING }}
+ */
+ VarIdentifier v2 = context.newVariable();
+ VariableExpr v2Ref1 = new VariableExpr(v2);
+ v2Ref1.setSourceLocation(sourceLoc);
+ LetClause lc2 = new LetClause(v2Ref1, se1);
+ lc2.setSourceLocation(sourceLoc);
+
+ VariableExpr v2Ref2 = new VariableExpr(v2);
+ v2Ref2.setSourceLocation(sourceLoc);
+ List<Expression> lenArgs = new ArrayList<>(1);
+ lenArgs.add(v2Ref2);
+ CallExpr lenExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.LEN), lenArgs);
+ lenExpr.setSourceLocation(sourceLoc);
+
+ OperatorExpr minusExpr = new OperatorExpr();
+ minusExpr.setCurrentop(true);
+ minusExpr.addOperator(OperatorType.MINUS);
+ minusExpr.addOperand(lenExpr);
+ minusExpr.addOperand(new LiteralExpr(new IntegerLiteral(1)));
+ minusExpr.setSourceLocation(sourceLoc);
+
+ OperatorExpr mulExpr = new OperatorExpr();
+ mulExpr.setCurrentop(true);
+ mulExpr.addOperator(OperatorType.MUL);
+ mulExpr.addOperand(minusExpr);
+ mulExpr.addOperand(new LiteralExpr(new IntegerLiteral(2)));
+ mulExpr.setSourceLocation(sourceLoc);
+
+ VariableExpr v2Ref3 = new VariableExpr(v2);
+ v2Ref3.setSourceLocation(sourceLoc);
+ IndexAccessor iaExpr = new IndexAccessor(v2Ref3, IndexAccessor.IndexKind.ELEMENT, mulExpr);
+ iaExpr.setSourceLocation(sourceLoc);
+
+ SelectElement sv2 = new SelectElement(iaExpr);
+ sv2.setSourceLocation(sourceLoc);
+ SelectClause sc2 = new SelectClause(sv2, null, false);
+ sc2.setSourceLocation(sourceLoc);
+ List<AbstractClause> sb2Clauses = new ArrayList<>(1);
+ sb2Clauses.add(lc2);
+ SelectBlock sb2 = new SelectBlock(sc2, null, sb2Clauses, null, null);
+ sb2.setSourceLocation(sourceLoc);
+ SelectSetOperation sop2 = new SelectSetOperation(new SetOperationInput(sb2, null), null);
+ sop2.setSourceLocation(sourceLoc);
+ SelectExpression se2 = new SelectExpression(null, sop2, null, null, true);
+ se2.setSourceLocation(sourceLoc);
+
+ /*
+ * E3: STRICT_FIRST_ELEMENT(..E2..)
+ */
+ List<Expression> firstElemArgs = new ArrayList<>(1);
+ firstElemArgs.add(se2);
+ FunctionIdentifier firstElemFun =
+ FunctionMapUtil.createCoreAggregateFunctionIdentifier(BuiltinFunctions.SCALAR_FIRST_ELEMENT);
+ CallExpr firstElemExpr = new CallExpr(new FunctionSignature(firstElemFun), firstElemArgs);
+ firstElemExpr.setSourceLocation(sourceLoc);
+ return firstElemExpr;
+ case NONE:
+ // indicates that no transformation is necessary
+ return inExpr;
+ default:
+ throw new CompilationException(ErrorCode.ILLEGAL_STATE, inExpr.getSourceLocation(), ann.toString());
+ }
+ }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
index 8559c96..592d3f6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGatherFunctionCallsVisitor.java
@@ -89,10 +89,7 @@
@Override
public Void visit(Projection projection, Void arg) throws CompilationException {
- if (!projection.star()) {
- projection.getExpression().accept(this, arg);
- }
- return null;
+ return projection.hasExpression() ? projection.getExpression().accept(this, arg) : null;
}
@Override
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupingSetsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupingSetsVisitor.java
index 791bfe1..1259b64 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupingSetsVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppGroupingSetsVisitor.java
@@ -165,7 +165,7 @@
// For regular SELECT we need to add ORDERBY/LIMIT free variables to the projection list of the SELECT clause,
// so they can be accessed using field-access-by-name after SELECT.
// Some of these variables might be already projected by SELECT, so we need to account for that.
- // We currently do not support (fail) SELECT v.* because in this case we cannot statically compute
+ // We currently do not support (fail) SELECT v.* and *.* because in this case we cannot statically compute
// the schema of the record produced by the SELECT and therefore cannot guarantee that the field
// names we generate will not conflict with the existing fields in the SELECT output.
// The added projections will be later removed by the outer query.
@@ -177,33 +177,44 @@
List<VariableExpr> gbyBindingVars = null, postGbyBindingVars = null;
for (VariableExpr freeVarPostSetOp : freeVarsPostSetOp) {
String projectionName = null;
- for (int i = projectionList.size() - 1; i >= 0; i--) {
+ inner_loop: for (int i = projectionList.size() - 1; i >= 0; i--) {
Projection projection = projectionList.get(i);
- if (projection.varStar()) {
- throw new CompilationException(ErrorCode.UNSUPPORTED_GBY_OBY_SELECT_COMBO,
- selectBlock.getSourceLocation());
- } else if (projection.star()) {
- if (gbyBindingVars == null) {
- gbyBindingVars = SqlppVariableUtil.getBindingVariables(selectBlock.getGroupbyClause());
- }
- if (postGbyBindingVars == null) {
- postGbyBindingVars = selectBlock.hasLetHavingClausesAfterGroupby()
- ? SqlppVariableUtil.getLetBindingVariables(selectBlock.getLetHavingListAfterGroupby())
- : Collections.emptyList();
- }
- if (gbyBindingVars.contains(freeVarPostSetOp) || postGbyBindingVars.contains(freeVarPostSetOp)) {
- projectionName = SqlppVariableUtil
- .variableNameToDisplayedFieldName(freeVarPostSetOp.getVar().getValue());
+ switch (projection.getKind()) {
+ case VAR_STAR:
+ case EVERY_VAR_STAR:
+ throw new CompilationException(ErrorCode.UNSUPPORTED_GBY_OBY_SELECT_COMBO,
+ selectBlock.getSourceLocation());
+ case STAR:
+ if (gbyBindingVars == null) {
+ gbyBindingVars = SqlppVariableUtil.getBindingVariables(selectBlock.getGroupbyClause());
+ }
+ if (postGbyBindingVars == null) {
+ postGbyBindingVars =
+ selectBlock.hasLetHavingClausesAfterGroupby()
+ ? SqlppVariableUtil
+ .getLetBindingVariables(selectBlock.getLetHavingListAfterGroupby())
+ : Collections.emptyList();
+ }
+ if (gbyBindingVars.contains(freeVarPostSetOp)
+ || postGbyBindingVars.contains(freeVarPostSetOp)) {
+ projectionName = SqlppVariableUtil
+ .variableNameToDisplayedFieldName(freeVarPostSetOp.getVar().getValue());
+ break;
+ }
break;
- }
- } else if (projection.hasName()) {
- if (projection.getExpression().equals(freeVarPostSetOp)) {
- projectionName = projection.getName();
+ case NAMED_EXPR:
+ if (!projection.hasName()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ selectBlock.getSourceLocation(), "");
+ }
+ if (projection.getExpression().equals(freeVarPostSetOp)) {
+ projectionName = projection.getName();
+ break inner_loop;
+ }
break;
- }
- } else {
- throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, selectBlock.getSourceLocation(),
- "");
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ selectBlock.getSourceLocation(), "");
}
}
@@ -396,7 +407,8 @@
}
SelectElement selectElement = selectClause.getSelectElement();
List<Projection> projectionList = new ArrayList<>(1);
- projectionList.add(new Projection(selectElement.getExpression(), mainProjectionName, false, false));
+ projectionList
+ .add(new Projection(Projection.Kind.NAMED_EXPR, selectElement.getExpression(), mainProjectionName));
SelectRegular newSelectRegular = new SelectRegular(projectionList);
newSelectRegular.setSourceLocation(selectElement.getSourceLocation());
SelectClause newSelectClause = new SelectClause(null, newSelectRegular, selectClause.distinct());
@@ -412,8 +424,8 @@
}
SelectRegular selectRegular = selectClause.getSelectRegular();
for (Map.Entry<VariableExpr, String> me : projections.entrySet()) {
- Projection newProjection =
- new Projection((VariableExpr) SqlppRewriteUtil.deepCopy(me.getKey()), me.getValue(), false, false);
+ Projection newProjection = new Projection(Projection.Kind.NAMED_EXPR,
+ (VariableExpr) SqlppRewriteUtil.deepCopy(me.getKey()), me.getValue());
selectRegular.getProjections().add(newProjection);
}
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
index 5200b94..9f93363 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppInlineUdfsVisitor.java
@@ -24,12 +24,14 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.expression.ListSliceExpression;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.AbstractInlineUdfsVisitor;
import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
@@ -61,9 +63,12 @@
* manages ids of variables and guarantees uniqueness of variables.
* @param usedUDFs,
* user defined functions used by this query.
+ * @param usedViews,
+ * views used by this query.
*/
- public SqlppInlineUdfsVisitor(LangRewritingContext context, Map<FunctionSignature, FunctionDecl> usedUDFs) {
- super(context, usedUDFs, new SqlppCloneAndSubstituteVariablesVisitor(context));
+ public SqlppInlineUdfsVisitor(LangRewritingContext context, Map<FunctionSignature, FunctionDecl> usedUDFs,
+ Map<DatasetFullyQualifiedName, ViewDecl> usedViews) {
+ super(context, usedUDFs, usedViews, new SqlppCloneAndSubstituteVariablesVisitor(context));
}
@Override
@@ -85,7 +90,7 @@
@Override
public Boolean visit(FromTerm fromTerm, Void arg) throws CompilationException {
boolean changed = false;
- Pair<Boolean, Expression> p = inlineUdfsInExpr(fromTerm.getLeftExpression());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(fromTerm.getLeftExpression());
fromTerm.setLeftExpression(p.second);
changed |= p.first;
for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
@@ -96,28 +101,28 @@
@Override
public Boolean visit(JoinClause joinClause, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p1 = inlineUdfsInExpr(joinClause.getRightExpression());
+ Pair<Boolean, Expression> p1 = inlineUdfsAndViewsInExpr(joinClause.getRightExpression());
joinClause.setRightExpression(p1.second);
- Pair<Boolean, Expression> p2 = inlineUdfsInExpr(joinClause.getConditionExpression());
+ Pair<Boolean, Expression> p2 = inlineUdfsAndViewsInExpr(joinClause.getConditionExpression());
joinClause.setConditionExpression(p2.second);
return p1.first || p2.first;
}
@Override
public Boolean visit(NestClause nestClause, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p1 = inlineUdfsInExpr(nestClause.getRightExpression());
+ Pair<Boolean, Expression> p1 = inlineUdfsAndViewsInExpr(nestClause.getRightExpression());
nestClause.setRightExpression(p1.second);
- Pair<Boolean, Expression> p2 = inlineUdfsInExpr(nestClause.getConditionExpression());
+ Pair<Boolean, Expression> p2 = inlineUdfsAndViewsInExpr(nestClause.getConditionExpression());
nestClause.setConditionExpression(p2.second);
return p1.first || p2.first;
}
@Override
public Boolean visit(Projection projection, Void arg) throws CompilationException {
- if (projection.star()) {
+ if (!projection.hasExpression()) {
return false;
}
- Pair<Boolean, Expression> p = inlineUdfsInExpr(projection.getExpression());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(projection.getExpression());
projection.setExpression(p.second);
return p.first;
}
@@ -158,7 +163,7 @@
@Override
public Boolean visit(SelectElement selectElement, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(selectElement.getExpression());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(selectElement.getExpression());
selectElement.setExpression(p.second);
return p.first;
}
@@ -202,21 +207,21 @@
@Override
public Boolean visit(UnnestClause unnestClause, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(unnestClause.getRightExpression());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(unnestClause.getRightExpression());
unnestClause.setRightExpression(p.second);
return p.first;
}
@Override
public Boolean visit(HavingClause havingClause, Void arg) throws CompilationException {
- Pair<Boolean, Expression> p = inlineUdfsInExpr(havingClause.getFilterExpression());
+ Pair<Boolean, Expression> p = inlineUdfsAndViewsInExpr(havingClause.getFilterExpression());
havingClause.setFilterExpression(p.second);
return p.first;
}
@Override
public Boolean visit(CaseExpression caseExpr, Void arg) throws CompilationException {
- Pair<Boolean, Expression> result = inlineUdfsInExpr(caseExpr.getConditionExpr());
+ Pair<Boolean, Expression> result = inlineUdfsAndViewsInExpr(caseExpr.getConditionExpr());
caseExpr.setConditionExpr(result.second);
boolean inlined = result.first;
@@ -228,7 +233,7 @@
inlined = inlined || inlinedList.first;
caseExpr.setThenExprs(inlinedList.second);
- result = inlineUdfsInExpr(caseExpr.getElseExpr());
+ result = inlineUdfsAndViewsInExpr(caseExpr.getElseExpr());
caseExpr.setElseExpr(result.second);
return inlined || result.first;
}
@@ -247,12 +252,12 @@
inlined |= inlinedList.first;
}
if (winExpr.hasFrameStartExpr()) {
- Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameStartExpr());
+ Pair<Boolean, Expression> inlinedExpr = inlineUdfsAndViewsInExpr(winExpr.getFrameStartExpr());
winExpr.setFrameStartExpr(inlinedExpr.second);
inlined |= inlinedExpr.first;
}
if (winExpr.hasFrameEndExpr()) {
- Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getFrameEndExpr());
+ Pair<Boolean, Expression> inlinedExpr = inlineUdfsAndViewsInExpr(winExpr.getFrameEndExpr());
winExpr.setFrameEndExpr(inlinedExpr.second);
inlined |= inlinedExpr.first;
}
@@ -263,7 +268,7 @@
inlined |= inlinedList.first;
}
if (winExpr.hasAggregateFilterExpr()) {
- Pair<Boolean, Expression> inlinedExpr = inlineUdfsInExpr(winExpr.getAggregateFilterExpr());
+ Pair<Boolean, Expression> inlinedExpr = inlineUdfsAndViewsInExpr(winExpr.getAggregateFilterExpr());
winExpr.setAggregateFilterExpr(inlinedExpr.second);
inlined |= inlinedExpr.first;
}
@@ -275,17 +280,19 @@
@Override
public Boolean visit(ListSliceExpression expression, Void arg) throws CompilationException {
- Pair<Boolean, Expression> expressionResult = inlineUdfsInExpr(expression.getExpr());
+ Pair<Boolean, Expression> expressionResult = inlineUdfsAndViewsInExpr(expression.getExpr());
expression.setExpr(expressionResult.second);
boolean inlined = expressionResult.first;
- Pair<Boolean, Expression> startIndexExpressResult = inlineUdfsInExpr(expression.getStartIndexExpression());
+ Pair<Boolean, Expression> startIndexExpressResult =
+ inlineUdfsAndViewsInExpr(expression.getStartIndexExpression());
expression.setStartIndexExpression(startIndexExpressResult.second);
inlined |= startIndexExpressResult.first;
// End index expression can be null (optional)
if (expression.hasEndExpression()) {
- Pair<Boolean, Expression> endIndexExpressionResult = inlineUdfsInExpr(expression.getEndIndexExpression());
+ Pair<Boolean, Expression> endIndexExpressionResult =
+ inlineUdfsAndViewsInExpr(expression.getEndIndexExpression());
expression.setEndIndexExpression(endIndexExpressionResult.second);
inlined |= endIndexExpressionResult.first;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
index a8db06f..26c6915 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
@@ -32,6 +32,7 @@
import org.apache.asterix.lang.common.base.Clause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
@@ -161,9 +162,10 @@
VariableExpr rightVar = joinClause.getRightVariable();
VariableExpr rightPosVar = joinClause.getPositionalVariable();
Expression condExpr = joinClause.getConditionExpression();
+ Literal.Type outerMissingValueType = joinClause.getOuterJoinMissingValueType();
if (i == 0) {
- JoinClause newJoinClause =
- new JoinClause(JoinType.LEFTOUTER, fromExpr, fromVar, fromPosVar, condExpr);
+ JoinClause newJoinClause = new JoinClause(JoinType.LEFTOUTER, fromExpr, fromVar, fromPosVar,
+ condExpr, outerMissingValueType);
newJoinClause.setSourceLocation(joinClauseSourceLoc);
fromExpr = rightExpr;
@@ -249,7 +251,7 @@
? newVariableExpr(newPrecedingClauseRightPosVar,
precedingClauseRightPosVarSourceLoc)
: null,
- newCondExpr);
+ newCondExpr, joinPrecedingClause.getOuterJoinMissingValueType());
newPrecedingClause.setSourceLocation(precedingClauseSourceLoc);
break;
case UNNEST_CLAUSE:
@@ -260,10 +262,10 @@
newPrecedingClause = new UnnestClause(unnestPrecedingClause.getUnnestType(),
newRightExpr,
newVariableExpr(newPrecedingClauseRightVar, precedingClauseRightVarSourceLoc),
- newPrecedingClauseRightPosVar != null
- ? newVariableExpr(newPrecedingClauseRightPosVar,
- precedingClauseRightPosVarSourceLoc)
- : null);
+ newPrecedingClauseRightPosVar != null ? newVariableExpr(
+ newPrecedingClauseRightPosVar, precedingClauseRightPosVarSourceLoc)
+ : null,
+ unnestPrecedingClause.getOuterUnnestMissingValueType());
newPrecedingClause.setSourceLocation(precedingClauseSourceLoc);
break;
default:
@@ -291,8 +293,8 @@
Expression newCondExpr = SqlppRewriteUtil.substituteExpression(
(Expression) SqlppRewriteUtil.deepCopy(condExpr), substMapOuterFinal, context);
- JoinClause newJoinClause =
- new JoinClause(JoinType.LEFTOUTER, newRightExpr, newRightVarExpr, null, newCondExpr);
+ JoinClause newJoinClause = new JoinClause(JoinType.LEFTOUTER, newRightExpr, newRightVarExpr, null,
+ newCondExpr, outerMissingValueType);
newJoinClause.setSourceLocation(joinClauseSourceLoc);
fromExpr = rightExpr;
@@ -370,7 +372,8 @@
}
private Projection createProjection(VariableExpr var, String fieldName, SourceLocation sourceLoc) {
- Projection projection = new Projection(newVariableExpr(var.getVar(), null), fieldName, false, false);
+ Projection projection =
+ new Projection(Projection.Kind.NAMED_EXPR, newVariableExpr(var.getVar(), null), fieldName);
projection.setSourceLocation(sourceLoc);
return projection;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
index 6d8e52e..bf3e227 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
@@ -117,8 +117,11 @@
Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> liveAnnotatedVars =
scopeChecker.getCurrentScope().getLiveVariables();
Set<VariableExpr> liveVars = liveAnnotatedVars.keySet();
- Set<VariableExpr> liveContextVars = Scope.findVariablesAnnotatedBy(liveVars,
- SqlppVariableAnnotation.CONTEXT_VARIABLE, liveAnnotatedVars, winExpr.getSourceLocation());
+
+ Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> localAnnotatedVars =
+ scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope());
+ Set<VariableExpr> liveContextVars = Scope.findVariablesAnnotatedBy(localAnnotatedVars.keySet(),
+ SqlppVariableAnnotation.CONTEXT_VARIABLE, localAnnotatedVars, winExpr.getSourceLocation());
List<Pair<Expression, Identifier>> winFieldList = winExpr.getWindowFieldList();
Map<VariableExpr, Identifier> winVarFieldMap =
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index bbf1a46..4b0caca 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -25,10 +25,12 @@
import java.util.Map;
import java.util.Set;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.Expression.Kind;
@@ -38,8 +40,11 @@
import org.apache.asterix.lang.common.expression.FieldAccessor;
import org.apache.asterix.lang.common.expression.LiteralExpr;
import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.FalseLiteral;
import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.literal.TrueLiteral;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
+import org.apache.asterix.lang.common.statement.ViewDecl;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.lang.sqlpp.expression.WindowExpression;
@@ -169,15 +174,30 @@
return null;
}
SourceLocation sourceLoc = varExpr.getSourceLocation();
- Pair<Dataset, Boolean> datasetSynonymPair = findDataset(dataverseName, datasetName, sourceLoc);
- if (datasetSynonymPair == null) {
- throw createUnresolvableError(dataverseName, datasetName, sourceLoc);
+ DataverseName resolvedDataverseName;
+ String resolvedDatasetName;
+ boolean viaSynonym, isView;
+ ViewDecl viewDecl = findDeclaredView(dataverseName, datasetName);
+ if (viewDecl != null) {
+ resolvedDataverseName = viewDecl.getViewName().getDataverseName();
+ resolvedDatasetName = viewDecl.getViewName().getDatasetName();
+ viaSynonym = false;
+ isView = true;
+ } else {
+ Pair<Dataset, Boolean> p = findDataset(dataverseName, datasetName, true, sourceLoc);
+ if (p == null) {
+ throw createUnresolvableError(dataverseName, datasetName, sourceLoc);
+ }
+ Dataset resolvedDataset = p.first;
+ resolvedDataverseName = resolvedDataset.getDataverseName();
+ resolvedDatasetName = resolvedDataset.getDatasetName();
+ viaSynonym = p.second;
+ isView = resolvedDataset.getDatasetType() == DatasetConfig.DatasetType.VIEW;
}
- Dataset dataset = datasetSynonymPair.first;
- boolean viaSynonym = datasetSynonymPair.second;
- List<Expression> argList = new ArrayList<>(4);
- argList.add(new LiteralExpr(new StringLiteral(dataset.getDataverseName().getCanonicalForm())));
- argList.add(new LiteralExpr(new StringLiteral(dataset.getDatasetName())));
+ List<Expression> argList = new ArrayList<>(3 + (viaSynonym ? 2 : 0));
+ argList.add(new LiteralExpr(new StringLiteral(resolvedDataverseName.getCanonicalForm())));
+ argList.add(new LiteralExpr(new StringLiteral(resolvedDatasetName)));
+ argList.add(new LiteralExpr(isView ? TrueLiteral.INSTANCE : FalseLiteral.INSTANCE));
if (viaSynonym) {
argList.add(new LiteralExpr(new StringLiteral(dataverseName.getCanonicalForm())));
argList.add(new LiteralExpr(new StringLiteral(datasetName)));
@@ -241,24 +261,30 @@
dataverseName == null ? defaultDataverseName : dataverseName);
}
- private Pair<Dataset, Boolean> findDataset(DataverseName dataverseName, String datasetName,
+ private Pair<Dataset, Boolean> findDataset(DataverseName dataverseName, String datasetName, boolean includingViews,
SourceLocation sourceLoc) throws CompilationException {
try {
Boolean viaSynonym = false;
Triple<DataverseName, String, Boolean> dsName =
- metadataProvider.resolveDatasetNameUsingSynonyms(dataverseName, datasetName);
+ metadataProvider.resolveDatasetNameUsingSynonyms(dataverseName, datasetName, includingViews);
if (dsName != null) {
dataverseName = dsName.first;
datasetName = dsName.second;
viaSynonym = dsName.third;
}
- Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+ Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName, includingViews);
return dataset == null ? null : new Pair<>(dataset, viaSynonym);
} catch (AlgebricksException e) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
}
}
+ private ViewDecl findDeclaredView(DataverseName dataverseName, String viewName) {
+ Map<DatasetFullyQualifiedName, ViewDecl> declaredViews = context.getDeclaredViews();
+ return declaredViews.isEmpty() ? null
+ : declaredViews.get(new DatasetFullyQualifiedName(dataverseName, viewName));
+ }
+
@Override
public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
// skip variables inside SQL-92 aggregates (they will be resolved by SqlppGroupByAggregationSugarVisitor)
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
index a46f10b..2d41345 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
@@ -204,4 +204,9 @@
}
return functionName;
}
+
+ public static FunctionIdentifier createCoreAggregateFunctionIdentifier(FunctionIdentifier scalarfi) {
+ return BuiltinFunctions.getAggregateFunction(scalarfi) != null ? new FunctionIdentifier(scalarfi.getNamespace(),
+ CORE_AGGREGATE_PREFIX + scalarfi.getName(), scalarfi.getArity()) : null;
+ }
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java
index 5281dd9..6cac4c9 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java
@@ -21,6 +21,8 @@
import java.util.List;
import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.utils.TypeUtil;
+import org.apache.hyracks.util.OptionalBoolean;
public class SqlppStatementUtil {
@@ -32,6 +34,9 @@
public static final String CREATE_INDEX = "CREATE INDEX ";
public static final String CREATE_PRIMARY_INDEX = "CREATE PRIMARY INDEX ";
public static final String DROP_INDEX = "DROP INDEX ";
+ public static final String INCLUDE_UNKNOWN_KEY = " INCLUDE UNKNOWN KEY ";
+ public static final String EXCLUDE_UNKNOWN_KEY = " EXCLUDE UNKNOWN KEY ";
+ public static final String CAST_DEFAULT_NULL = " CAST (DEFAULT NULL ";
public static final String ON = " ON ";
public static final String WHERE = " WHERE ";
public static final String AND = " AND ";
@@ -69,10 +74,35 @@
@SuppressWarnings("squid:S1172") // unused variable
public static StringBuilder getCreateIndexStatement(StringBuilder stringBuilder, DataverseName dataverseName,
- String datasetName, String indexName, String fields, int version) {
+ String datasetName, String indexName, String fields, OptionalBoolean excludeUnknown,
+ boolean castDefaultNull, String dateTimeFmt, String dateFmt, String timeFmt, int version) {
stringBuilder.append(CREATE_INDEX);
enclose(stringBuilder, indexName).append(ON);
- return enclose(stringBuilder, dataverseName, datasetName).append(fields).append(SEMI_COLON);
+ StringBuilder appender = enclose(stringBuilder, dataverseName, datasetName).append(fields);
+ if (excludeUnknown.isPresent()) {
+ if (excludeUnknown.get()) {
+ appender.append(EXCLUDE_UNKNOWN_KEY);
+ } else {
+ appender.append(INCLUDE_UNKNOWN_KEY);
+ }
+ }
+ if (castDefaultNull) {
+ appender.append(CAST_DEFAULT_NULL);
+ if (dateTimeFmt != null) {
+ appender.append(TypeUtil.DATETIME_PARAMETER_NAME).append(' ');
+ quote(appender, dateTimeFmt).append(' ');
+ }
+ if (dateFmt != null) {
+ appender.append(TypeUtil.DATE_PARAMETER_NAME).append(' ');
+ quote(appender, dateFmt).append(' ');
+ }
+ if (timeFmt != null) {
+ appender.append(TypeUtil.TIME_PARAMETER_NAME).append(' ');
+ quote(appender, timeFmt).append(' ');
+ }
+ appender.append(R_PARENTHESIS);
+ }
+ return appender.append(SEMI_COLON);
}
@SuppressWarnings("squid:S1172") // unused variable
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
index 26f2a35..9d50160 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppVariableUtil.java
@@ -103,6 +103,10 @@
return EXTERNAL_VAR_PREFIX + varName;
}
+ public static VarIdentifier toExternalVariableIdentifier(String idName) {
+ return new VarIdentifier(toExternalVariableName(idName));
+ }
+
public static boolean isPositionalVariableIdentifier(VarIdentifier varId) {
try {
Integer.parseInt(toUserDefinedName(varId.getValue()));
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
index 027560c..3192541 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSql92AggregateVisitor.java
@@ -240,10 +240,7 @@
@Override
public Boolean visit(Projection projection, ILangExpression parentSelectBlock) throws CompilationException {
- if (projection.star()) {
- return false;
- }
- return projection.getExpression().accept(this, parentSelectBlock);
+ return projection.hasExpression() ? projection.getExpression().accept(this, parentSelectBlock) : false;
}
@Override
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
index bc31e62..a903279 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/CheckSubqueryVisitor.java
@@ -107,10 +107,7 @@
@Override
public Boolean visit(Projection projection, ILangExpression arg) throws CompilationException {
- if (projection.star()) {
- return false;
- }
- return visit(projection.getExpression(), arg);
+ return projection.hasExpression() && visit(projection.getExpression(), arg);
}
@Override
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index 998d8e6..d74d047 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -112,7 +112,7 @@
: (VariableExpr) joinClause.getPositionalVariable().accept(this, arg);
Expression conditionExpresion = (Expression) joinClause.getConditionExpression().accept(this, arg);
JoinClause copy = new JoinClause(joinClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
- conditionExpresion);
+ conditionExpresion, joinClause.getOuterJoinMissingValueType());
copy.setSourceLocation(joinClause.getSourceLocation());
return copy;
}
@@ -136,16 +136,17 @@
VariableExpr rightVar = (VariableExpr) unnestClause.getRightVariable().accept(this, arg);
VariableExpr rightPositionVar = unnestClause.getPositionalVariable() == null ? null
: (VariableExpr) unnestClause.getPositionalVariable().accept(this, arg);
- UnnestClause copy = new UnnestClause(unnestClause.getUnnestType(), rightExpression, rightVar, rightPositionVar);
+ UnnestClause copy = new UnnestClause(unnestClause.getUnnestType(), rightExpression, rightVar, rightPositionVar,
+ unnestClause.getOuterUnnestMissingValueType());
copy.setSourceLocation(unnestClause.getSourceLocation());
return copy;
}
@Override
public Projection visit(Projection projection, Void arg) throws CompilationException {
- Projection copy =
- new Projection(projection.star() ? null : (Expression) projection.getExpression().accept(this, arg),
- projection.getName(), projection.star(), projection.varStar());
+ Projection copy = new Projection(projection.getKind(),
+ projection.hasExpression() ? (Expression) projection.getExpression().accept(this, arg) : null,
+ projection.getName());
copy.setSourceLocation(projection.getSourceLocation());
return copy;
}
@@ -276,7 +277,8 @@
for (Expression orderExpr : oc.getOrderbyList()) {
newOrderbyList.add((Expression) orderExpr.accept(this, arg));
}
- OrderbyClause copy = new OrderbyClause(newOrderbyList, new ArrayList<>(oc.getModifierList()));
+ OrderbyClause copy = new OrderbyClause(newOrderbyList, new ArrayList<>(oc.getModifierList()),
+ new ArrayList<>(oc.getNullModifierList()));
copy.setSourceLocation(oc.getSourceLocation());
return copy;
}
@@ -526,6 +528,8 @@
List<Expression> newOrderbyList = winExpr.hasOrderByList() ? copyExprList(winExpr.getOrderbyList(), arg) : null;
List<OrderbyClause.OrderModifier> newOrderbyModifierList =
winExpr.hasOrderByList() ? new ArrayList<>(winExpr.getOrderbyModifierList()) : null;
+ List<OrderbyClause.NullOrderModifier> newOrderbyNullModifierList =
+ winExpr.hasOrderByList() ? new ArrayList<>(winExpr.getOrderbyNullModifierList()) : null;
Expression newFrameStartExpr =
winExpr.hasFrameStartExpr() ? (Expression) winExpr.getFrameStartExpr().accept(this, arg) : null;
Expression newFrameEndExpr =
@@ -534,11 +538,11 @@
winExpr.hasWindowVar() ? (VariableExpr) winExpr.getWindowVar().accept(this, arg) : null;
List<Pair<Expression, Identifier>> newWindowFieldList =
winExpr.hasWindowFieldList() ? copyFieldList(winExpr.getWindowFieldList(), arg) : null;
- WindowExpression copy =
- new WindowExpression(winExpr.getFunctionSignature(), newExprList, newAggFilterExpr, newPartitionList,
- newOrderbyList, newOrderbyModifierList, winExpr.getFrameMode(), winExpr.getFrameStartKind(),
- newFrameStartExpr, winExpr.getFrameEndKind(), newFrameEndExpr, winExpr.getFrameExclusionKind(),
- newWindowVar, newWindowFieldList, winExpr.getIgnoreNulls(), winExpr.getFromLast());
+ WindowExpression copy = new WindowExpression(winExpr.getFunctionSignature(), newExprList, newAggFilterExpr,
+ newPartitionList, newOrderbyList, newOrderbyModifierList, newOrderbyNullModifierList,
+ winExpr.getFrameMode(), winExpr.getFrameStartKind(), newFrameStartExpr, winExpr.getFrameEndKind(),
+ newFrameEndExpr, winExpr.getFrameExclusionKind(), newWindowVar, newWindowFieldList,
+ winExpr.getIgnoreNulls(), winExpr.getFromLast());
copy.setSourceLocation(winExpr.getSourceLocation());
copy.addHints(winExpr.getHints());
return copy;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
index 0c14688..77b9991 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/FreeVariableVisitor.java
@@ -146,10 +146,7 @@
@Override
public Void visit(Projection projection, Collection<VariableExpr> freeVars) throws CompilationException {
- if (!projection.star()) {
- projection.getExpression().accept(this, freeVars);
- }
- return null;
+ return projection.hasExpression() ? projection.getExpression().accept(this, freeVars) : null;
}
@Override
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index 083015b..c803d83 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -22,12 +22,13 @@
import java.util.Collections;
import java.util.List;
import java.util.Map;
-import java.util.stream.Collectors;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.OrderbyClause;
@@ -59,6 +60,7 @@
import org.apache.asterix.lang.sqlpp.struct.SetOperationRight;
import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
public class SqlppAstPrintVisitor extends QueryPrintVisitor implements ISqlppVisitor<Void, Integer> {
@@ -108,6 +110,7 @@
out.print(" AT ");
joinClause.getPositionalVariable().accept(this, 0);
}
+ out.print(Literal.Type.NULL.equals(joinClause.getOuterJoinMissingValueType()) ? "(OR NULL) " : "");
out.println(skip(step + 1) + "ON");
joinClause.getConditionExpression().accept(this, step + 1);
return null;
@@ -130,11 +133,21 @@
@Override
public Void visit(Projection projection, Integer step) throws CompilationException {
- if (projection.star()) {
- out.println(skip(step) + "*");
- } else {
- projection.getExpression().accept(this, step);
- out.println(skip(step) + (projection.varStar() ? ".*" : projection.getName()));
+ switch (projection.getKind()) {
+ case STAR:
+ out.println(skip(step) + "*");
+ break;
+ case EVERY_VAR_STAR:
+ out.println(skip(step) + "*.*");
+ break;
+ case VAR_STAR:
+ projection.getExpression().accept(this, step);
+ out.println(skip(step) + ".*");
+ break;
+ case NAMED_EXPR:
+ projection.getExpression().accept(this, step);
+ out.println(skip(step) + projection.getName());
+ break;
}
return null;
}
@@ -252,8 +265,10 @@
FunctionSignature functionSignature = callExpr.getFunctionSignature();
//TODO(MULTI_PART_DATAVERSE_NAME):temporary workaround to preserve AST reference results
if (FunctionUtil.isBuiltinDatasetFunction(functionSignature)) {
- String singleArg = callExpr.getExprList().stream().map(LiteralExpr.class::cast).map(LiteralExpr::getValue)
- .map(StringLiteral.class::cast).map(StringLiteral::getValue).collect(Collectors.joining("."));
+ Triple<DatasetFullyQualifiedName, Boolean, DatasetFullyQualifiedName> dsArgs =
+ FunctionUtil.parseDatasetFunctionArguments(callExpr);
+ String singleArg =
+ String.join(".", dsArgs.first.getDataverseName().getParts()) + "." + dsArgs.first.getDatasetName();
printFunctionCall(functionSignature, 1,
Collections.singletonList(new LiteralExpr(new StringLiteral(singleArg))),
callExpr.getAggregateFilterExpr(), step);
@@ -390,9 +405,13 @@
out.println(skip(step + 1) + "ORDER BY");
List<Expression> orderbyList = winExpr.getOrderbyList();
List<OrderbyClause.OrderModifier> orderbyModifierList = winExpr.getOrderbyModifierList();
+ List<OrderbyClause.NullOrderModifier> orderbyNullModifierList = winExpr.getOrderbyNullModifierList();
for (int i = 0, ln = orderbyList.size(); i < ln; i++) {
orderbyList.get(i).accept(this, step + 2);
- out.println(skip(step + 2) + orderbyModifierList.get(i));
+ OrderbyClause.OrderModifier orderModifier = orderbyModifierList.get(i);
+ OrderbyClause.NullOrderModifier nullOrderModifier = orderbyNullModifierList.get(i);
+ out.println(skip(step + 2) + orderModifier
+ + (nullOrderModifier != null ? " NULLS " + nullOrderModifier : ""));
}
}
if (winExpr.hasFrameDefinition()) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
index efe1c3d..ce0d0a1 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
@@ -143,8 +143,8 @@
// The condition can refer to the newRightVar and newRightPosVar.
Expression conditionExpr = (Expression) joinClause.getConditionExpression().accept(this, currentEnv).first;
- JoinClause newJoinClause =
- new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar, conditionExpr);
+ JoinClause newJoinClause = new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar,
+ conditionExpr, joinClause.getOuterJoinMissingValueType());
newJoinClause.setSourceLocation(joinClause.getSourceLocation());
return new Pair<>(newJoinClause, currentEnv);
}
@@ -193,8 +193,8 @@
currentEnv.removeSubstitution(newRightPosVar);
}
// The condition can refer to the newRightVar and newRightPosVar.
- UnnestClause newUnnestClause =
- new UnnestClause(unnestClause.getUnnestType(), rightExpr, newRightVar, newRightPosVar);
+ UnnestClause newUnnestClause = new UnnestClause(unnestClause.getUnnestType(), rightExpr, newRightVar,
+ newRightPosVar, unnestClause.getOuterUnnestMissingValueType());
newUnnestClause.setSourceLocation(unnestClause.getSourceLocation());
return new Pair<>(newUnnestClause, currentEnv);
}
@@ -202,11 +202,9 @@
@Override
public Pair<ILangExpression, VariableSubstitutionEnvironment> visit(Projection projection,
VariableSubstitutionEnvironment env) throws CompilationException {
- if (projection.star()) {
- return new Pair<>(projection, env);
- }
- Projection newProjection = new Projection((Expression) projection.getExpression().accept(this, env).first,
- projection.getName(), projection.star(), projection.varStar());
+ Projection newProjection = new Projection(projection.getKind(),
+ projection.hasExpression() ? (Expression) projection.getExpression().accept(this, env).first : null,
+ projection.getName());
newProjection.setSourceLocation(projection.getSourceLocation());
return new Pair<>(newProjection, env);
}
@@ -414,6 +412,8 @@
? VariableCloneAndSubstitutionUtil.visitAndCloneExprList(winExpr.getOrderbyList(), env, this) : null;
List<OrderbyClause.OrderModifier> newOrderbyModifierList =
winExpr.hasOrderByList() ? new ArrayList<>(winExpr.getOrderbyModifierList()) : null;
+ List<OrderbyClause.NullOrderModifier> newOrderbyNullModifierList =
+ winExpr.hasOrderByList() ? new ArrayList<>(winExpr.getOrderbyNullModifierList()) : null;
Expression newFrameStartExpr =
winExpr.hasFrameStartExpr() ? (Expression) winExpr.getFrameStartExpr().accept(this, env).first : null;
Expression newFrameEndExpr =
@@ -422,11 +422,11 @@
winExpr.hasWindowVar() ? (VariableExpr) winExpr.getWindowVar().accept(this, env).first : null;
List<Pair<Expression, Identifier>> newWindowFieldList = winExpr.hasWindowFieldList()
? VariableCloneAndSubstitutionUtil.substInFieldList(winExpr.getWindowFieldList(), env, this) : null;
- WindowExpression newWinExpr =
- new WindowExpression(winExpr.getFunctionSignature(), newExprList, newAggFilterExpr, newPartitionList,
- newOrderbyList, newOrderbyModifierList, winExpr.getFrameMode(), winExpr.getFrameStartKind(),
- newFrameStartExpr, winExpr.getFrameEndKind(), newFrameEndExpr, winExpr.getFrameExclusionKind(),
- newWindowVar, newWindowFieldList, winExpr.getIgnoreNulls(), winExpr.getFromLast());
+ WindowExpression newWinExpr = new WindowExpression(winExpr.getFunctionSignature(), newExprList,
+ newAggFilterExpr, newPartitionList, newOrderbyList, newOrderbyModifierList, newOrderbyNullModifierList,
+ winExpr.getFrameMode(), winExpr.getFrameStartKind(), newFrameStartExpr, winExpr.getFrameEndKind(),
+ newFrameEndExpr, winExpr.getFrameExclusionKind(), newWindowVar, newWindowFieldList,
+ winExpr.getIgnoreNulls(), winExpr.getFromLast());
newWinExpr.setSourceLocation(winExpr.getSourceLocation());
newWinExpr.addHints(winExpr.getHints());
return new Pair<>(newWinExpr, env);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
index bdf5c9d..975379d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
@@ -123,18 +123,24 @@
@Override
public Void visit(Projection projection, Integer step) throws CompilationException {
- if (projection.star()) {
- out.print(" * ");
- return null;
- }
- projection.getExpression().accept(this, step);
- if (projection.varStar()) {
- out.print(".* ");
- } else {
- String name = projection.getName();
- if (name != null) {
- out.print(" as " + name);
- }
+ switch (projection.getKind()) {
+ case STAR:
+ out.print(" * ");
+ break;
+ case EVERY_VAR_STAR:
+ out.print(" *.* ");
+ break;
+ case VAR_STAR:
+ projection.getExpression().accept(this, step);
+ out.print(".* ");
+ break;
+ case NAMED_EXPR:
+ projection.getExpression().accept(this, step);
+ String name = projection.getName();
+ if (name != null) {
+ out.print(" as " + name);
+ }
+ break;
}
return null;
}
@@ -364,7 +370,8 @@
}
if (windowExpr.hasOrderByList()) {
out.print(skip(step + 1) + "order by ");
- printDelimitedObyExpressions(windowExpr.getOrderbyList(), windowExpr.getOrderbyModifierList(), step + 2);
+ printDelimitedObyExpressions(windowExpr.getOrderbyList(), windowExpr.getOrderbyModifierList(),
+ windowExpr.getOrderbyNullModifierList(), step + 2);
out.println();
}
if (windowExpr.hasFrameDefinition()) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSynonymRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSynonymRewriteVisitor.java
index a0ffdf2..b47eb6e 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSynonymRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppSynonymRewriteVisitor.java
@@ -44,7 +44,7 @@
@Override
public Void visit(LoadStatement loadStmt, MetadataProvider metadataProvider) throws CompilationException {
Triple<DataverseName, String, Boolean> dsName = resolveDatasetNameUsingSynonyms(metadataProvider,
- loadStmt.getDataverseName(), loadStmt.getDatasetName(), loadStmt.getSourceLocation());
+ loadStmt.getDataverseName(), loadStmt.getDatasetName(), false, loadStmt.getSourceLocation());
if (dsName != null) {
loadStmt.setDataverseName(dsName.first);
loadStmt.setDatasetName(dsName.second);
@@ -55,7 +55,7 @@
@Override
public Void visit(InsertStatement insertStmt, MetadataProvider metadataProvider) throws CompilationException {
Triple<DataverseName, String, Boolean> dsName = resolveDatasetNameUsingSynonyms(metadataProvider,
- insertStmt.getDataverseName(), insertStmt.getDatasetName(), insertStmt.getSourceLocation());
+ insertStmt.getDataverseName(), insertStmt.getDatasetName(), false, insertStmt.getSourceLocation());
if (dsName != null) {
insertStmt.setDataverseName(dsName.first);
insertStmt.setDatasetName(dsName.second);
@@ -66,7 +66,7 @@
@Override
public Void visit(DeleteStatement deleteStmt, MetadataProvider metadataProvider) throws CompilationException {
Triple<DataverseName, String, Boolean> dsName = resolveDatasetNameUsingSynonyms(metadataProvider,
- deleteStmt.getDataverseName(), deleteStmt.getDatasetName(), deleteStmt.getSourceLocation());
+ deleteStmt.getDataverseName(), deleteStmt.getDatasetName(), false, deleteStmt.getSourceLocation());
if (dsName != null) {
deleteStmt.setDataverseName(dsName.first);
deleteStmt.setDatasetName(dsName.second);
@@ -75,9 +75,10 @@
}
private Triple<DataverseName, String, Boolean> resolveDatasetNameUsingSynonyms(MetadataProvider metadataProvider,
- DataverseName dataverseName, String datasetName, SourceLocation sourceLoc) throws CompilationException {
+ DataverseName dataverseName, String datasetName, boolean includingViews, SourceLocation sourceLoc)
+ throws CompilationException {
try {
- return metadataProvider.resolveDatasetNameUsingSynonyms(dataverseName, datasetName);
+ return metadataProvider.resolveDatasetNameUsingSynonyms(dataverseName, datasetName, includingViews);
} catch (AlgebricksException e) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
index e331173..ef8b43c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
@@ -113,7 +113,7 @@
@Override
public Expression visit(Projection projection, ILangExpression arg) throws CompilationException {
- if (!projection.star()) {
+ if (projection.hasExpression()) {
projection.setExpression(visit(projection.getExpression(), arg));
}
return null;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index c4c86f4..4e64944 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -63,6 +63,7 @@
import org.apache.asterix.common.annotations.RangeAnnotation;
import org.apache.asterix.common.annotations.SecondaryIndexSearchPreferenceAnnotation;
import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
import org.apache.asterix.common.annotations.TypeDataGen;
import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -74,6 +75,7 @@
import org.apache.asterix.common.functions.FunctionConstants;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.AbstractLangExpression;
import org.apache.asterix.lang.common.base.AbstractStatement;
@@ -135,6 +137,7 @@
import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateViewStatement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -162,11 +165,15 @@
import org.apache.asterix.lang.common.statement.TypeDropStatement;
import org.apache.asterix.lang.common.statement.UpdateStatement;
import org.apache.asterix.lang.common.statement.UpsertStatement;
+import org.apache.asterix.lang.common.statement.ViewDecl;
+import org.apache.asterix.lang.common.statement.ViewDropStatement;
import org.apache.asterix.lang.common.statement.WriteStatement;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.struct.OperatorType;
import org.apache.asterix.lang.common.struct.QuantifiedPair;
import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.DatasetDeclParametersUtil;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
import org.apache.asterix.lang.common.util.RangeMapBuilder;
import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
import org.apache.asterix.lang.sqlpp.clause.FromClause;
@@ -199,6 +206,7 @@
import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
+import org.apache.asterix.om.exceptions.TypeMismatchException;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.commons.lang3.ArrayUtils;
@@ -221,9 +229,12 @@
// tokens parsed as identifiers
private static final String CUBE = "CUBE";
private static final String CURRENT = "CURRENT";
+ private static final String DEFAULT = "DEFAULT";
private static final String EXCLUDE = "EXCLUDE";
+ private static final String INCLUDE = "INCLUDE";
private static final String FIRST = "FIRST";
private static final String FOLLOWING = "FOLLOWING";
+ private static final String FOREIGN = "FOREIGN";
private static final String GROUPING = "GROUPING";
private static final String GROUPS = "GROUPS";
private static final String IGNORE = "IGNORE";
@@ -235,6 +246,7 @@
private static final String PARTITION = "PARTITION";
private static final String PRECEDING = "PRECEDING";
private static final String RANGE = "RANGE";
+ private static final String REFERENCES = "REFERENCES";
private static final String RESPECT = "RESPECT";
private static final String ROLLUP = "ROLLUP";
private static final String ROW = "ROW";
@@ -273,7 +285,7 @@
this.gramLength = gramLength;
this.fullTextConfig = fullTextConfig;
}
- };
+ }
private static class FunctionName {
public DataverseName dataverse;
@@ -338,7 +350,7 @@
super.setInput(s);
}
- public static void main(String args[]) throws ParseException, TokenMgrError, IOException, FileNotFoundException, CompilationException {
+ public static void main(String[] args) throws ParseException, TokenMgrError, IOException, FileNotFoundException, CompilationException {
File file = new File(args[0]);
Reader fis = new BufferedReader(new InputStreamReader(new FileInputStream(file), "UTF-8"));
SQLPPParser parser = new SQLPPParser(fis);
@@ -393,6 +405,19 @@
return new SQLPPParser(text).parseParenthesizedIdentifierList();
}
+ private List<Literal> parseParenthesizedLiteralList() throws CompilationException {
+ return parseImpl(new ParseFunction<List<Literal>>() {
+ @Override
+ public List<Literal> parse() throws ParseException {
+ return SQLPPParser.this.ParenthesizedLiteralList();
+ }
+ });
+ }
+
+ private static List<Literal> parseParenthesizedLiteralList(String text) throws CompilationException {
+ return new SQLPPParser(text).parseParenthesizedLiteralList();
+ }
+
@Override
public FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames, boolean isStored)
throws CompilationException {
@@ -414,6 +439,22 @@
});
}
+ @Override
+ public ViewDecl parseViewBody(DatasetFullyQualifiedName viewName) throws CompilationException {
+ return parseImpl(new ParseFunction<ViewDecl>() {
+ @Override
+ public ViewDecl parse() throws ParseException {
+ DataverseName dataverse = defaultDataverse;
+ defaultDataverse = viewName.getDataverseName();
+ createNewScope();
+ Expression viewBodyExpr = SQLPPParser.this.ViewBody();
+ removeCurrentScope();
+ defaultDataverse = dataverse;
+ return new ViewDecl(viewName, viewBodyExpr);
+ }
+ });
+ }
+
private <T> T parseImpl(ParseFunction<T> parseFunction) throws CompilationException {
warningCollector.clear();
hintCollector.clear();
@@ -631,6 +672,47 @@
List<String> indexNames = parseParenthesizedIdentifierList(hintToken.hintParams);
return SkipSecondaryIndexSearchExpressionAnnotation.newInstance(indexNames);
}
+ case SPATIAL_JOIN_HINT:
+ List<Literal> hintValues = parseParenthesizedLiteralList(hintToken.hintParams);
+
+ // Handle exceptions
+ if (hintValues.size() != 6) {
+ throw new SqlppParseException(getSourceLocation(hintToken), String.format("Unexpected hint: %s. 6 arguments are required.",
+ hintToken.hint.toString()));
+ }
+
+ for (int i = 0; i < 4; i++) {
+ Literal lit = hintValues.get(i);
+ if ((lit.getLiteralType() != Literal.Type.DOUBLE)
+ && (lit.getLiteralType() != Literal.Type.FLOAT)
+ && (lit.getLiteralType() != Literal.Type.LONG)
+ && (lit.getLiteralType() != Literal.Type.INTEGER)){
+ throw new SqlppParseException(getSourceLocation(hintToken), String.format("Unexpected hint: %s. Numeric value is required for first 4 arguments.",
+ hintToken.hint.toString()));
+ }
+ }
+
+ for (int i = 4; i < 6; i++) {
+ Literal lit = hintValues.get(i);
+ if ((lit.getLiteralType() != Literal.Type.LONG)
+ && (lit.getLiteralType() != Literal.Type.INTEGER)) {
+ throw new SqlppParseException(getSourceLocation(hintToken), String.format("Unexpected hint: %s. Long/int is required for last 2 arguments.",
+ hintToken.hint.toString()));
+ }
+ }
+
+ try {
+ double minX = ExpressionUtils.getDoubleValue(hintValues.get(0));
+ double minY = ExpressionUtils.getDoubleValue(hintValues.get(1));
+ double maxX = ExpressionUtils.getDoubleValue(hintValues.get(2));
+ double maxY = ExpressionUtils.getDoubleValue(hintValues.get(3));
+ int numRows = (int) ExpressionUtils.getLongValue(hintValues.get(4));
+ int numColumns = (int) ExpressionUtils.getLongValue(hintValues.get(5));
+ SpatialJoinAnnotation spatialJoinAnn = new SpatialJoinAnnotation(minX, minY, maxX, maxY, numRows, numColumns);
+ return spatialJoinAnn;
+ } catch (TypeMismatchException e) {
+ throw new SqlppParseException(getSourceLocation(hintToken), e.getMessage());
+ }
case USE_SECONDARY_INDEX_SEARCH_HINT:
if (hintToken.hintParams == null) {
throw new SqlppParseException(getSourceLocation(hintToken), "Expected index name(s)");
@@ -796,6 +878,7 @@
| stmt = CreateFeedStatement(startToken)
| stmt = CreateFeedPolicyStatement(startToken)
| stmt = CreateFullTextStatement(startToken)
+ | stmt = CreateViewStatement(startToken, false)
)
{
return stmt;
@@ -811,6 +894,7 @@
<OR> <IDENTIFIER> { replaceToken = token; }
(
stmt = CreateFunctionStatement(startStmtToken, true)
+ | stmt = CreateViewStatement(startStmtToken, true)
)
{
// check expected token here to make the grammar extension plugin happy
@@ -955,6 +1039,11 @@
try {
InternalDetailsDecl idd = new InternalDetailsDecl(primaryKeyFields.second, primaryKeyFields.first, autogenerated,
filterField == null? null : filterField.first, filterField == null? null : filterField.second);
+ DatasetDeclParametersUtil.adjustInlineTypeDecl(typeExpr, primaryKeyFields.second, primaryKeyFields.first, false);
+ if (metaTypeExpr != null) {
+ DatasetDeclParametersUtil.adjustInlineTypeDecl(metaTypeExpr, primaryKeyFields.second, primaryKeyFields.first,
+ true);
+ }
stmt = new DatasetDecl(nameComponents.first, nameComponents.second, typeExpr, metaTypeExpr, hints,
DatasetType.INTERNAL, idd, withRecord, ifNotExists);
return addSourceLocation(stmt, startStmtToken);
@@ -1021,7 +1110,7 @@
}
}
-TypeExpression DatasetRecordTypeSpecification(boolean allowRecordKindModifier) throws ParseException:
+RecordTypeDefinition DatasetRecordTypeSpecification(boolean allowRecordKindModifier) throws ParseException:
{
RecordTypeDefinition recordTypeDef = null;
RecordTypeDefinition.RecordKind recordKind = null;
@@ -1108,6 +1197,10 @@
boolean hasUnnest = false;
String fullTextConfigName = null;
Token startElementToken = null;
+ Boolean excludeUnknown = null;
+ Pair<Map<String, String>, Boolean> castConfigDefaultNull;
+ Boolean castDefaultNull = null;
+ Map<String, String> castConfig = null;
}
{
(
@@ -1126,6 +1219,24 @@
)*
<RIGHTPAREN>
( <TYPE> indexParams = IndexType() )? ( <ENFORCED> { enforced = true; } )?
+ ( LOOKAHEAD({laIdentifier(EXCLUDE) || laIdentifier(INCLUDE)}) <IDENTIFIER>
+ {
+ if (isToken(EXCLUDE)) {
+ excludeUnknown = true;
+ } else if (isToken(INCLUDE)) {
+ excludeUnknown = false;
+ } else {
+ throw createUnexpectedTokenError();
+ }
+ } <UNKNOWN> <KEY>
+ )?
+
+ ( <CAST><LEFTPAREN> castConfigDefaultNull = CastDefaultNull() <RIGHTPAREN>
+ {
+ castConfig = castConfigDefaultNull.first;
+ castDefaultNull = castConfigDefaultNull.second;
+ }
+ )?
)
{
IndexType indexType;
@@ -1140,7 +1251,8 @@
fullTextConfigName = null;
}
CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
- new Identifier(indexName), indexType, indexedElementList, enforced, gramLength, fullTextConfigName, ifNotExists);
+ new Identifier(indexName), indexType, indexedElementList, enforced, gramLength, fullTextConfigName, ifNotExists,
+ excludeUnknown, castDefaultNull, castConfig);
return addSourceLocation(stmt, startStmtToken);
}
}
@@ -1269,7 +1381,8 @@
indexName = "primary_idx_" + nameComponents.second;
}
CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
- new Identifier(indexName), IndexType.BTREE, Collections.emptyList(), false, -1, null, ifNotExists);
+ new Identifier(indexName), IndexType.BTREE, Collections.emptyList(), false, -1, null, ifNotExists, null, null,
+ Collections.emptyMap());
return addSourceLocation(stmt, startStmtToken);
}
}
@@ -1381,6 +1494,132 @@
}
}
+CreateViewStatement CreateViewStatement(Token startStmtToken, boolean orReplace) throws ParseException:
+{
+ CreateViewStatement stmt = null;
+}
+{
+ <VIEW> stmt = ViewSpecification(startStmtToken, orReplace)
+ {
+ return stmt;
+ }
+}
+
+CreateViewStatement ViewSpecification(Token startStmtToken, boolean orReplace) throws ParseException:
+{
+ Pair<DataverseName, Identifier> nameComponents = null;
+ TypeExpression typeExpr = null;
+ boolean ifNotExists = false;
+ Token beginPos = null, endPos = null;
+ Expression viewBodyExpr = null;
+ Pair<Map<String, String>, Boolean> viewConfigDefaultNull;
+ Boolean defaultNull = null;
+ Map<String, String> viewConfig = null;
+ Pair<List<Integer>, List<List<String>>> primaryKeyFields = null;
+ Pair<List<Integer>, List<List<String>>> foreignKeyFields = null;
+ Pair<DataverseName, Identifier> refNameComponents = null;
+ List<CreateViewStatement.ForeignKeyDecl> foreignKeyDecls = null;
+ DataverseName currentDataverse = defaultDataverse;
+}
+{
+ nameComponents = QualifiedName()
+ (
+ (
+ typeExpr = DatasetTypeSpecification()
+ ifNotExists = IfNotExists()
+ viewConfigDefaultNull = CastDefaultNull()
+ {
+ viewConfig = viewConfigDefaultNull.first;
+ defaultNull = viewConfigDefaultNull.second;
+ }
+ (
+ <PRIMARY> <KEY> <LEFTPAREN> primaryKeyFields = PrimaryKeyFields() <RIGHTPAREN>
+ <NOT> <ENFORCED>
+ )?
+ (
+ <IDENTIFIER> { expectToken(FOREIGN); } <KEY> <LEFTPAREN> foreignKeyFields = PrimaryKeyFields() <RIGHTPAREN>
+ <IDENTIFIER> { expectToken(REFERENCES); } refNameComponents = QualifiedName()
+ <NOT> <ENFORCED>
+ {
+ if (foreignKeyDecls == null) {
+ foreignKeyDecls = new ArrayList<CreateViewStatement.ForeignKeyDecl>();
+ }
+ foreignKeyDecls.add(new CreateViewStatement.ForeignKeyDecl(foreignKeyFields.second,
+ foreignKeyFields.first, refNameComponents.first, refNameComponents.second));
+ }
+ )*
+ )
+ |
+ ( ifNotExists = IfNotExists() )
+ )
+ {
+ if (orReplace && ifNotExists) {
+ throw new SqlppParseException(getSourceLocation(startStmtToken), "Unexpected IF NOT EXISTS");
+ }
+ }
+ <AS>
+ {
+ beginPos = token;
+ createNewScope();
+ if (nameComponents.first != null) {
+ defaultDataverse = nameComponents.first;
+ }
+ }
+ viewBodyExpr = ViewBody()
+ {
+ endPos = token;
+ String viewBody = extractFragment(beginPos.beginLine, beginPos.beginColumn + 1, endPos.endLine,
+ endPos.endColumn + 1);
+ removeCurrentScope();
+ defaultDataverse = currentDataverse;
+ if (typeExpr != null && primaryKeyFields != null) {
+ DatasetDeclParametersUtil.adjustInlineTypeDecl(typeExpr, primaryKeyFields.second, primaryKeyFields.first, false);
+ }
+ CreateViewStatement.KeyDecl primaryKeyDecl = primaryKeyFields != null ?
+ new CreateViewStatement.KeyDecl(primaryKeyFields.second, primaryKeyFields.first) : null;
+ CreateViewStatement stmt = new CreateViewStatement(nameComponents.first, nameComponents.second.getValue(),
+ typeExpr, viewBody, viewBodyExpr, defaultNull, viewConfig, primaryKeyDecl, foreignKeyDecls, orReplace,
+ ifNotExists);
+ return addSourceLocation(stmt, startStmtToken);
+ }
+}
+
+Expression ViewBody() throws ParseException:
+{
+ Expression viewBodyExpr = null;
+}
+{
+ (
+ ( viewBodyExpr = VariableRef() ( viewBodyExpr = FieldAccessor(viewBodyExpr) )* )
+ | viewBodyExpr = SelectExpression(true)
+ )
+ {
+ return viewBodyExpr;
+ }
+}
+
+Pair<Map<String, String>, Boolean> CastDefaultNull() throws ParseException:
+{
+ Map<String, String> castConfig = null;
+ Boolean defaultNull = null;
+ String propertyName = null, propertyValue = null;
+}
+{
+ <IDENTIFIER> { expectToken(DEFAULT); } <NULL> { defaultNull = true; }
+ (
+ LOOKAHEAD(2) <IDENTIFIER> { propertyName = token.image.toLowerCase(); } propertyValue = StringLiteral()
+ {
+ if (castConfig == null) {
+ castConfig = new HashMap<String, String>();
+ }
+ castConfig.put(propertyName, propertyValue);
+ }
+ )*
+ {
+ return new Pair<Map<String, String>, Boolean>(castConfig, defaultNull);
+ }
+}
+
CreateFunctionStatement CreateFunctionStatement(Token startStmtToken, boolean orReplace) throws ParseException:
{
CreateFunctionStatement stmt = null;
@@ -1421,6 +1660,11 @@
signature = new FunctionSignature(fctName.dataverse, fctName.function, arity);
}
ifNotExists = IfNotExists()
+ {
+ if (orReplace && ifNotExists) {
+ throw new SqlppParseException(getSourceLocation(token), "Unexpected IF NOT EXISTS");
+ }
+ }
returnType = FunctionReturnType()
(
(
@@ -1457,9 +1701,6 @@
)
)
{
- if (orReplace && ifNotExists) {
- throw new SqlppParseException(getSourceLocation(startStmtToken), "Unexpected IF NOT EXISTS");
- }
defaultDataverse = currentDataverse;
return addSourceLocation(stmt, startStmtToken);
}
@@ -1812,12 +2053,23 @@
Pair<List<Integer>, List<List<String>>> PrimaryKey() throws ParseException:
{
+ Pair<List<Integer>, List<List<String>>> primaryKeyFields = null;
+}
+{
+ <PRIMARY> <KEY> primaryKeyFields = PrimaryKeyFields()
+ {
+ return primaryKeyFields;
+ }
+}
+
+Pair<List<Integer>, List<List<String>>> PrimaryKeyFields() throws ParseException:
+{
Pair<Integer, List<String>> tmp = null;
List<Integer> keyFieldSourceIndicators = new ArrayList<Integer>();
List<List<String>> primaryKeyFields = new ArrayList<List<String>>();
}
{
- <PRIMARY> <KEY> tmp = NestedField()
+ tmp = NestedField()
{
keyFieldSourceIndicators.add(tmp.first);
primaryKeyFields.add(tmp.second);
@@ -1852,6 +2104,7 @@
| stmt = DropFeedPolicyStatement(startToken)
| stmt = DropSynonymStatement(startToken)
| stmt = DropFullTextStatement(startToken)
+ | stmt = DropViewStatement(startToken)
)
{
return stmt;
@@ -1882,6 +2135,30 @@
}
}
+ViewDropStatement DropViewStatement(Token startStmtToken) throws ParseException:
+{
+ ViewDropStatement stmt = null;
+}
+{
+ <VIEW> stmt = DropViewSpecification(startStmtToken)
+ {
+ return stmt;
+ }
+}
+
+ViewDropStatement DropViewSpecification(Token startStmtToken) throws ParseException:
+{
+ Pair<DataverseName,Identifier> pairId = null;
+ boolean ifExists = false;
+}
+{
+ pairId = QualifiedName() ifExists = IfExists()
+ {
+ ViewDropStatement stmt = new ViewDropStatement(pairId.first, pairId.second, ifExists);
+ return addSourceLocation(stmt, startStmtToken);
+ }
+}
+
IndexDropStatement DropIndexStatement(Token startStmtToken) throws ParseException:
{
IndexDropStatement stmt = null;
@@ -2717,8 +2994,8 @@
// Note: there's a copy of this production in PrimaryExpr() (LOOKAHEAD for FunctionCallExpr())
// that copy must be kept in sync with this code
prefix = MultipartIdentifierWithHints(
- SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT,
- SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
+ SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
+ SqlppHint.SPATIAL_JOIN_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
)
(<SHARP> suffix = Identifier())?
{
@@ -2797,6 +3074,49 @@
}
}
+List<Literal> ParenthesizedLiteralList() throws ParseException:
+{
+ List<Literal> list = new ArrayList<Literal>();
+ boolean minus = false;
+ Expression litExpr = null;
+ Literal lit = null;
+}
+{
+ <LEFTPAREN>
+ (<MINUS> { minus = true; })? litExpr = Literal()
+ {
+ lit = ((LiteralExpr) litExpr).getValue();
+ if (minus)
+ {
+ try {
+ lit = ExpressionUtils.reverseSign(lit);
+ } catch (TypeMismatchException e) {
+ throw new SqlppParseException(getSourceLocation(token), e.getMessage());
+ }
+ minus = false;
+ }
+ list.add(lit);
+ }
+ ( <COMMA> (<MINUS> { minus = true; })? litExpr = Literal()
+ {
+ lit = ((LiteralExpr) litExpr).getValue();
+ if (minus)
+ {
+ try {
+ lit = ExpressionUtils.reverseSign(lit);
+ } catch (TypeMismatchException e) {
+ throw new SqlppParseException(getSourceLocation(token), e.getMessage());
+ }
+ minus = false;
+ }
+ list.add(lit);
+ } )*
+ <RIGHTPAREN>
+ {
+ return list;
+ }
+}
+
Pair<Integer, Pair<List<String>, IndexedTypeExpression>> OpenField() throws ParseException:
{
IndexedTypeExpression fieldType = null;
@@ -2874,12 +3194,48 @@
String StringLiteral() throws ParseException:
{
+ StringBuilder str = null;
+ char quote = 0;
+ boolean ext = false;
+ Token litToken = null;
+ String lit = null;
}
{
- <STRING_LITERAL>
+ ( <STRING_LITERAL>
{
- return removeQuotesAndEscapes(token.image);
+ String quoted = token.image;
+ char q = quoted.charAt(0);
+ boolean e = q == 'E';
+ if (e) {
+ q = quoted.charAt(1);
+ quoted = quoted.substring(1);
+ }
+ if (lit == null) {
+ quote = q;
+ ext = e;
+ } else {
+ boolean isAdjacent = litToken.endLine == token.beginLine && litToken.endColumn + 1 == token.beginColumn;
+ if (!isAdjacent || ext || e || (q != quote)) {
+ throw new SqlppParseException(getSourceLocation(token), "Invalid string literal");
+ }
+ if (str == null) {
+ str = new StringBuilder();
+ }
+ str.append(lit);
+ str.append(quote);
+ }
+ lit = removeQuotesAndEscapes(quoted);
+ litToken = token;
}
+ )+
+ {
+ if (str == null) {
+ return lit;
+ } else {
+ str.append(lit);
+ return str.toString();
+ }
+ }
}
Triple<List<String>, Token, Token> MultipartIdentifier() throws ParseException:
@@ -3122,6 +3478,7 @@
{
boolean not = false;
OperatorExpr op = null;
+ Token opToken = null;
Expression operand = null;
IExpressionAnnotation annotation = null;
}
@@ -3129,8 +3486,12 @@
operand = BetweenExpr()
(
- LOOKAHEAD(2)( <LT> | <GT> | <LE> | <GE> | <EQ> | <NE> | <LG> |<SIMILAR> | (<NOT> { not = true; })? <IN>)
+ LOOKAHEAD(3)( <LT> | <GT> | <LE> | <GE> | <EQ> | <NE> | <LG> |<SIMILAR> | (<NOT> { not = true; })? <IN> |
+ <IS> (<NOT> { not = true; })? <DISTINCT> { opToken = token; } <FROM> )
{
+ if (opToken == null) {
+ opToken = token;
+ }
Token hintToken = fetchHint(token,
SqlppHint.HASH_BROADCAST_JOIN_HINT, SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT,
SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
@@ -3138,7 +3499,7 @@
if (hintToken != null) {
annotation = parseExpressionAnnotation(hintToken);
}
- String operator = token.image.toLowerCase();
+ String operator = opToken.image.toLowerCase();
if (operator.equals("<>")){
operator = "!=";
}
@@ -3240,7 +3601,9 @@
}
{
operand = LikeExpr()
- ( <IS>
+ (
+ LOOKAHEAD(3)
+ <IS>
(<NOT> { not = true; notToken = token; })?
(
<NULL> { fn = BuiltinFunctions.IS_NULL; } |
@@ -3876,6 +4239,7 @@
WindowExpression WindowExpr(FunctionSignature signature, List<Expression> argList, Expression aggFilterExpr)
throws ParseException:
{
+ WindowExpression windowExpr = null;
Boolean fromLast = null, ignoreNulls = null;
}
{
@@ -3913,9 +4277,9 @@
}
}
)?
- <OVER>
+ <OVER> windowExpr = OverClause(signature, argList, aggFilterExpr, token, fromLast, ignoreNulls)
{
- return OverClause(signature, argList, aggFilterExpr, token, fromLast, ignoreNulls);
+ return windowExpr;
}
}
@@ -3927,6 +4291,7 @@
OrderbyClause orderByClause = null;
List<Expression> orderbyList = null;
List<OrderbyClause.OrderModifier> orderbyModifierList = null;
+ List<OrderbyClause.NullOrderModifier> orderbyNullModifierList = null;
WindowExpression.FrameMode frameMode = null;
Pair<WindowExpression.FrameBoundaryKind, Expression> frameStart = null, frameEnd = null;
WindowExpression.FrameBoundaryKind frameStartKind = null, frameEndKind = null;
@@ -3955,6 +4320,7 @@
{
orderbyList = orderByClause.getOrderbyList();
orderbyModifierList = orderByClause.getModifierList();
+ orderbyNullModifierList = orderByClause.getNullModifierList();
}
(
frameMode = WindowFrameMode()
@@ -3981,8 +4347,8 @@
<RIGHTPAREN>
{
WindowExpression winExpr = new WindowExpression(signature, argList, aggFilterExpr, partitionExprs, orderbyList,
- orderbyModifierList, frameMode, frameStartKind, frameStartExpr, frameEndKind, frameEndExpr, frameExclusionKind,
- windowVar, windowFieldList, ignoreNulls, fromLast);
+ orderbyModifierList, orderbyNullModifierList, frameMode, frameStartKind, frameStartExpr, frameEndKind,
+ frameEndExpr, frameExclusionKind, windowVar, windowFieldList, ignoreNulls, fromLast);
return addSourceLocation(winExpr, startToken);
}
}
@@ -4079,17 +4445,35 @@
Expression ParenthesizedExpression() throws ParseException:
{
- Expression expr;
+ Token startToken = null;
+ Expression expr = null, expr2 = null;
+ List<Expression> exprList = null;
}
{
(
LOOKAHEAD(2)
- <LEFTPAREN> expr = Expression() <RIGHTPAREN>
+ <LEFTPAREN> { startToken = token; } expr = Expression()
+ (
+ <COMMA> expr2 = Expression()
+ {
+ if (exprList == null) {
+ exprList = new ArrayList<Expression>();
+ exprList.add(expr);
+ }
+ exprList.add(expr2);
+ }
+ )*
+ <RIGHTPAREN>
|
expr = Subquery()
)
{
- return expr;
+ if (exprList != null) {
+ ListConstructor listExpr = new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, exprList);
+ return addSourceLocation(listExpr, startToken);
+ } else {
+ return expr;
+ }
}
}
@@ -4210,6 +4594,7 @@
}
{
(
+ (
selectClause = SelectClause() { startSrcLoc = selectClause.getSourceLocation(); }
(
(
@@ -4217,6 +4602,14 @@
(
fromLetClauses = LetClause()
)?
+ (whereClause = WhereClause())?
+ (
+ groupbyClause = GroupbyClause()
+ (
+ gbyLetClauses = LetClause()
+ )?
+ (havingClause = HavingClause())?
+ )?
)
|
(
@@ -4236,17 +4629,12 @@
fromTerms.add(new FromTerm(listExpr, fromVar, null, new ArrayList<AbstractBinaryCorrelateClause>()));
fromClause = new FromClause(fromTerms);
}
+ (whereClause = WhereClause())?
)
)?
- (whereClause = WhereClause())?
- (
- groupbyClause = GroupbyClause()
- (
- gbyLetClauses = LetClause()
- )?
- (havingClause = HavingClause())?
- )?
+ )
|
+ (
fromClause = FromClause() { startSrcLoc = fromClause.getSourceLocation(); }
(
fromLetClauses = LetClause()
@@ -4260,6 +4648,7 @@
(havingClause = HavingClause())?
)?
selectClause = SelectClause()
+ )
)
{
if (fromLetClauses != null) {
@@ -4298,7 +4687,7 @@
{
SourceLocation sourceLoc = getSourceLocation(startToken);
if (selectRegular == null && selectElement == null){
- Projection projection = new Projection(null, null, true, false);
+ Projection projection = new Projection(Projection.Kind.STAR, null, null);
projection.setSourceLocation(sourceLoc);
List<Projection> projections = new ArrayList<Projection>();
projections.add(projection);
@@ -4355,15 +4744,17 @@
Expression expr = null;
Identifier identifier = null;
String name = null;
+ Projection.Kind kind = null;
boolean star = false;
boolean varStar = false;
}
{
(
- <MUL> { star = true; startSrcLoc = getSourceLocation(token); }
- | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> { varStar = true; }
+ <MUL> { kind = Projection.Kind.STAR; startSrcLoc = getSourceLocation(token); }
+ | LOOKAHEAD(3) expr = VariableRef() <DOT> <MUL> { kind = Projection.Kind.VAR_STAR; }
| expr = Expression() ((<AS>)? name = Identifier())?
{
+ kind = Projection.Kind.NAMED_EXPR;
if (name == null) {
String generatedColumnIdentifier = ExpressionToVariableUtil.getGeneratedIdentifier(expr, false);
if (generatedColumnIdentifier != null) {
@@ -4373,7 +4764,7 @@
}
)
{
- Projection projection = new Projection(expr, name, star, varStar);
+ Projection projection = new Projection(kind, expr, name);
projection.setSourceLocation(expr != null ? expr.getSourceLocation() : startSrcLoc);
return projection;
}
@@ -4413,6 +4804,7 @@
| ( <INNER> correlateClause = JoinOrUnnestClause(JoinType.INNER, UnnestType.INNER) )
| ( <LEFT> ( <OUTER> )? correlateClause = JoinOrUnnestClause(JoinType.LEFTOUTER, UnnestType.LEFTOUTER) )
| ( <RIGHT> ( <OUTER> )? correlateClause = JoinClause(JoinType.RIGHTOUTER) )
+ | ( <CROSS> correlateClause = CrossJoinClause() )
)
{
correlateClauses.add(correlateClause);
@@ -4442,19 +4834,46 @@
JoinClause JoinClause(JoinType joinType) throws ParseException :
{
Token startToken = null;
- Expression rightExpr = null;
- VariableExpr rightVar = null;
- VariableExpr posVar = null;
+ Triple<Expression, VariableExpr, VariableExpr> rightInput = null;
Expression conditionExpr = null;
}
{
- <JOIN> { startToken = token; } rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())? <ON> conditionExpr = Expression()
+ <JOIN> { startToken = token; } rightInput = JoinClauseRightInput() <ON> conditionExpr = Expression()
+ {
+ JoinClause joinClause = new JoinClause(joinType, rightInput.first, rightInput.second, rightInput.third,
+ conditionExpr, joinType == JoinType.INNER ? null : Literal.Type.MISSING);
+ return addSourceLocation(joinClause, startToken);
+ }
+}
+
+JoinClause CrossJoinClause() throws ParseException :
+{
+ Token startToken = null;
+ Triple<Expression, VariableExpr, VariableExpr> rightInput = null;
+ Expression conditionExpr = null;
+}
+{
+ <JOIN> { startToken = token; } rightInput = JoinClauseRightInput()
+ {
+ JoinClause joinClause = new JoinClause(JoinType.INNER, rightInput.first, rightInput.second, rightInput.third,
+ new LiteralExpr(TrueLiteral.INSTANCE), null);
+ return addSourceLocation(joinClause, startToken);
+ }
+}
+
+Triple<Expression, VariableExpr, VariableExpr> JoinClauseRightInput() throws ParseException :
+{
+ Expression rightExpr = null;
+ VariableExpr rightVar = null;
+ VariableExpr posVar = null;
+}
+{
+ rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())?
{
if (rightVar == null) {
rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
}
- JoinClause joinClause = new JoinClause(joinType, rightExpr, rightVar, posVar, conditionExpr);
- return addSourceLocation(joinClause, startToken);
+ return new Triple<Expression, VariableExpr, VariableExpr>(rightExpr, rightVar, posVar);
}
}
@@ -4471,7 +4890,8 @@
if (rightVar == null) {
rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
}
- UnnestClause unnestClause = new UnnestClause(unnestType, rightExpr, rightVar, posVar);
+ UnnestClause unnestClause = new UnnestClause(unnestType, rightExpr, rightVar, posVar,
+ unnestType == UnnestType.INNER ? null : Literal.Type.MISSING);
return addSourceLocation(unnestClause, startToken);
}
}
@@ -4509,65 +4929,89 @@
{
Token startToken = null;
OrderbyClause oc = new OrderbyClause();
- Expression orderbyExpr;
+ Triple<Expression, OrderbyClause.OrderModifier, OrderbyClause.NullOrderModifier> orderbyExpr = null;
List<Expression> orderbyList = new ArrayList<Expression>();
List<OrderbyClause.OrderModifier> modifierList = new ArrayList<OrderbyClause.OrderModifier>();
- int numOfOrderby = 0;
+ List<OrderbyClause.NullOrderModifier> nullModifierList = new ArrayList<OrderbyClause.NullOrderModifier>();
}
{
- <ORDER>
- {
- startToken = token;
- Token hintToken = fetchHint(token, SqlppHint.INMEMORY_HINT, SqlppHint.RANGE_HINT);
- if (hintToken != null) {
- switch (hintToken.hint) {
- case INMEMORY_HINT:
- String[] splits = hintToken.hintParams.split("\\s+");
- int numFrames = Integer.parseInt(splits[0]);
- int numTuples = Integer.parseInt(splits[1]);
- oc.setNumFrames(numFrames);
- oc.setNumTuples(numTuples);
- break;
- case RANGE_HINT:
- try {
- Expression rangeExpr = parseExpression(hintToken.hintParams);
- RangeMap rangeMap = RangeMapBuilder.parseHint(rangeExpr);
- oc.setRangeMap(rangeMap);
- } catch (CompilationException e) {
- throw new SqlppParseException(getSourceLocation(hintToken), e.getMessage());
- }
- break;
+ <ORDER>
+ {
+ startToken = token;
+ Token hintToken = fetchHint(token, SqlppHint.INMEMORY_HINT, SqlppHint.RANGE_HINT);
+ if (hintToken != null) {
+ switch (hintToken.hint) {
+ case INMEMORY_HINT:
+ String[] splits = hintToken.hintParams.split("\\s+");
+ int numFrames = Integer.parseInt(splits[0]);
+ int numTuples = Integer.parseInt(splits[1]);
+ oc.setNumFrames(numFrames);
+ oc.setNumTuples(numTuples);
+ break;
+ case RANGE_HINT:
+ try {
+ Expression rangeExpr = parseExpression(hintToken.hintParams);
+ RangeMap rangeMap = RangeMapBuilder.parseHint(rangeExpr);
+ oc.setRangeMap(rangeMap);
+ } catch (CompilationException e) {
+ throw new SqlppParseException(getSourceLocation(hintToken), e.getMessage());
}
- }
+ break;
}
- <BY> orderbyExpr = Expression()
- {
- orderbyList.add(orderbyExpr);
- OrderbyClause.OrderModifier modif = OrderbyClause.OrderModifier.ASC;
}
- ( (<ASC> { modif = OrderbyClause.OrderModifier.ASC; })
- | (<DESC> { modif = OrderbyClause.OrderModifier.DESC; }))?
+ }
+ <BY> orderbyExpr = OrderByExpression()
+ {
+ orderbyList.add(orderbyExpr.first);
+ modifierList.add(orderbyExpr.second);
+ nullModifierList.add(orderbyExpr.third);
+ }
+ (
+ LOOKAHEAD(2) <COMMA> orderbyExpr = OrderByExpression()
{
- modifierList.add(modif);
+ orderbyList.add(orderbyExpr.first);
+ modifierList.add(orderbyExpr.second);
+ nullModifierList.add(orderbyExpr.third);
}
+ )*
+ {
+ oc.setOrderbyList(orderbyList);
+ oc.setModifierList(modifierList);
+ oc.setNullModifierList(nullModifierList);
+ return addSourceLocation(oc, startToken);
+ }
+}
- (LOOKAHEAD(2) <COMMA> orderbyExpr = Expression()
+Triple<Expression, OrderbyClause.OrderModifier, OrderbyClause.NullOrderModifier> OrderByExpression()
+ throws ParseException:
+{
+ Expression orderbyExpr = null;
+ OrderbyClause.OrderModifier modif = OrderbyClause.OrderModifier.ASC;
+ OrderbyClause.NullOrderModifier nullModif = null;
+}
+{
+ orderbyExpr = Expression()
+ (
+ <ASC> { modif = OrderbyClause.OrderModifier.ASC; }
+ |
+ <DESC> { modif = OrderbyClause.OrderModifier.DESC; }
+ )?
+ (
+ LOOKAHEAD({ laIdentifier(NULLS) }) <IDENTIFIER> { expectToken(NULLS); } <IDENTIFIER>
{
- orderbyList.add(orderbyExpr);
- modif = OrderbyClause.OrderModifier.ASC;
+ if (isToken(FIRST)) {
+ nullModif = OrderbyClause.NullOrderModifier.FIRST;
+ } else if (isToken(LAST)) {
+ nullModif = OrderbyClause.NullOrderModifier.LAST;
+ } else {
+ throw createUnexpectedTokenError();
+ }
}
- ( (<ASC> { modif = OrderbyClause.OrderModifier.ASC; })
- | (<DESC> { modif = OrderbyClause.OrderModifier.DESC; }))?
- {
- modifierList.add(modif);
- }
- )*
-
- {
- oc.setModifierList(modifierList);
- oc.setOrderbyList(orderbyList);
- return addSourceLocation(oc, startToken);
- }
+ )?
+ {
+ return new Triple<Expression, OrderbyClause.OrderModifier, OrderbyClause.NullOrderModifier>(orderbyExpr, modif,
+ nullModif);
+ }
}
GroupbyClause GroupbyClause()throws ParseException :
@@ -4781,8 +5225,10 @@
createNewScope();
}
- ( ((<ANY>|<SOME>) { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); })
- | (<EVERY> { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); }))
+ ( LOOKAHEAD(2)
+ (<ANY>|<SOME>)<AND><EVERY> { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME_AND_EVERY); }
+ | (<ANY>|<SOME>) { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); }
+ | <EVERY> { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); } )
var = Variable() <IN> inExpr = Expression()
{
pair = new QuantifiedPair(var, inExpr);
@@ -4883,8 +5329,10 @@
| <BTREE : "btree">
| <BY : "by">
| <CASE : "case">
+ | <CAST : "cast">
| <CLOSED : "closed">
| <CREATE : "create">
+ | <CROSS : "cross">
| <COMPACTION : "compaction"> // no longer used
| <COMPACT : "compact">
| <CONNECT : "connect">
@@ -4988,6 +5436,7 @@
| <USING : "using">
| <VALUE : "value">
| <VALUED : "valued">
+ | <VIEW : "view">
| <WHEN : "when">
| <WHERE : "where">
| <WITH : "with">
@@ -5098,7 +5547,7 @@
| <EscapeCr>
| <EscapeTab>
| ~["`","\\"])* "`">
- | <STRING_LITERAL : ("\"" (
+ | <STRING_LITERAL : ( ("E")? "\"" (
<EscapeQuot>
| <EscapeBslash>
| <EscapeSlash>
@@ -5108,7 +5557,7 @@
| <EscapeCr>
| <EscapeTab>
| ~["\"","\\"])* "\"")
- | ("\'"(
+ | ( ("E")? "\'" (
<EscapeApos>
| <EscapeBslash>
| <EscapeSlash>
diff --git a/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/sqlpp/parser/ParserTest.java b/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/sqlpp/parser/ParserTest.java
index 5b7ef16..8413c38 100644
--- a/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/sqlpp/parser/ParserTest.java
+++ b/asterixdb/asterix-lang-sqlpp/src/test/java/org/apache/asterix/lang/sqlpp/parser/ParserTest.java
@@ -19,8 +19,12 @@
package org.apache.asterix.lang.sqlpp.parser;
import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.IParser;
import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.common.base.Literal;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.junit.Assert;
import org.junit.Test;
public class ParserTest {
@@ -105,4 +109,87 @@
parser = factory.createParser(query);
parser.parse();
}
+
+ @Test
+ public void testStringLiterals() {
+ // test different quote combinations
+ String v1 = "a1";
+ String[] prefixes = new String[] { "", "E" };
+ String[] quotes = new String[] { "'", "\"" };
+
+ IParserFactory factory = new SqlppParserFactory();
+ StringBuilder qb = new StringBuilder();
+
+ for (String p : prefixes) {
+ for (String q1 : quotes) {
+ for (String q2 : quotes) {
+ qb.setLength(0);
+ qb.append(p).append(q1).append(v1).append(q2).append(';');
+
+ boolean expectSuccess = q1.equals(q2);
+ String expectedValue = expectSuccess ? v1 : "Lexical error";
+ testParseStringLiteral(qb.toString(), expectSuccess, expectedValue, factory);
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testStringLiteralsMulti() {
+ // test different quote combinations
+ String v1 = "a1", v2 = "b2", v3 = "c3";
+ String[] prefixesStart = new String[] { "", "E" };
+ String[] prefixesRest = new String[] { "", "E", " ", "\n" };
+ String[] quotes = new String[] { "'", "\"" };
+
+ IParserFactory factory = new SqlppParserFactory();
+ StringBuilder qb = new StringBuilder();
+ for (String p1 : prefixesStart) {
+ for (String q1 : quotes) {
+ for (String p2 : prefixesRest) {
+ for (String q2 : quotes) {
+ for (String p3 : prefixesRest) {
+ for (String q3 : quotes) {
+ qb.setLength(0);
+ qb.append(p1).append(q1).append(v1).append(q1);
+ qb.append(p2).append(q2).append(v2).append(q2);
+ qb.append(p3).append(q3).append(v3).append(q3);
+ qb.append(';');
+
+ boolean expectSuccess =
+ p1.isEmpty() && p2.isEmpty() && p3.isEmpty() && q1.equals(q2) && q1.equals(q3);
+ String expectedValue =
+ expectSuccess ? v1 + q1 + v2 + q1 + v3 : "Invalid string literal";
+ testParseStringLiteral(qb.toString(), expectSuccess, expectedValue, factory);
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ private void testParseStringLiteral(String query, boolean expectSuccess, String expectedValueOrError,
+ IParserFactory factory) {
+ try {
+ IParser parser = factory.createParser(query);
+ Expression expr = parser.parseExpression();
+ if (!expectSuccess) {
+ Assert.fail("Unexpected parsing success for: " + query);
+ }
+ Assert.assertEquals(Expression.Kind.LITERAL_EXPRESSION, expr.getKind());
+ LiteralExpr litExpr = (LiteralExpr) expr;
+ Literal lit = litExpr.getValue();
+ Assert.assertEquals(Literal.Type.STRING, lit.getLiteralType());
+ String value = lit.getStringValue();
+ Assert.assertEquals(expectedValueOrError, value);
+ } catch (CompilationException e) {
+ if (expectSuccess) {
+ Assert.fail("Unexpected parsing failure for: " + query + " : " + e);
+ } else {
+ Assert.assertTrue(e.getMessage() + " does not contain: " + expectedValueOrError,
+ e.getMessage().contains(expectedValueOrError));
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-license/pom.xml b/asterixdb/asterix-license/pom.xml
index ea56748..3a6f1c3 100644
--- a/asterixdb/asterix-license/pom.xml
+++ b/asterixdb/asterix-license/pom.xml
@@ -88,7 +88,6 @@
</licenses>
<templateProperties>
<asterixAppResourcesPrefix>asterix-app/src/main/resources/</asterixAppResourcesPrefix>
- <hivecompatPrefix>asterix-hivecompat/src/main/java/</hivecompatPrefix>
<asterixDashboardResourcesPrefix>asterix-dashboard/src/main/resources/</asterixDashboardResourcesPrefix>
<packageName />
</templateProperties>
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
index 3c9e18f..cd00397 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
@@ -45,7 +45,6 @@
<dependency>
<groupId>org.apache.maven.plugin-tools</groupId>
<artifactId>maven-plugin-annotations</artifactId>
- <version>3.6.0</version>
<scope>provided</scope>
</dependency>
<dependency>
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java
index d47e9a6..19bfe1d 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/src/main/java/org/apache/asterix/extension/grammar/GrammarExtensionMojo.java
@@ -459,7 +459,9 @@
prevCharEscape = chars[bufferPosn] == ExternalDataConstants.ESCAPE;
}
} else {
- if (chars[bufferPosn] == ExternalDataConstants.QUOTE) {
+ if (chars[bufferPosn] == ExternalDataConstants.QUOTE && chars[bufferPosn] != start
+ && chars[bufferPosn - 1] != '\'') {
+ // If we see a quote that is not preceded by an apostrophe, then we are inside a string.
inString = true;
} else if (chars[bufferPosn] == start) {
depth += 1;
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 4a6d6ddb..61507d1 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -71,12 +71,6 @@
<groupId>org.apache.asterix</groupId>
<artifactId>asterix-external-data</artifactId>
<version>${project.version}</version>
- <exclusions>
- <exclusion>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-exec</artifactId>
- </exclusion>
- </exclusions>
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index f8ba574..87f5129 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -20,7 +20,6 @@
package org.apache.asterix.metadata;
import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.PLURAL;
-import static org.apache.asterix.common.exceptions.ErrorCode.FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
import java.rmi.RemoteException;
@@ -34,7 +33,6 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.dataflow.LSMIndexUtil;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.MetadataException;
@@ -50,7 +48,9 @@
import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.formats.nontagged.TypeTraitProvider;
import org.apache.asterix.metadata.api.ExtensionMetadataDataset;
import org.apache.asterix.metadata.api.ExtensionMetadataDatasetId;
import org.apache.asterix.metadata.api.IExtensionMetadataEntity;
@@ -66,6 +66,7 @@
import org.apache.asterix.metadata.entities.DatasourceAdapter;
import org.apache.asterix.metadata.entities.Datatype;
import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.entities.DependencyKind;
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
@@ -78,6 +79,7 @@
import org.apache.asterix.metadata.entities.Node;
import org.apache.asterix.metadata.entities.NodeGroup;
import org.apache.asterix.metadata.entities.Synonym;
+import org.apache.asterix.metadata.entities.ViewDetails;
import org.apache.asterix.metadata.entitytupletranslators.CompactionPolicyTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.DatasetTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
@@ -361,9 +363,8 @@
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
// Add the primary index for the dataset.
InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
- Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
- dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), id.getKeySourceIndicator(),
- id.getPrimaryKeyType(), false, false, true, dataset.getPendingOp());
+ Index primaryIndex = Index.createPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName(),
+ id.getPrimaryKey(), id.getKeySourceIndicator(), id.getPrimaryKeyType(), dataset.getPendingOp());
addIndex(txnId, primaryIndex);
}
@@ -770,15 +771,16 @@
@Override
public void dropDataset(TxnId txnId, DataverseName dataverseName, String datasetName, boolean force)
throws AlgebricksException {
- if (!force) {
- confirmDatasetCanBeDeleted(txnId, dataverseName, datasetName);
- }
-
Dataset dataset = getDataset(txnId, dataverseName, datasetName);
if (dataset == null) {
throw new AsterixException(org.apache.asterix.common.exceptions.ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE,
datasetName, dataverseName);
}
+ if (!force) {
+ String datasetTypeDisplayName = DatasetUtil.getDatasetTypeDisplayName(dataset.getDatasetType());
+ confirmDatasetCanBeDeleted(txnId, datasetTypeDisplayName, dataverseName, datasetName);
+ }
+
try {
// Delete entry from the 'datasets' dataset.
ITupleReference searchKey = createTuple(dataverseName, datasetName);
@@ -788,25 +790,37 @@
try {
datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
- // Delete entry(s) from the 'indexes' dataset.
- List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
- if (datasetIndexes != null) {
- for (Index index : datasetIndexes) {
- dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
- }
- }
-
- if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
- // Delete External Files
- // As a side effect, acquires an S lock on the 'ExternalFile' dataset
- // on behalf of txnId.
- List<ExternalFile> datasetFiles = getExternalFiles(txnId, dataset);
- if (datasetFiles != null && !datasetFiles.isEmpty()) {
- // Drop all external files in this dataset.
- for (ExternalFile file : datasetFiles) {
- dropExternalFile(txnId, dataverseName, file.getDatasetName(), file.getFileNumber());
+ switch (dataset.getDatasetType()) {
+ case INTERNAL:
+ // Delete entry(s) from the 'indexes' dataset.
+ List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
+ if (datasetIndexes != null) {
+ for (Index index : datasetIndexes) {
+ dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
+ }
}
- }
+ break;
+ case EXTERNAL:
+ // Delete entry(s) from the 'indexes' dataset.
+ datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
+ if (datasetIndexes != null) {
+ for (Index index : datasetIndexes) {
+ dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
+ }
+ }
+ // Delete External Files
+ // As a side effect, acquires an S lock on the 'ExternalFile' dataset
+ // on behalf of txnId.
+ List<ExternalFile> datasetFiles = getExternalFiles(txnId, dataset);
+ if (datasetFiles != null && !datasetFiles.isEmpty()) {
+ // Drop all external files in this dataset.
+ for (ExternalFile file : datasetFiles) {
+ dropExternalFile(txnId, dataverseName, file.getDatasetName(), file.getFileNumber());
+ }
+ }
+ break;
+ case VIEW:
+ break;
}
} catch (HyracksDataException hde) {
// ignore this exception and continue deleting all relevant
@@ -1122,44 +1136,60 @@
// uses a type from this dataverse
// throw an error
List<Dataset> datasets = getAllDatasets(txnId);
- for (Dataset set : datasets) {
- if (set.getDataverseName().equals(dataverseName)) {
+ for (Dataset dataset : datasets) {
+ if (dataset.getDataverseName().equals(dataverseName)) {
continue;
}
- if (set.getItemTypeDataverseName().equals(dataverseName)) {
+ if (dataset.getItemTypeDataverseName().equals(dataverseName)) {
throw new AsterixException(
org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS, "type",
- TypeUtil.getFullyQualifiedDisplayName(set.getItemTypeDataverseName(), set.getItemTypeName()),
- dataset(), DatasetUtil.getFullyQualifiedDisplayName(set));
+ TypeUtil.getFullyQualifiedDisplayName(dataset.getItemTypeDataverseName(),
+ dataset.getItemTypeName()),
+ dataset(), DatasetUtil.getFullyQualifiedDisplayName(dataset));
}
- if (set.getMetaItemTypeDataverseName() != null
- && set.getMetaItemTypeDataverseName().equals(dataverseName)) {
+ if (dataset.hasMetaPart() && dataset.getMetaItemTypeDataverseName().equals(dataverseName)) {
throw new AsterixException(
org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS, "type",
- TypeUtil.getFullyQualifiedDisplayName(set.getMetaItemTypeDataverseName(),
- set.getMetaItemTypeName()),
- dataset(), DatasetUtil.getFullyQualifiedDisplayName(set));
+ TypeUtil.getFullyQualifiedDisplayName(dataset.getMetaItemTypeDataverseName(),
+ dataset.getMetaItemTypeName()),
+ dataset(), DatasetUtil.getFullyQualifiedDisplayName(dataset));
+ }
+ if (dataset.getDatasetType() == DatasetType.VIEW) {
+ ViewDetails viewDetails = (ViewDetails) dataset.getDatasetDetails();
+ List<DependencyKind> dependenciesSchema = ViewDetails.DEPENDENCIES_SCHEMA;
+ List<List<Triple<DataverseName, String, String>>> dependencies = viewDetails.getDependencies();
+ for (int i = 0, n = dependencies.size(); i < n; i++) {
+ for (Triple<DataverseName, String, String> dependency : dependencies.get(i)) {
+ if (dependency.first.equals(dataverseName)) {
+ DependencyKind dependencyKind = dependenciesSchema.get(i);
+ throw new AsterixException(
+ org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS,
+ dependencyKind, dependencyKind.getDependencyDisplayName(dependency), "view",
+ DatasetUtil.getFullyQualifiedDisplayName(dataset));
+ }
+ }
+ }
}
}
// If a function from a DIFFERENT dataverse
// uses datasets, functions, datatypes, or synonyms from this dataverse
// throw an error
- Function.FunctionDependencyKind[] functionDependencyKinds = Function.FunctionDependencyKind.values();
List<Function> functions = getAllFunctions(txnId);
for (Function function : functions) {
if (function.getDataverseName().equals(dataverseName)) {
continue;
}
+ List<DependencyKind> dependenciesSchema = Function.DEPENDENCIES_SCHEMA;
List<List<Triple<DataverseName, String, String>>> dependencies = function.getDependencies();
for (int i = 0, n = dependencies.size(); i < n; i++) {
for (Triple<DataverseName, String, String> dependency : dependencies.get(i)) {
if (dependency.first.equals(dataverseName)) {
- Function.FunctionDependencyKind functionDependencyKind = functionDependencyKinds[i];
+ DependencyKind dependencyKind = dependenciesSchema.get(i);
throw new AsterixException(
org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS,
- functionDependencyKind, functionDependencyKind.getDependencyDisplayName(dependency),
- "function", function.getSignature());
+ dependencyKind, dependencyKind.getDependencyDisplayName(dependency), "function",
+ function.getSignature());
}
}
}
@@ -1184,6 +1214,7 @@
}
private void confirmFunctionCanBeDeleted(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
+ confirmFunctionIsUnusedByViews(txnId, signature);
confirmFunctionIsUnusedByFunctions(txnId, signature);
// if any other feed connection uses this function, throw an error
@@ -1198,18 +1229,34 @@
}
}
- private void confirmFunctionIsUnusedByFunctions(TxnId txnId, FunctionSignature signature)
- throws AlgebricksException {
- confirmObjectIsUnusedByFunctions(txnId, Function.FunctionDependencyKind.FUNCTION, signature.getDataverseName(),
+ private void confirmFunctionIsUnusedByViews(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
+ confirmObjectIsUnusedByViews(txnId, "function", DependencyKind.FUNCTION, signature.getDataverseName(),
signature.getName(), Integer.toString(signature.getArity()));
}
- private void confirmObjectIsUnusedByFunctions(TxnId txnId, Function.FunctionDependencyKind dependencyKind,
- DataverseName dataverseName, String objectName, String objectArg) throws AlgebricksException {
+ private void confirmFunctionIsUnusedByFunctions(TxnId txnId, FunctionSignature signature)
+ throws AlgebricksException {
+ confirmObjectIsUnusedByFunctions(txnId, "function", DependencyKind.FUNCTION, signature.getDataverseName(),
+ signature.getName(), Integer.toString(signature.getArity()));
+ }
+
+ private void confirmObjectIsUnusedByFunctions(TxnId txnId, String objectKindDisplayName,
+ DependencyKind dependencyKind, DataverseName dataverseName, String objectName, String objectArg)
+ throws AlgebricksException {
// If any function uses this object, throw an error
- int functionDependencyIdx = dependencyKind.ordinal();
List<Function> functions = getAllFunctions(txnId);
- for (Function function : functions) {
+ confirmObjectIsUnusedByFunctionsImpl(functions, objectKindDisplayName, dependencyKind, dataverseName,
+ objectName, objectArg);
+ }
+
+ private void confirmObjectIsUnusedByFunctionsImpl(List<Function> allFunctions, String objectKindDisplayName,
+ DependencyKind dependencyKind, DataverseName dataverseName, String objectName, String objectArg)
+ throws AlgebricksException {
+ int functionDependencyIdx = Function.DEPENDENCIES_SCHEMA.indexOf(dependencyKind);
+ if (functionDependencyIdx < 0) {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE);
+ }
+ for (Function function : allFunctions) {
List<List<Triple<DataverseName, String, String>>> functionDependencies = function.getDependencies();
if (functionDependencyIdx < functionDependencies.size()) {
List<Triple<DataverseName, String, String>> functionObjectDependencies =
@@ -1220,9 +1267,46 @@
&& (objectArg == null || objectArg.equals(dependency.third))) {
throw new AsterixException(
org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_OBJECT_DEPENDENT_EXISTS,
- dependencyKind.toString().toLowerCase(),
- dependencyKind.getDependencyDisplayName(dependency), "function",
- function.getSignature());
+ objectKindDisplayName, dependencyKind.getDependencyDisplayName(dependency),
+ "function", function.getSignature());
+ }
+ }
+ }
+ }
+ }
+ }
+
+ private void confirmObjectIsUnusedByViews(TxnId txnId, String objectKindDisplayName, DependencyKind dependencyKind,
+ DataverseName dataverseName, String objectName, String objectArg) throws AlgebricksException {
+ // If any function uses this object, throw an error
+ List<Dataset> datasets = getAllDatasets(txnId);
+ confirmObjectIsUnusedByViewsImpl(datasets, objectKindDisplayName, dependencyKind, dataverseName, objectName,
+ objectArg);
+ }
+
+ private void confirmObjectIsUnusedByViewsImpl(List<Dataset> allDatasets, String objectKindDisplayName,
+ DependencyKind dependencyKind, DataverseName dataverseName, String objectName, String objectArg)
+ throws AlgebricksException {
+ int viewDependencyIdx = ViewDetails.DEPENDENCIES_SCHEMA.indexOf(dependencyKind);
+ if (viewDependencyIdx < 0) {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE);
+ }
+ for (Dataset dataset : allDatasets) {
+ if (dataset.getDatasetType() == DatasetType.VIEW) {
+ ViewDetails viewDetails = (ViewDetails) dataset.getDatasetDetails();
+ List<List<Triple<DataverseName, String, String>>> viewDependencies = viewDetails.getDependencies();
+ if (viewDependencyIdx < viewDependencies.size()) {
+ List<Triple<DataverseName, String, String>> viewObjectDependencies =
+ viewDependencies.get(viewDependencyIdx);
+ if (viewObjectDependencies != null) {
+ for (Triple<DataverseName, String, String> dependency : viewObjectDependencies) {
+ if (dependency.first.equals(dataverseName) && dependency.second.equals(objectName)
+ && (objectArg == null || objectArg.equals(dependency.third))) {
+ throw new AsterixException(
+ org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_OBJECT_DEPENDENT_EXISTS,
+ objectKindDisplayName, dependencyKind.getDependencyDisplayName(dependency),
+ "view", DatasetUtil.getFullyQualifiedDisplayName(dataset));
+ }
}
}
}
@@ -1233,7 +1317,8 @@
private void confirmFullTextConfigCanBeDeleted(TxnId txnId, DataverseName dataverseNameFullTextConfig,
String configName) throws AlgebricksException {
if (Strings.isNullOrEmpty(configName)) {
- throw new MetadataException(FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED);
+ throw new MetadataException(
+ org.apache.asterix.common.exceptions.ErrorCode.FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED);
}
// If any index uses this full-text config, throw an error
@@ -1261,14 +1346,21 @@
}
}
- private void confirmDatasetCanBeDeleted(TxnId txnId, DataverseName dataverseName, String datasetName)
- throws AlgebricksException {
- confirmDatasetIsUnusedByFunctions(txnId, dataverseName, datasetName);
+ private void confirmDatasetCanBeDeleted(TxnId txnId, String datasetTypeDisplayName, DataverseName dataverseName,
+ String datasetName) throws AlgebricksException {
+ confirmDatasetIsUnusedByFunctions(txnId, datasetTypeDisplayName, dataverseName, datasetName);
+ confirmDatasetIsUnusedByViews(txnId, datasetTypeDisplayName, dataverseName, datasetName);
}
- private void confirmDatasetIsUnusedByFunctions(TxnId txnId, DataverseName dataverseName, String datasetName)
- throws AlgebricksException {
- confirmObjectIsUnusedByFunctions(txnId, Function.FunctionDependencyKind.DATASET, dataverseName, datasetName,
+ private void confirmDatasetIsUnusedByFunctions(TxnId txnId, String datasetKindDisplayName,
+ DataverseName dataverseName, String datasetName) throws AlgebricksException {
+ confirmObjectIsUnusedByFunctions(txnId, datasetKindDisplayName, DependencyKind.DATASET, dataverseName,
+ datasetName, null);
+ }
+
+ private void confirmDatasetIsUnusedByViews(TxnId txnId, String datasetKindDisplayName, DataverseName dataverseName,
+ String datasetName) throws AlgebricksException {
+ confirmObjectIsUnusedByViews(txnId, datasetKindDisplayName, DependencyKind.DATASET, dataverseName, datasetName,
null);
}
@@ -1318,14 +1410,21 @@
throws AlgebricksException {
// If any dataset uses this type, throw an error
List<Dataset> datasets = getAllDatasets(txnId);
- for (Dataset set : datasets) {
- if (set.getItemTypeName().equals(datatypeName) && set.getItemTypeDataverseName().equals(dataverseName)) {
+ for (Dataset dataset : datasets) {
+ if ((dataset.getItemTypeName().equals(datatypeName)
+ && dataset.getItemTypeDataverseName().equals(dataverseName))
+ || ((dataset.hasMetaPart() && dataset.getMetaItemTypeName().equals(datatypeName)
+ && dataset.getMetaItemTypeDataverseName().equals(dataverseName)))) {
throw new AsterixException(
org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_OBJECT_DEPENDENT_EXISTS, "type",
TypeUtil.getFullyQualifiedDisplayName(dataverseName, datatypeName), dataset(),
- DatasetUtil.getFullyQualifiedDisplayName(set));
+ DatasetUtil.getFullyQualifiedDisplayName(dataset));
}
}
+
+ // additionally, if a view uses this type, throw an error
+ // Note: for future use. currently views don't have any type dependencies
+ confirmObjectIsUnusedByViewsImpl(datasets, null, DependencyKind.TYPE, dataverseName, datatypeName, null);
}
private void confirmDatatypeIsUnusedByDatatypes(TxnId txnId, DataverseName dataverseName, String datatypeName)
@@ -1355,8 +1454,7 @@
private void confirmDatatypeIsUnusedByFunctions(TxnId txnId, DataverseName dataverseName, String dataTypeName)
throws AlgebricksException {
- confirmObjectIsUnusedByFunctions(txnId, Function.FunctionDependencyKind.TYPE, dataverseName, dataTypeName,
- null);
+ confirmObjectIsUnusedByFunctions(txnId, "datatype", DependencyKind.TYPE, dataverseName, dataTypeName, null);
}
private void confirmFullTextFilterCanBeDeleted(TxnId txnId, DataverseName dataverseName, String fullTextFilterName)
@@ -1539,7 +1637,8 @@
private ITupleReference getTupleToBeDeleted(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference searchKey)
throws AlgebricksException, HyracksDataException {
- IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
+ IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits(),
+ TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
List<ITupleReference> results = new ArrayList<>();
searchIndex(txnId, metadataIndex, searchKey, valueExtractor, results);
if (results.isEmpty()) {
@@ -2297,12 +2396,17 @@
private void confirmSynonymCanBeDeleted(TxnId txnId, DataverseName dataverseName, String synonymName)
throws AlgebricksException {
confirmSynonymIsUnusedByFunctions(txnId, dataverseName, synonymName);
+ confirmSynonymIsUnusedByViews(txnId, dataverseName, synonymName);
}
private void confirmSynonymIsUnusedByFunctions(TxnId txnId, DataverseName dataverseName, String synonymName)
throws AlgebricksException {
- confirmObjectIsUnusedByFunctions(txnId, Function.FunctionDependencyKind.SYNONYM, dataverseName, synonymName,
- null);
+ confirmObjectIsUnusedByFunctions(txnId, "synonym", DependencyKind.SYNONYM, dataverseName, synonymName, null);
+ }
+
+ private void confirmSynonymIsUnusedByViews(TxnId txnId, DataverseName dataverseName, String synonymName)
+ throws AlgebricksException {
+ confirmObjectIsUnusedByViews(txnId, "synonym", DependencyKind.SYNONYM, dataverseName, synonymName, null);
}
@Override
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index a6b1f4b..48fb450 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -190,6 +190,8 @@
public void dropFullTextFilter(DataverseName dataverseName, String filterName) {
AbstractFullTextFilterDescriptor filter = new AbstractFullTextFilterDescriptor(dataverseName, filterName) {
+ private static final long serialVersionUID = -8222222581298765902L;
+
@Override
public FullTextFilterType getFilterType() {
return null;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 047b823..fd0d30b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -45,6 +45,8 @@
import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
import org.apache.asterix.external.indexing.ExternalFile;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
+import org.apache.asterix.formats.nontagged.TypeTraitProvider;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataNode;
@@ -66,6 +68,7 @@
import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
@@ -428,7 +431,8 @@
storageComponentProvider.getIoOperationSchedulerProvider(),
appContext.getMetadataMergePolicyFactory(),
StorageConstants.DEFAULT_COMPACTION_POLICY_PROPERTIES, true, bloomFilterKeyFields,
- bloomFilterFalsePositiveRate, true, null, NoOpCompressorDecompressorFactory.INSTANCE, true);
+ bloomFilterFalsePositiveRate, true, null, NoOpCompressorDecompressorFactory.INSTANCE, true,
+ TypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
DatasetLocalResourceFactory dsLocalResourceFactory =
new DatasetLocalResourceFactory(datasetId, lsmBtreeFactory);
// TODO(amoudi) Creating the index should be done through the same code path as
@@ -503,6 +507,10 @@
private static void recoverDataset(MetadataTransactionContext mdTxnCtx, Dataset dataset)
throws AlgebricksException {
+ if (dataset.getDatasetType() == DatasetType.VIEW) {
+ // Views don't need any recovery and cannot be in a pending state
+ return;
+ }
if (dataset.getPendingOp() != MetadataUtil.PENDING_NO_OP) {
// drop pending dataset
MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataset.getDataverseName(), dataset.getDatasetName(), true);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 2e75319..f00090a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -37,6 +37,7 @@
public static final String FIELD_NAME_ARITY = "Arity";
public static final String FIELD_NAME_ARGS = "Arguments";
public static final String FIELD_NAME_AUTOGENERATED = "Autogenerated";
+ public static final String FIELD_NAME_CAST = "Cast";
public static final String FIELD_NAME_CLASSNAME = "Classname";
public static final String FIELD_NAME_COMPACTION_POLICY = "CompactionPolicy";
public static final String FIELD_NAME_COMPACTION_POLICY_PROPERTIES = "CompactionPolicyProperties";
@@ -48,6 +49,7 @@
public static final String FIELD_NAME_DATATYPE_NAME = "DatatypeName";
public static final String FIELD_NAME_DATAVERSE_NAME = "DataverseName";
public static final String FIELD_NAME_DATA_FORMAT = "DataFormat";
+ public static final String FIELD_NAME_DEFAULT = "Default";
public static final String FIELD_NAME_DEFINITION = "Definition";
public static final String FIELD_NAME_DEPENDENCIES = "Dependencies";
public static final String FIELD_NAME_DERIVED = "Derived";
@@ -63,12 +65,15 @@
public static final String FIELD_NAME_FILE_NUMBER = "FileNumber";
public static final String FIELD_NAME_FILE_SIZE = "FileSize";
public static final String FIELD_NAME_FILE_STRUCTURE = "FileStructure";
+ public static final String FIELD_NAME_FOREIGN_KEY = "ForeignKey";
+ public static final String FIELD_NAME_FOREIGN_KEYS = "ForeignKeys";
public static final String FIELD_NAME_GROUP_NAME = "GroupName";
public static final String FIELD_NAME_HINTS = "Hints";
public static final String FIELD_NAME_INDEX_NAME = "IndexName";
public static final String FIELD_NAME_INDEX_STRUCTURE = "IndexStructure";
public static final String FIELD_NAME_INTERNAL_DETAILS = "InternalDetails";
public static final String FIELD_NAME_IS_ANONYMOUS = "IsAnonymous";
+ public static final String FIELD_NAME_IS_ENFORCED = "IsEnforced";
public static final String FIELD_NAME_IS_MISSABLE = "IsMissable";
public static final String FIELD_NAME_IS_NULLABLE = "IsNullable";
public static final String FIELD_NAME_IS_OPEN = "IsOpen";
@@ -94,8 +99,11 @@
public static final String FIELD_NAME_PENDING_OP = "PendingOp";
public static final String FIELD_NAME_POLICY_NAME = "PolicyName";
public static final String FIELD_NAME_PRIMARY_KEY = "PrimaryKey";
+ public static final String FIELD_NAME_PRIMARY_KEY_ENFORCED = "PrimaryKeyEnforced";
public static final String FIELD_NAME_PROPERTIES = "Properties";
public static final String FIELD_NAME_RECORD = "Record";
+ public static final String FIELD_NAME_REF_DATAVERSE_NAME = "RefDataverseName";
+ public static final String FIELD_NAME_REF_DATASET_NAME = "RefDatasetName";
public static final String FIELD_NAME_RETURN_TYPE = "ReturnType";
public static final String FIELD_NAME_RETURN_TYPE_DATAVERSE_NAME = "ReturnTypeDataverseName";
public static final String FIELD_NAME_SEARCH_KEY = "SearchKey";
@@ -107,6 +115,7 @@
public static final String FIELD_NAME_TYPE = "Type";
public static final String FIELD_NAME_UNORDERED_LIST = "UnorderedList";
public static final String FIELD_NAME_VALUE = "Value";
+ public static final String FIELD_NAME_VIEW_DETAILS = "ViewDetails";
public static final String FIELD_NAME_WORKING_MEMORY_SIZE = "WorkingMemorySize";
public static final String FIELD_NAME_APPLIED_FUNCTIONS = "AppliedFunctions";
public static final String FIELD_NAME_WHERE_CLAUSE = "WhereClause";
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
index 7c19282..c0bdc75 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
@@ -18,7 +18,6 @@
*/
package org.apache.asterix.metadata.declared;
-import java.util.List;
import java.util.Map;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -26,12 +25,14 @@
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
import org.apache.asterix.metadata.api.IResourceFactoryProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.utils.ArrayIndexUtil;
import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -66,6 +67,7 @@
int[] filterFields = IndexUtil.getFilterFields(dataset, index, filterTypeTraits);
int[] btreeFields = IndexUtil.getBtreeFieldsIfFiltered(dataset, index);
IStorageComponentProvider storageComponentProvider = mdProvider.getStorageComponentProvider();
+ ITypeTraitProvider typeTraitProvider = storageComponentProvider.getTypeTraitProvider();
ITypeTraits[] typeTraits = getTypeTraits(mdProvider, dataset, index, recordType, metaType);
IBinaryComparatorFactory[] cmpFactories = getCmpFactories(mdProvider, dataset, index, recordType, metaType);
double bloomFilterFalsePositiveRate = mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate();
@@ -97,7 +99,8 @@
filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, null, bloomFilterFalsePositiveRate,
- index.isPrimaryIndex(), btreeFields, compDecompFactory, false);
+ index.isPrimaryIndex(), btreeFields, compDecompFactory, false,
+ typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
@@ -130,10 +133,8 @@
sourceType = metaType;
}
for (int i = 0; i < e.getProjectList().size(); i++) {
- List<String> project = e.getProjectList().get(i);
Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
- ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project), sourceType,
- ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+ e.getUnnestList(), e.getProjectList().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryTypeTraits[secondaryTypeTraitPos++] = typeTraitProvider.getTypeTrait(keyType);
}
@@ -171,10 +172,8 @@
sourceType = metaType;
}
for (int i = 0; i < e.getProjectList().size(); i++) {
- List<String> project = e.getProjectList().get(i);
Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
- ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project), sourceType,
- ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+ e.getUnnestList(), e.getProjectList().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryCmpFactories[secondaryCmpFactoriesPos++] =
cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 7f56f2c..168b56e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -28,11 +28,13 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.indexing.FilesIndexDescription;
import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
import org.apache.asterix.metadata.api.IResourceFactoryProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -82,6 +84,7 @@
ILSMIOOperationSchedulerProvider ioSchedulerProvider =
storageComponentProvider.getIoOperationSchedulerProvider();
boolean hasBloomFilter = bloomFilterFields != null;
+ ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
switch (dataset.getDatasetType()) {
case EXTERNAL:
return index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))
@@ -89,12 +92,14 @@
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
- bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter)
+ bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+ typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE)
: new ExternalBTreeWithBuddyLocalResourceFactory(storageManager, typeTraits, cmpFactories,
filterTypeTraits, filterCmpFactories, filterFields, opTrackerFactory,
ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
- bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter);
+ bloomFilterFalsePositiveRate, false, btreeFields, hasBloomFilter,
+ typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
case INTERNAL:
AsterixVirtualBufferCacheProvider vbcProvider =
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
@@ -112,7 +117,7 @@
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, bloomFilterFields,
bloomFilterFalsePositiveRate, index.isPrimaryIndex(), btreeFields, compDecompFactory,
- hasBloomFilter);
+ hasBloomFilter, typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
default:
throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
dataset.getDatasetType().toString());
@@ -141,7 +146,7 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
@@ -178,7 +183,7 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryCmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 17f91b4..1d82b19 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -20,6 +20,7 @@
import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
+import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -27,6 +28,7 @@
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.external.api.ITypedAdapterFactory;
+import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.entities.Dataset;
@@ -36,6 +38,7 @@
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.projection.DataProjectionInfo;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -114,7 +117,7 @@
ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
int externalScanBufferSize = physicalOptimizationConfig.getExternalScanBufferSize();
- Map<String, String> properties = addProjectionInfo(projectionInfo, edd.getProperties());
+ Map<String, String> properties = addExternalProjectionInfo(projectionInfo, edd.getProperties());
properties.put(KEY_EXTERNAL_SCAN_BUFFER_SIZE, String.valueOf(externalScanBufferSize));
ITypedAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
edd.getAdapter(), properties, (ARecordType) itemType, null, context.getWarningCollector());
@@ -129,22 +132,27 @@
int[] minFilterFieldIndexes = createFilterIndexes(minFilterVars, opSchema);
int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
- return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false,
+ return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
- true, false, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
- false, false);
+ true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
+ outputLimit, false, false);
default:
throw new AlgebricksException("Unknown datasource type");
}
}
- private Map<String, String> addProjectionInfo(IProjectionInfo<?> projectionInfo, Map<String, String> properties) {
+ private Map<String, String> addExternalProjectionInfo(IProjectionInfo<?> projectionInfo,
+ Map<String, String> properties) {
Map<String, String> propertiesCopy = properties;
if (projectionInfo != null) {
//properties could be cached and reused, so we make a copy per query
propertiesCopy = new HashMap<>(properties);
- ExternalDataProjectionInfo fieldNamesInfo = (ExternalDataProjectionInfo) projectionInfo;
- fieldNamesInfo.addToProperties(propertiesCopy);
+ try {
+ DataProjectionInfo externalProjectionInfo = (DataProjectionInfo) projectionInfo;
+ ExternalDataUtils.setExternalDataProjectionInfo(externalProjectionInfo, propertiesCopy);
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
}
return propertiesCopy;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ExternalDataProjectionInfo.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ExternalDataProjectionInfo.java
deleted file mode 100644
index fcbf522..0000000
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ExternalDataProjectionInfo.java
+++ /dev/null
@@ -1,98 +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.
- */
-package org.apache.asterix.metadata.declared;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
-
-public class ExternalDataProjectionInfo implements IProjectionInfo<List<String>> {
- private final List<List<String>> projectedFieldNames;
-
- public ExternalDataProjectionInfo() {
- projectedFieldNames = new ArrayList<>();
- }
-
- private ExternalDataProjectionInfo(List<List<String>> projectedFieldNames) {
- this.projectedFieldNames = new ArrayList<>();
- for (List<String> path : projectedFieldNames) {
- List<String> newPath = new ArrayList<>(path);
- this.projectedFieldNames.add(newPath);
- }
- }
-
- @Override
- public List<List<String>> getProjectionInfo() {
- return projectedFieldNames;
- }
-
- @Override
- public IProjectionInfo<List<String>> createCopy() {
- return new ExternalDataProjectionInfo(projectedFieldNames);
- }
-
- @Override
- public boolean equals(Object other) {
- if (!(other instanceof ExternalDataProjectionInfo)) {
- return false;
- }
- List<List<String>> otherProjectedFieldNames = ((ExternalDataProjectionInfo) other).projectedFieldNames;
- return projectedFieldNames.size() == otherProjectedFieldNames.size()
- && VariableUtilities.varListEqualUnordered(projectedFieldNames, otherProjectedFieldNames);
- }
-
- public String toString() {
- if (projectedFieldNames.isEmpty()) {
- return "";
- }
- final StringBuilder fieldNamesBuilder = new StringBuilder();
- append(fieldNamesBuilder);
- return fieldNamesBuilder.toString();
- }
-
- /**
- * Append projected field names to the external dataset properties
- */
- public void addToProperties(Map<String, String> properties) {
- final String pushedFieldNames = toString();
- if (!pushedFieldNames.isEmpty()) {
- properties.put(ExternalDataConstants.KEY_REQUESTED_FIELDS, toString());
- }
- }
-
- private void append(StringBuilder builder) {
- appendFieldNames(projectedFieldNames.get(0), builder);
- for (int i = 1; i < projectedFieldNames.size(); i++) {
- builder.append(", ");
- appendFieldNames(projectedFieldNames.get(i), builder);
- }
- }
-
- private void appendFieldNames(List<String> fieldNames, StringBuilder builder) {
- builder.append(fieldNames.get(0));
- for (int i = 1; i < fieldNames.size(); i++) {
- builder.append('.').append(fieldNames.get(i));
- }
- }
-
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index 1376008..ee629b1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -18,14 +18,13 @@
*/
package org.apache.asterix.metadata.declared;
-import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
import static org.apache.asterix.common.utils.IdentifierUtil.dataverse;
import java.util.ArrayList;
import java.util.List;
import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.metadata.DataverseName;
@@ -103,17 +102,25 @@
return adapter;
}
- public static Dataset findDataset(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String dataset)
- throws AlgebricksException {
- return MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, dataset);
+ public static Dataset findDataset(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
+ String datasetName, boolean includingViews) throws AlgebricksException {
+ Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
+ if (!includingViews && dataset != null && dataset.getDatasetType() == DatasetConfig.DatasetType.VIEW) {
+ return null;
+ }
+ return dataset;
+ }
+
+ public static Dataset findDataset(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
+ String datasetName) throws AlgebricksException {
+ return findDataset(mdTxnCtx, dataverseName, datasetName, false);
}
public static Dataset findExistingDataset(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
String datasetName) throws AlgebricksException {
Dataset dataset = findDataset(mdTxnCtx, dataverseName, datasetName);
if (dataset == null) {
- throw new AlgebricksException(
- "Unknown " + dataset() + " " + datasetName + " in " + dataverse() + " " + dataverseName);
+ throw new AsterixException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, datasetName, dataverseName);
}
return dataset;
}
@@ -185,13 +192,25 @@
MetadataTransactionContext mdTxnCtx, DataSourceId id) throws AlgebricksException {
Dataset dataset = findDataset(mdTxnCtx, id.getDataverseName(), id.getDatasourceName());
if (dataset == null) {
- throw new AlgebricksException("Datasource with id " + id + " was not found.");
+ throw new AsterixException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, id.getDatasourceName(),
+ id.getDataverseName());
}
+ byte datasourceType;
+ switch (dataset.getDatasetType()) {
+ case INTERNAL:
+ datasourceType = DataSource.Type.INTERNAL_DATASET;
+ break;
+ case EXTERNAL:
+ datasourceType = DataSource.Type.EXTERNAL_DATASET;
+ break;
+ default:
+ throw new AsterixException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, id.getDatasourceName(),
+ id.getDataverseName());
+ }
+
IAType itemType = findType(mdTxnCtx, dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
IAType metaItemType = findType(mdTxnCtx, dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
INodeDomain domain = findNodeDomain(clusterStateManager, mdTxnCtx, dataset.getNodeGroupName());
- byte datasourceType = dataset.getDatasetType().equals(DatasetType.EXTERNAL) ? DataSource.Type.EXTERNAL_DATASET
- : DataSource.Type.INTERNAL_DATASET;
return new DatasetDataSource(id, dataset, itemType, metaItemType, datasourceType, dataset.getDatasetDetails(),
domain);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 1721975..4cc83ab 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -20,7 +20,6 @@
import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.PLURAL;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
-import static org.apache.asterix.common.utils.IdentifierUtil.dataverse;
import static org.apache.asterix.metadata.utils.MetadataConstants.METADATA_OBJECT_NAME_INVALID_CHARS;
import java.io.File;
@@ -69,8 +68,7 @@
import org.apache.asterix.external.provider.AdapterFactoryProvider;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.formats.base.IDataFormat;
-import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
-import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryIntegerInspector;
import org.apache.asterix.formats.nontagged.LinearizeComparatorFactoryProvider;
import org.apache.asterix.formats.nontagged.TypeTraitProvider;
import org.apache.asterix.metadata.MetadataManager;
@@ -141,7 +139,6 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.IResultSerializerFactory;
@@ -360,13 +357,18 @@
}
public Dataset findDataset(DataverseName dataverseName, String datasetName) throws AlgebricksException {
+ return findDataset(dataverseName, datasetName, false);
+ }
+
+ public Dataset findDataset(DataverseName dataverseName, String datasetName, boolean includingViews)
+ throws AlgebricksException {
DataverseName dvName = getActiveDataverseName(dataverseName);
if (dvName == null) {
return null;
}
appCtx.getMetadataLockManager().acquireDataverseReadLock(locks, dvName);
appCtx.getMetadataLockManager().acquireDatasetReadLock(locks, dvName, datasetName);
- return MetadataManagerUtil.findDataset(mdTxnCtx, dvName, datasetName);
+ return MetadataManagerUtil.findDataset(mdTxnCtx, dvName, datasetName, includingViews);
}
public INodeDomain findNodeDomain(String nodeGroupName) throws AlgebricksException {
@@ -436,13 +438,13 @@
}
public Triple<DataverseName, String, Boolean> resolveDatasetNameUsingSynonyms(DataverseName dataverseName,
- String datasetName) throws AlgebricksException {
+ String datasetName, boolean includingViews) throws AlgebricksException {
DataverseName dvName = getActiveDataverseName(dataverseName);
if (dvName == null) {
return null;
}
Synonym synonym = null;
- while (MetadataManagerUtil.findDataset(mdTxnCtx, dvName, datasetName) == null) {
+ while (MetadataManagerUtil.findDataset(mdTxnCtx, dvName, datasetName, includingViews) == null) {
synonym = findSynonym(dvName, datasetName);
if (synonym == null) {
return null;
@@ -536,8 +538,9 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
- boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
- boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
+ boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory, Dataset dataset, String indexName,
+ int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+ boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch) throws AlgebricksException {
boolean isSecondary = true;
@@ -597,27 +600,28 @@
btreeSearchOp = !isSecondary && isPrimaryIndexPointSearch
? new LSMBTreeBatchPointSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
- retainMissing, context.getMissingWriterFactory(), searchCallbackFactory,
- minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
+ retainMissing, nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
+ maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
: new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
- context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, propagateFilter, tupleFilterFactory, outputLimit,
+ nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
+ propagateFilter, nonFilterWriterFactory, tupleFilterFactory, outputLimit,
proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
} else {
btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
- context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+ nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
+ ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
}
return new Pair<>(btreeSearchOp, spPc.second);
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
- JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
- int[] keyFields, boolean propagateFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
- boolean isIndexOnlyPlan) throws AlgebricksException {
+ JobGenContext context, boolean retainInput, boolean retainMissing,
+ IMissingWriterFactory nonMatchWriterFactory, Dataset dataset, String indexName, int[] keyFields,
+ boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
+ int[] maxFilterFieldIndexes, boolean isIndexOnlyPlan) throws AlgebricksException {
int numPrimaryKeys = dataset.getPrimaryKeys().size();
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), indexName);
@@ -641,8 +645,8 @@
ARecordType recType = (ARecordType) findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
List<List<String>> secondaryKeyFields = secondaryIndexDetails.getKeyFieldNames();
List<IAType> secondaryKeyTypes = secondaryIndexDetails.getKeyFieldTypes();
- Pair<IAType, Boolean> keyTypePair =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(secondaryIndex,
+ secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
IAType keyType = keyTypePair.first;
int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
int numNestedSecondaryKeyFields = numDimensions * 2;
@@ -663,13 +667,13 @@
new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
- indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+ indexDataflowHelperFactory, retainInput, retainMissing, nonMatchWriterFactory,
searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter,
- isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
+ nonFilterWriterFactory, isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
} else {
// Create the operator
rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
- indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+ indexDataflowHelperFactory, retainInput, retainMissing, nonMatchWriterFactory,
searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
}
@@ -781,7 +785,7 @@
boolean bulkload, List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
throws AlgebricksException {
return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema,
- inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
+ inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, null, recordDesc,
context, spec, bulkload, null, null, null, secondaryKeysPipelines, pipelineTopSchema);
}
@@ -794,7 +798,7 @@
List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
throws AlgebricksException {
return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema,
- inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
+ inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, null, recordDesc,
context, spec, false, null, null, null, secondaryKeysPipelines, pipelineTopSchema);
}
@@ -803,12 +807,13 @@
IDataSourceIndex<String, DataSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalFilteringKeys,
- ILogicalExpression filterExpr, LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
- LogicalVariable prevAdditionalFilteringKey, RecordDescriptor recordDesc, JobGenContext context,
- JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException {
+ ILogicalExpression filterExpr, ILogicalExpression prevFilterExpr, LogicalVariable operationVar,
+ List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKey,
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
+ List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException {
return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.UPSERT, dataSourceIndex, propagatedSchema,
- inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, recordDesc,
- context, spec, false, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKey,
+ inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, prevFilterExpr,
+ recordDesc, context, spec, false, operationVar, prevSecondaryKeys, prevAdditionalFilteringKey,
secondaryKeysPipelines, null);
}
@@ -986,11 +991,11 @@
List<LogicalVariable> primaryKeys, LogicalVariable payload, List<LogicalVariable> filterKeys,
List<LogicalVariable> additionalNonFilterFields, RecordDescriptor recordDesc, JobGenContext context,
JobSpecification spec) throws AlgebricksException {
+ DataverseName dataverseName = dataSource.getId().getDataverseName();
String datasetName = dataSource.getId().getDatasourceName();
- Dataset dataset = findDataset(dataSource.getId().getDataverseName(), datasetName);
+ Dataset dataset = findDataset(dataverseName, datasetName);
if (dataset == null) {
- throw new AlgebricksException("Unknown " + dataset() + " " + datasetName + " in " + dataverse() + " "
- + dataSource.getId().getDataverseName());
+ throw new AsterixException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, datasetName, dataverseName);
}
int numKeys = primaryKeys.size();
int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
@@ -1059,53 +1064,6 @@
return new Pair<>(dataScanner, constraint);
}
- private Pair<IBinaryComparatorFactory[], ITypeTraits[]> getComparatorFactoriesAndTypeTraitsOfSecondaryBTreeIndex(
- List<List<String>> sidxKeyFieldNames, List<IAType> sidxKeyFieldTypes, List<List<String>> pidxKeyFieldNames,
- ARecordType recType, DatasetType dsType, boolean hasMeta, List<Integer> primaryIndexKeyIndicators,
- List<Integer> secondaryIndexIndicators, ARecordType metaType) throws AlgebricksException {
-
- IBinaryComparatorFactory[] comparatorFactories;
- ITypeTraits[] typeTraits;
- int sidxKeyFieldCount = sidxKeyFieldNames.size();
- int pidxKeyFieldCount = pidxKeyFieldNames.size();
- typeTraits = new ITypeTraits[sidxKeyFieldCount + pidxKeyFieldCount];
- comparatorFactories = new IBinaryComparatorFactory[sidxKeyFieldCount + pidxKeyFieldCount];
-
- int i = 0;
- for (; i < sidxKeyFieldCount; ++i) {
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(sidxKeyFieldTypes.get(i), sidxKeyFieldNames.get(i),
- (hasMeta && secondaryIndexIndicators.get(i).intValue() == 1) ? metaType : recType);
- IAType keyType = keyPairType.first;
- comparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
- typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
- }
-
- for (int j = 0; j < pidxKeyFieldCount; ++j, ++i) {
- IAType keyType = null;
- try {
- switch (dsType) {
- case INTERNAL:
- keyType = (hasMeta && primaryIndexKeyIndicators.get(j).intValue() == 1)
- ? metaType.getSubFieldType(pidxKeyFieldNames.get(j))
- : recType.getSubFieldType(pidxKeyFieldNames.get(j));
- break;
- case EXTERNAL:
- keyType = IndexingConstants.getFieldType(j);
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE, dsType.toString());
- }
- } catch (AsterixException e) {
- throw new AlgebricksException(e);
- }
- comparatorFactories[i] = BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
- typeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(keyType);
- }
-
- return new Pair<>(comparatorFactories, typeTraits);
- }
-
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOperation indexOp,
IDataSource<DataSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields, RecordDescriptor inputRecordDesc,
@@ -1212,10 +1170,11 @@
IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr,
- RecordDescriptor inputRecordDesc, JobGenContext context, JobSpecification spec, boolean bulkload,
- LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
- LogicalVariable prevAdditionalFilteringKey, List<List<AlgebricksPipeline>> secondaryKeysPipelines,
- IOperatorSchema pipelineTopSchema) throws AlgebricksException {
+ ILogicalExpression prevFilterExpr, RecordDescriptor inputRecordDesc, JobGenContext context,
+ JobSpecification spec, boolean bulkload, LogicalVariable operationVar,
+ List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKey,
+ List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
+ throws AlgebricksException {
String indexName = dataSourceIndex.getId();
DataverseName dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
String datasetName = dataSourceIndex.getDataSource().getId().getDatasourceName();
@@ -1232,45 +1191,48 @@
// If we have a pipeline, then we need to pass the schema of the pipeline to the filter factory.
AsterixTupleFilterFactory filterFactory;
+ AsterixTupleFilterFactory prevFilterFactory;
if (pipelineTopSchema != null) {
IOperatorSchema[] schemasForFilterFactory = new IOperatorSchema[inputSchemas.length + 1];
System.arraycopy(inputSchemas, 0, schemasForFilterFactory, 0, inputSchemas.length);
schemasForFilterFactory[inputSchemas.length] = pipelineTopSchema;
filterFactory = createTupleFilterFactory(schemasForFilterFactory, typeEnv, filterExpr, context);
-
+ prevFilterFactory = createTupleFilterFactory(schemasForFilterFactory, typeEnv, prevFilterExpr, context);
} else {
filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+ prevFilterFactory = createTupleFilterFactory(inputSchemas, typeEnv, prevFilterExpr, context);
}
switch (secondaryIndex.getIndexType()) {
case BTREE:
return getBTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
- secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
- bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
+ secondaryKeys, additionalNonKeyFields, filterFactory, prevFilterFactory, inputRecordDesc,
+ context, spec, indexOp, bulkload, operationVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
case ARRAY:
if (bulkload) {
// In the case of bulk-load, we do not handle any nested plans. We perform the exact same behavior
// as a normal B-Tree bulk load.
return getBTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
- secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec,
- indexOp, bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
+ secondaryKeys, additionalNonKeyFields, filterFactory, prevFilterFactory, inputRecordDesc,
+ context, spec, indexOp, bulkload, operationVar, prevSecondaryKeys,
+ prevAdditionalFilteringKeys);
} else {
return getArrayIndexRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
- additionalNonKeyFields, inputRecordDesc, spec, indexOp, upsertIndicatorVar,
+ additionalNonKeyFields, inputRecordDesc, spec, indexOp, operationVar,
secondaryKeysPipelines);
}
case RTREE:
return getRTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
- secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
- bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
+ secondaryKeys, additionalNonKeyFields, filterFactory, prevFilterFactory, inputRecordDesc,
+ context, spec, indexOp, bulkload, operationVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
case SINGLE_PARTITION_WORD_INVIX:
case SINGLE_PARTITION_NGRAM_INVIX:
case LENGTH_PARTITIONED_WORD_INVIX:
case LENGTH_PARTITIONED_NGRAM_INVIX:
return getInvertedIndexRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
- secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
- secondaryIndex.getIndexType(), bulkload, upsertIndicatorVar, prevSecondaryKeys,
- prevAdditionalFilteringKeys);
+ secondaryKeys, additionalNonKeyFields, filterFactory, prevFilterFactory, inputRecordDesc,
+ context, spec, indexOp, secondaryIndex.getIndexType(), bulkload, operationVar,
+ prevSecondaryKeys, prevAdditionalFilteringKeys);
default:
throw new AlgebricksException(
indexOp.name() + " not implemented for index type: " + secondaryIndex.getIndexType());
@@ -1280,10 +1242,10 @@
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeRuntime(DataverseName dataverseName,
String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
- AsterixTupleFilterFactory filterFactory, RecordDescriptor inputRecordDesc, JobGenContext context,
- JobSpecification spec, IndexOperation indexOp, boolean bulkload, LogicalVariable upsertIndicatorVar,
- List<LogicalVariable> prevSecondaryKeys, List<LogicalVariable> prevAdditionalFilteringKeys)
- throws AlgebricksException {
+ AsterixTupleFilterFactory filterFactory, AsterixTupleFilterFactory prevFilterFactory,
+ RecordDescriptor inputRecordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
+ boolean bulkload, LogicalVariable operationVar, List<LogicalVariable> prevSecondaryKeys,
+ List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
int numKeys = primaryKeys.size() + secondaryKeys.size();
int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
@@ -1350,10 +1312,10 @@
StorageConstants.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, idfh, null,
BulkLoadUsage.LOAD, dataset.getDatasetId(), filterFactory);
} else if (indexOp == IndexOperation.UPSERT) {
- int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
+ int operationFieldIndex = propagatedSchema.findVariable(operationVar);
op = new LSMSecondaryUpsertOperatorDescriptor(spec, inputRecordDesc, fieldPermutation, idfh,
- filterFactory, modificationCallbackFactory, upsertIndicatorFieldIndex,
- BinaryBooleanInspector.FACTORY, prevFieldPermutation);
+ filterFactory, prevFilterFactory, modificationCallbackFactory, operationFieldIndex,
+ BinaryIntegerInspector.FACTORY, prevFieldPermutation);
} else {
op = new LSMTreeInsertDeleteOperatorDescriptor(spec, inputRecordDesc, fieldPermutation, indexOp, idfh,
filterFactory, false, modificationCallbackFactory);
@@ -1367,8 +1329,8 @@
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getArrayIndexRuntime(DataverseName dataverseName,
String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
List<LogicalVariable> additionalNonKeyFields, RecordDescriptor inputRecordDesc, JobSpecification spec,
- IndexOperation indexOp, LogicalVariable upsertIndicatorVar,
- List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException {
+ IndexOperation indexOp, LogicalVariable operationVar, List<List<AlgebricksPipeline>> secondaryKeysPipelines)
+ throws AlgebricksException {
Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
int numPrimaryKeys = primaryKeys.size();
@@ -1404,9 +1366,9 @@
storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
IOperatorDescriptor op;
if (indexOp == IndexOperation.UPSERT) {
- int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
+ int operationFieldIndex = propagatedSchema.findVariable(operationVar);
op = new LSMSecondaryUpsertWithNestedPlanOperatorDescriptor(spec, inputRecordDesc, fieldPermutation,
- idfh, modificationCallbackFactory, upsertIndicatorFieldIndex, BinaryBooleanInspector.FACTORY,
+ idfh, modificationCallbackFactory, operationFieldIndex, BinaryIntegerInspector.FACTORY,
secondaryKeysPipelines.get(0), secondaryKeysPipelines.get(1));
} else {
op = new LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor(spec, inputRecordDesc,
@@ -1421,10 +1383,10 @@
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeRuntime(DataverseName dataverseName,
String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
- AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
- JobSpecification spec, IndexOperation indexOp, boolean bulkload, LogicalVariable upsertIndicatorVar,
- List<LogicalVariable> prevSecondaryKeys, List<LogicalVariable> prevAdditionalFilteringKeys)
- throws AlgebricksException {
+ AsterixTupleFilterFactory filterFactory, AsterixTupleFilterFactory prevFilterFactory,
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
+ boolean bulkload, LogicalVariable operationVar, List<LogicalVariable> prevSecondaryKeys,
+ List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
String itemTypeName = dataset.getItemTypeName();
IAType itemType = MetadataManager.INSTANCE
@@ -1436,8 +1398,8 @@
Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
List<List<String>> secondaryKeyExprs = secondaryIndexDetails.getKeyFieldNames();
List<IAType> secondaryKeyTypes = secondaryIndexDetails.getKeyFieldTypes();
- Pair<IAType, Boolean> keyPairType =
- Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryIndex, secondaryKeyTypes.get(0),
+ secondaryKeyExprs.get(0), recType);
IAType spatialType = keyPairType.first;
int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
int numSecondaryKeys = dimension * 2;
@@ -1505,10 +1467,10 @@
StorageConstants.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false,
indexDataflowHelperFactory, null, BulkLoadUsage.LOAD, dataset.getDatasetId(), filterFactory);
} else if (indexOp == IndexOperation.UPSERT) {
- int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
+ int operationFieldIndex = propagatedSchema.findVariable(operationVar);
op = new LSMSecondaryUpsertOperatorDescriptor(spec, recordDesc, fieldPermutation,
- indexDataflowHelperFactory, filterFactory, modificationCallbackFactory, upsertIndicatorFieldIndex,
- BinaryBooleanInspector.FACTORY, prevFieldPermutation);
+ indexDataflowHelperFactory, filterFactory, prevFilterFactory, modificationCallbackFactory,
+ operationFieldIndex, BinaryIntegerInspector.FACTORY, prevFieldPermutation);
} else {
op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, fieldPermutation, indexOp,
indexDataflowHelperFactory, filterFactory, false, modificationCallbackFactory);
@@ -1520,10 +1482,10 @@
DataverseName dataverseName, String datasetName, String indexName, IOperatorSchema propagatedSchema,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
- RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
- IndexType indexType, boolean bulkload, LogicalVariable upsertIndicatorVar,
- List<LogicalVariable> prevSecondaryKeys, List<LogicalVariable> prevAdditionalFilteringKeys)
- throws AlgebricksException {
+ AsterixTupleFilterFactory prevFilterFactory, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec, IndexOperation indexOp, IndexType indexType, boolean bulkload,
+ LogicalVariable operationVar, List<LogicalVariable> prevSecondaryKeys,
+ List<LogicalVariable> prevAdditionalFilteringKeys) throws AlgebricksException {
// Check the index is length-partitioned or not.
boolean isPartitioned;
if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
@@ -1536,7 +1498,7 @@
// Sanity checks.
if (primaryKeys.size() > 1) {
throw new AlgebricksException(
- "Cannot create inverted index on " + dataset(PLURAL) + "with composite primary key.");
+ "Cannot create inverted index on " + dataset(PLURAL) + " with composite primary key.");
}
// The size of secondaryKeys can be two if it receives input from its
// TokenizeOperator- [token, number of token]
@@ -1618,10 +1580,10 @@
StorageConstants.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, indexDataFlowFactory,
null, BulkLoadUsage.LOAD, dataset.getDatasetId(), filterFactory);
} else if (indexOp == IndexOperation.UPSERT) {
- int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
+ int upsertOperationFieldIndex = propagatedSchema.findVariable(operationVar);
op = new LSMSecondaryUpsertOperatorDescriptor(spec, recordDesc, fieldPermutation, indexDataFlowFactory,
- filterFactory, modificationCallbackFactory, upsertIndicatorFieldIndex,
- BinaryBooleanInspector.FACTORY, prevFieldPermutation);
+ filterFactory, prevFilterFactory, modificationCallbackFactory, upsertOperationFieldIndex,
+ BinaryIntegerInspector.FACTORY, prevFieldPermutation);
} else {
op = new LSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc, fieldPermutation, indexOp,
indexDataFlowFactory, filterFactory, false, modificationCallbackFactory);
@@ -1748,8 +1710,8 @@
// return the derived type.
// e.g. UNORDERED LIST -> return UNORDERED LIST type
IAType secondaryKeyType;
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypeEntries.get(0),
- secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryIndex,
+ secondaryKeyTypeEntries.get(0), secondaryKeyExprs.get(0), recType);
secondaryKeyType = keyPairType.first;
List<List<String>> partitioningKeys = dataset.getPrimaryKeys();
i = 0;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 34e0c41..37e22ed 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -389,7 +389,7 @@
mdTxnCtx.setValue(MetadataManager.INSTANCE.beginTransaction());
bActiveTxn.setValue(true);
metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
- } else {
+ } else if (getDatasetType() == DatasetType.EXTERNAL) {
// External dataset
ExternalDatasetsRegistry.INSTANCE.removeDatasetInfo(this);
// #. prepare jobs to drop the datatset and the indexes in NC
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/DependencyKind.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/DependencyKind.java
new file mode 100644
index 0000000..19939f9
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/DependencyKind.java
@@ -0,0 +1,54 @@
+/*
+ * 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.metadata.entities;
+
+import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
+
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.metadata.utils.TypeUtil;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+
+public enum DependencyKind {
+ DATASET(dependency -> DatasetUtil.getFullyQualifiedDisplayName(dependency.first, dependency.second)),
+ FUNCTION(
+ dependency -> new FunctionSignature(dependency.first, dependency.second, Integer.parseInt(dependency.third))
+ .toString()),
+ TYPE(dependency -> TypeUtil.getFullyQualifiedDisplayName(dependency.first, dependency.second)),
+ SYNONYM(dependency -> MetadataUtil.getFullyQualifiedDisplayName(dependency.first, dependency.second));
+
+ private final java.util.function.Function<Triple<DataverseName, String, String>, String> dependencyDisplayNameAccessor;
+
+ DependencyKind(
+ java.util.function.Function<Triple<DataverseName, String, String>, String> dependencyDisplayNameAccessor) {
+ this.dependencyDisplayNameAccessor = dependencyDisplayNameAccessor;
+ }
+
+ public String getDependencyDisplayName(Triple<DataverseName, String, String> dependency) {
+ return dependencyDisplayNameAccessor.apply(dependency);
+ }
+
+ @Override
+ public String toString() {
+ return this == DATASET ? dataset() + " (or view)" : name().toLowerCase();
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java
index 3ee7106..9fba70a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java
@@ -24,6 +24,8 @@
import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
public class FullTextConfigMetadataEntity implements IMetadataEntity<FullTextConfigMetadataEntity> {
+ private static final long serialVersionUID = -8257829613982301855L;
+
private final FullTextConfigDescriptor fullTextConfig;
public FullTextConfigMetadataEntity(FullTextConfigDescriptor config) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java
index eb4c7bc..ca7184d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java
@@ -24,6 +24,8 @@
import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
public class FullTextFilterMetadataEntity implements IMetadataEntity<FullTextFilterMetadataEntity> {
+ private static final long serialVersionUID = 1442247723350731174L;
+
private final AbstractFullTextFilterDescriptor filterDescriptor;
public FullTextFilterMetadataEntity(AbstractFullTextFilterDescriptor filterDescriptor) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
index a9fa024..8853c02 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
@@ -18,8 +18,6 @@
*/
package org.apache.asterix.metadata.entities;
-import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
-
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -30,9 +28,6 @@
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.metadata.MetadataCache;
import org.apache.asterix.metadata.api.IMetadataEntity;
-import org.apache.asterix.metadata.utils.DatasetUtil;
-import org.apache.asterix.metadata.utils.MetadataUtil;
-import org.apache.asterix.metadata.utils.TypeUtil;
import org.apache.asterix.om.types.TypeSignature;
import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -166,12 +161,15 @@
return cache.dropFunction(this);
}
+ public static List<DependencyKind> DEPENDENCIES_SCHEMA =
+ Arrays.asList(DependencyKind.DATASET, DependencyKind.FUNCTION, DependencyKind.TYPE, DependencyKind.SYNONYM);
+
public static List<List<Triple<DataverseName, String, String>>> createDependencies(
List<Triple<DataverseName, String, String>> datasetDependencies,
List<Triple<DataverseName, String, String>> functionDependencies,
List<Triple<DataverseName, String, String>> typeDependencies,
List<Triple<DataverseName, String, String>> synonymDependencies) {
- List<List<Triple<DataverseName, String, String>>> depList = new ArrayList<>(4);
+ List<List<Triple<DataverseName, String, String>>> depList = new ArrayList<>(DEPENDENCIES_SCHEMA.size());
depList.add(datasetDependencies);
depList.add(functionDependencies);
depList.add(typeDependencies);
@@ -180,29 +178,4 @@
}
return depList;
}
-
- public enum FunctionDependencyKind {
- DATASET(dependency -> DatasetUtil.getFullyQualifiedDisplayName(dependency.first, dependency.second)),
- FUNCTION(
- dependency -> new FunctionSignature(dependency.first, dependency.second,
- Integer.parseInt(dependency.third)).toString()),
- TYPE(dependency -> TypeUtil.getFullyQualifiedDisplayName(dependency.first, dependency.second)),
- SYNONYM(dependency -> MetadataUtil.getFullyQualifiedDisplayName(dependency.first, dependency.second));
-
- private final java.util.function.Function<Triple<DataverseName, String, String>, String> dependencyDisplayNameAccessor;
-
- FunctionDependencyKind(
- java.util.function.Function<Triple<DataverseName, String, String>, String> dependencyDisplayNameAccessor) {
- this.dependencyDisplayNameAccessor = dependencyDisplayNameAccessor;
- }
-
- public String getDependencyDisplayName(Triple<DataverseName, String, String> dependency) {
- return dependencyDisplayNameAccessor.apply(dependency);
- }
-
- @Override
- public String toString() {
- return this == DATASET ? dataset() : name().toLowerCase();
- }
- }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index cf09779..9e67292 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -32,12 +32,14 @@
import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
import org.apache.asterix.metadata.MetadataCache;
import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.utils.IndexUtil;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.util.OptionalBoolean;
/**
* Metadata describing an index.
@@ -81,13 +83,23 @@
@Deprecated
public Index(DataverseName dataverseName, String datasetName, String indexName, IndexType indexType,
List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
- boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
- this(dataverseName,
- datasetName, indexName, indexType, createSimpleIndexDetails(indexType, keyFieldNames,
- keyFieldSourceIndicators, keyFieldTypes, overrideKeyFieldTypes),
+ boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex, int pendingOp,
+ OptionalBoolean excludeUnknownKey) {
+ this(dataverseName, datasetName,
+ indexName, indexType, createSimpleIndexDetails(indexType, keyFieldNames, keyFieldSourceIndicators,
+ keyFieldTypes, overrideKeyFieldTypes, excludeUnknownKey),
isEnforced, isPrimaryIndex, pendingOp);
}
+ public static Index createPrimaryIndex(DataverseName dataverseName, String datasetName,
+ List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
+ int pendingOp) {
+ return new Index(dataverseName, datasetName,
+ datasetName, IndexType.BTREE, new ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators,
+ keyFieldTypes, false, OptionalBoolean.empty(), OptionalBoolean.empty(), null, null, null),
+ false, true, pendingOp);
+ }
+
public DataverseName getDataverseName() {
return dataverseName;
}
@@ -143,8 +155,13 @@
return new Pair<>(actualKeyType, nullable);
}
- public static Pair<IAType, Boolean> getNonNullableOpenFieldType(IAType fieldType, List<String> fieldName,
- ARecordType recType) throws AlgebricksException {
+ public static Pair<IAType, Boolean> getNonNullableOpenFieldType(Index index, IAType fieldType,
+ List<String> fieldName, ARecordType recType) throws AlgebricksException {
+ if (IndexUtil.castDefaultNull(index)) {
+ Pair<IAType, Boolean> nonNullableType = getNonNullableType(fieldType);
+ nonNullableType.second = true;
+ return nonNullableType;
+ }
Pair<IAType, Boolean> keyPairType = null;
IAType subType = recType;
boolean nullable = false;
@@ -218,7 +235,7 @@
@Override
public int compareTo(Index otherIndex) {
- /** Gives a primary index first priority. */
+ /* Gives a primary index first priority. */
if (isPrimaryIndex && !otherIndex.isPrimaryIndex) {
return -1;
}
@@ -226,7 +243,7 @@
return 1;
}
- /** Gives a B-Tree index the second priority. */
+ /* Gives a B-Tree index the second priority. */
if (indexType == IndexType.BTREE && otherIndex.indexType != IndexType.BTREE) {
return -1;
}
@@ -234,7 +251,7 @@
return 1;
}
- /** Gives a R-Tree index the third priority */
+ /* Gives a R-Tree index the third priority */
if (indexType == IndexType.RTREE && otherIndex.indexType != IndexType.RTREE) {
return -1;
}
@@ -242,7 +259,7 @@
return 1;
}
- /** Finally, compares based on names. */
+ /* Finally, compares based on names. */
int result = indexName.compareTo(otherIndex.getIndexName());
if (result != 0) {
return result;
@@ -322,12 +339,29 @@
private final boolean overrideKeyFieldTypes;
+ private final Boolean excludeUnknownKey;
+
+ private final Boolean castDefaultNull;
+
+ private final String castDatetimeFormat;
+
+ private final String castDateFormat;
+
+ private final String castTimeFormat;
+
public ValueIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
- List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes) {
+ List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes, OptionalBoolean excludeUnknownKey,
+ OptionalBoolean castDefaultNull, String castDatetimeFormat, String castDateFormat,
+ String castTimeFormat) {
this.keyFieldNames = keyFieldNames;
this.keyFieldSourceIndicators = keyFieldSourceIndicators;
this.keyFieldTypes = keyFieldTypes;
this.overrideKeyFieldTypes = overrideKeyFieldTypes;
+ this.excludeUnknownKey = excludeUnknownKey.isEmpty() ? null : excludeUnknownKey.get();
+ this.castDefaultNull = castDefaultNull.isEmpty() ? null : castDefaultNull.get();
+ this.castDatetimeFormat = castDatetimeFormat;
+ this.castDateFormat = castDateFormat;
+ this.castTimeFormat = castTimeFormat;
}
@Override
@@ -347,6 +381,26 @@
return keyFieldTypes;
}
+ public OptionalBoolean getExcludeUnknownKey() {
+ return OptionalBoolean.ofNullable(excludeUnknownKey);
+ }
+
+ public OptionalBoolean getCastDefaultNull() {
+ return OptionalBoolean.ofNullable(castDefaultNull);
+ }
+
+ public String getCastDatetimeFormat() {
+ return castDatetimeFormat;
+ }
+
+ public String getCastDateFormat() {
+ return castDateFormat;
+ }
+
+ public String getCastTimeFormat() {
+ return castTimeFormat;
+ }
+
@Override
public boolean isOverridingKeyFieldTypes() {
return overrideKeyFieldTypes;
@@ -481,15 +535,19 @@
@Deprecated
private static Index.IIndexDetails createSimpleIndexDetails(IndexType indexType, List<List<String>> keyFieldNames,
- List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes) {
+ List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes,
+ OptionalBoolean excludeUnknownKey) {
if (indexType == null) {
return null;
}
switch (Index.IndexCategory.of(indexType)) {
case VALUE:
return new ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
- overrideKeyFieldTypes);
+ overrideKeyFieldTypes, excludeUnknownKey, OptionalBoolean.empty(), null, null, null);
case TEXT:
+ if (excludeUnknownKey.isPresent()) {
+ throw new IllegalArgumentException("excludeUnknownKey");
+ }
return new TextIndexDetails(keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
overrideKeyFieldTypes, -1, null);
default:
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ViewDetails.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ViewDetails.java
new file mode 100644
index 0000000..76c6b4d
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/ViewDetails.java
@@ -0,0 +1,347 @@
+/*
+ * 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.metadata.entities;
+
+import static org.apache.asterix.metadata.entitytupletranslators.AbstractTupleTranslator.writeDateTimeFormats;
+import static org.apache.asterix.om.types.AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
+
+import java.io.DataOutput;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.metadata.IDatasetDetails;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entitytupletranslators.AbstractTupleTranslator;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public class ViewDetails implements IDatasetDetails {
+
+ private static final long serialVersionUID = 1L;
+
+ public static List<DependencyKind> DEPENDENCIES_SCHEMA =
+ Arrays.asList(DependencyKind.DATASET, DependencyKind.FUNCTION, DependencyKind.TYPE, DependencyKind.SYNONYM);
+
+ private final String viewBody;
+
+ private final List<List<Triple<DataverseName, String, String>>> dependencies;
+
+ // Typed view parameters
+
+ private final Boolean defaultNull;
+
+ private final String datetimeFormat;
+
+ private final String dateFormat;
+
+ private final String timeFormat;
+
+ private final List<String> primaryKeyFields;
+
+ private final List<ForeignKey> foreignKeys;
+
+ public ViewDetails(String viewBody, List<List<Triple<DataverseName, String, String>>> dependencies,
+ Boolean defaultNull, List<String> primaryKeyFields, List<ForeignKey> foreignKeys, String datetimeFormat,
+ String dateFormat, String timeFormat) {
+ this.viewBody = Objects.requireNonNull(viewBody);
+ this.dependencies = Objects.requireNonNull(dependencies);
+ this.defaultNull = defaultNull;
+ this.primaryKeyFields = primaryKeyFields;
+ this.foreignKeys = foreignKeys;
+ this.datetimeFormat = datetimeFormat;
+ this.dateFormat = dateFormat;
+ this.timeFormat = timeFormat;
+ }
+
+ @Override
+ public DatasetConfig.DatasetType getDatasetType() {
+ return DatasetConfig.DatasetType.VIEW;
+ }
+
+ public String getViewBody() {
+ return viewBody;
+ }
+
+ public List<List<Triple<DataverseName, String, String>>> getDependencies() {
+ return dependencies;
+ }
+
+ // Typed view fields
+
+ public Boolean getDefaultNull() {
+ return defaultNull;
+ }
+
+ public List<String> getPrimaryKeyFields() {
+ return primaryKeyFields;
+ }
+
+ public List<ForeignKey> getForeignKeys() {
+ return foreignKeys;
+ }
+
+ public String getDatetimeFormat() {
+ return datetimeFormat;
+ }
+
+ public String getDateFormat() {
+ return dateFormat;
+ }
+
+ public String getTimeFormat() {
+ return timeFormat;
+ }
+
+ @Override
+ public void writeDatasetDetailsRecordType(DataOutput out) throws HyracksDataException {
+ IARecordBuilder viewRecordBuilder = new RecordBuilder();
+ viewRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+
+ ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+
+ AMutableString aString = new AMutableString("");
+ ISerializerDeserializer<AString> stringSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+ ISerializerDeserializer<ABoolean> booleanSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+ ISerializerDeserializer<ANull> nullSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+ // write field 'Definition'
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_DEFINITION);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ aString.setValue(viewBody);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ viewRecordBuilder.addField(fieldName, fieldValue);
+
+ // write field 'Dependencies'
+ if (!dependencies.isEmpty()) {
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_DEPENDENCIES);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+
+ OrderedListBuilder dependenciesListBuilder = new OrderedListBuilder();
+ OrderedListBuilder dependencyListBuilder = new OrderedListBuilder();
+ OrderedListBuilder dependencyNameListBuilder = new OrderedListBuilder();
+ List<String> dependencySubnames = new ArrayList<>(3);
+
+ dependenciesListBuilder.reset(FULL_OPEN_ORDEREDLIST_TYPE);
+ for (List<Triple<DataverseName, String, String>> dependenciesList : dependencies) {
+ dependencyListBuilder.reset(FULL_OPEN_ORDEREDLIST_TYPE);
+ for (Triple<DataverseName, String, String> dependency : dependenciesList) {
+ dependencyNameListBuilder.reset(FULL_OPEN_ORDEREDLIST_TYPE);
+ dependencySubnames.clear();
+ AbstractTupleTranslator.getDependencySubNames(dependency, dependencySubnames);
+ for (String subName : dependencySubnames) {
+ itemValue.reset();
+ aString.setValue(subName);
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ dependencyNameListBuilder.addItem(itemValue);
+ }
+ itemValue.reset();
+ dependencyNameListBuilder.write(itemValue.getDataOutput(), true);
+ dependencyListBuilder.addItem(itemValue);
+ }
+ itemValue.reset();
+ dependencyListBuilder.write(itemValue.getDataOutput(), true);
+ dependenciesListBuilder.addItem(itemValue);
+ }
+ fieldValue.reset();
+ dependenciesListBuilder.write(fieldValue.getDataOutput(), true);
+ viewRecordBuilder.addField(fieldName, fieldValue);
+ }
+
+ // write field 'DefaultNull'
+ if (defaultNull != null && defaultNull) {
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_DEFAULT);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ nullSerde.serialize(ANull.NULL, fieldValue.getDataOutput());
+ viewRecordBuilder.addField(fieldName, fieldValue);
+ }
+
+ // write field 'PrimaryKey'
+ if (primaryKeyFields != null && !primaryKeyFields.isEmpty()) {
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_PRIMARY_KEY);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+
+ // write value as list of lists to be consistent with how InternalDatasetDetails writes its primary key
+ fieldValue.reset();
+ OrderedListBuilder keyListBuilder = new OrderedListBuilder();
+ OrderedListBuilder fieldPathListBuilder = new OrderedListBuilder();
+ writeKeyFieldsList(primaryKeyFields, keyListBuilder, fieldPathListBuilder, aString, stringSerde, itemValue);
+ keyListBuilder.write(fieldValue.getDataOutput(), true);
+ viewRecordBuilder.addField(fieldName, fieldValue);
+
+ // write field 'PrimaryKeyEnforced'
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_PRIMARY_KEY_ENFORCED);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ booleanSerde.serialize(ABoolean.FALSE, fieldValue.getDataOutput());
+ viewRecordBuilder.addField(fieldName, fieldValue);
+ }
+
+ // write field 'ForeignKeys'
+ if (foreignKeys != null && !foreignKeys.isEmpty()) {
+ OrderedListBuilder foreignKeysListBuilder = new OrderedListBuilder();
+ foreignKeysListBuilder.reset(FULL_OPEN_ORDEREDLIST_TYPE);
+
+ IARecordBuilder foreignKeyRecordBuilder = new RecordBuilder();
+ OrderedListBuilder keyListBuilder = new OrderedListBuilder();
+ OrderedListBuilder fieldPathListBuilder = new OrderedListBuilder();
+
+ for (ViewDetails.ForeignKey foreignKey : foreignKeys) {
+ foreignKeyRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+
+ // write field 'ForeignKey'
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_FOREIGN_KEY);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ // write value as list of lists to be consistent with how InternalDatasetDetails writes its primary key
+ fieldValue.reset();
+ writeKeyFieldsList(foreignKey.getForeignKeyFields(), keyListBuilder, fieldPathListBuilder, aString,
+ stringSerde, itemValue);
+ keyListBuilder.write(fieldValue.getDataOutput(), true);
+ foreignKeyRecordBuilder.addField(fieldName, fieldValue);
+
+ // write field 'RefDataverseName'
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_REF_DATAVERSE_NAME);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ aString.setValue(foreignKey.getReferencedDatasetName().getDataverseName().getCanonicalForm());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ foreignKeyRecordBuilder.addField(fieldName, fieldValue);
+
+ // write field 'RefDatasetName'
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_REF_DATASET_NAME);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ aString.setValue(foreignKey.getReferencedDatasetName().getDatasetName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ foreignKeyRecordBuilder.addField(fieldName, fieldValue);
+
+ // write field 'IsEnforced'
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_IS_ENFORCED);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ booleanSerde.serialize(ABoolean.FALSE, fieldValue.getDataOutput());
+ foreignKeyRecordBuilder.addField(fieldName, fieldValue);
+
+ fieldValue.reset();
+ foreignKeyRecordBuilder.write(fieldValue.getDataOutput(), true);
+ foreignKeysListBuilder.addItem(fieldValue);
+ }
+
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_FOREIGN_KEYS);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ foreignKeysListBuilder.write(fieldValue.getDataOutput(), true);
+
+ viewRecordBuilder.addField(fieldName, fieldValue);
+ }
+
+ // write field 'Format'
+ writeDateTimeFormats(datetimeFormat, dateFormat, timeFormat, viewRecordBuilder, aString, nullSerde, stringSerde,
+ fieldName, fieldValue, itemValue);
+ viewRecordBuilder.write(out, true);
+ }
+
+ private void writeKeyFieldsList(List<String> keyFields, OrderedListBuilder keyListBuilder,
+ OrderedListBuilder fieldListBuilder, AMutableString aString, ISerializerDeserializer<AString> stringSerde,
+ ArrayBackedValueStorage itemValue) throws HyracksDataException {
+ keyListBuilder.reset(FULL_OPEN_ORDEREDLIST_TYPE);
+ for (String field : keyFields) {
+ fieldListBuilder.reset(FULL_OPEN_ORDEREDLIST_TYPE);
+ itemValue.reset();
+ aString.setValue(field);
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ fieldListBuilder.addItem(itemValue);
+ itemValue.reset();
+ fieldListBuilder.write(itemValue.getDataOutput(), true);
+ keyListBuilder.addItem(itemValue);
+ }
+ }
+
+ public static List<List<Triple<DataverseName, String, String>>> createDependencies(
+ List<Triple<DataverseName, String, String>> datasetDependencies,
+ List<Triple<DataverseName, String, String>> functionDependencies,
+ List<Triple<DataverseName, String, String>> typeDependencies,
+ List<Triple<DataverseName, String, String>> synonymDependencies) {
+ List<List<Triple<DataverseName, String, String>>> depList = new ArrayList<>(DEPENDENCIES_SCHEMA.size());
+ depList.add(datasetDependencies);
+ depList.add(functionDependencies);
+ depList.add(typeDependencies);
+ if (!synonymDependencies.isEmpty()) {
+ depList.add(synonymDependencies);
+ }
+ return depList;
+ }
+
+ public static final class ForeignKey implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final List<String> foreignKeyFields;
+
+ private final DatasetFullyQualifiedName referencedDatasetName;
+
+ public ForeignKey(List<String> foreignKeyFields, DatasetFullyQualifiedName referencedDatasetName) {
+ this.foreignKeyFields = Objects.requireNonNull(foreignKeyFields);
+ this.referencedDatasetName = Objects.requireNonNull(referencedDatasetName);
+ }
+
+ public List<String> getForeignKeyFields() {
+ return foreignKeyFields;
+ }
+
+ public DatasetFullyQualifiedName getReferencedDatasetName() {
+ return referencedDatasetName;
+ }
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
index a35246f..3f4e63e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/AbstractTupleTranslator.java
@@ -19,25 +19,36 @@
package org.apache.asterix.metadata.entitytupletranslators;
+import static org.apache.asterix.om.types.AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
+
import java.io.ByteArrayInputStream;
import java.io.DataInput;
import java.io.DataInputStream;
+import java.util.Collection;
import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.api.IMetadataEntityTupleTranslator;
import org.apache.asterix.metadata.api.IMetadataIndex;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableString;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -113,4 +124,79 @@
protected abstract T createMetadataEntityFromARecord(ARecord aRecord)
throws HyracksDataException, AlgebricksException;
+
+ public static void getDependencySubNames(Triple<DataverseName, String, String> dependency,
+ Collection<? super String> outSubnames) {
+ outSubnames.add(dependency.first.getCanonicalForm());
+ if (dependency.second != null) {
+ outSubnames.add(dependency.second);
+ }
+ if (dependency.third != null) {
+ outSubnames.add(dependency.third);
+ }
+ }
+
+ public static Triple<DataverseName, String, String> getDependency(AOrderedList dependencySubnames)
+ throws AlgebricksException {
+ String dataverseCanonicalName = ((AString) dependencySubnames.getItem(0)).getStringValue();
+ DataverseName dataverseName = DataverseName.createFromCanonicalForm(dataverseCanonicalName);
+ String second = null, third = null;
+ int ln = dependencySubnames.size();
+ if (ln > 1) {
+ second = ((AString) dependencySubnames.getItem(1)).getStringValue();
+ if (ln > 2) {
+ third = ((AString) dependencySubnames.getItem(2)).getStringValue();
+ }
+ }
+ return new Triple<>(dataverseName, second, third);
+ }
+
+ protected static String getStringValue(IAObject obj) {
+ return obj.getType().getTypeTag() == ATypeTag.STRING ? ((AString) obj).getStringValue() : null;
+ }
+
+ protected static Triple<String, String, String> getDateTimeFormats(ARecord record) {
+ Triple<String, String, String> formats = new Triple<>(null, null, null);
+ int formatFieldPos = record.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_DATA_FORMAT);
+ if (formatFieldPos >= 0) {
+ IACursor formatCursor = ((AOrderedList) record.getValueByPos(formatFieldPos)).getCursor();
+ if (formatCursor.next()) {
+ formats.first = getStringValue(formatCursor.get());
+ if (formatCursor.next()) {
+ formats.second = getStringValue(formatCursor.get());
+ if (formatCursor.next()) {
+ formats.third = getStringValue(formatCursor.get());
+ }
+ }
+ }
+ }
+ return formats;
+ }
+
+ public static void writeDateTimeFormats(String datetimeFormat, String dateFormat, String timeFormat,
+ IARecordBuilder recordBuilder, AMutableString aString, ISerializerDeserializer<ANull> nullSerde,
+ ISerializerDeserializer<AString> stringSerde, ArrayBackedValueStorage nameValue,
+ ArrayBackedValueStorage fieldValue, ArrayBackedValueStorage itemValue) throws HyracksDataException {
+ if (datetimeFormat != null || dateFormat != null || timeFormat != null) {
+ nameValue.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_DATA_FORMAT);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+
+ OrderedListBuilder formatListBuilder = new OrderedListBuilder();
+ formatListBuilder.reset(FULL_OPEN_ORDEREDLIST_TYPE);
+ for (String format : new String[] { datetimeFormat, dateFormat, timeFormat }) {
+ itemValue.reset();
+ if (format == null) {
+ nullSerde.serialize(ANull.NULL, itemValue.getDataOutput());
+ } else {
+ aString.setValue(format);
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ }
+ formatListBuilder.addItem(itemValue);
+ }
+ fieldValue.reset();
+ formatListBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(nameValue, fieldValue);
+ }
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 19722ef..0d05d46 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -35,6 +35,9 @@
import org.apache.asterix.builders.UnorderedListBuilder;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.TransactionState;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
@@ -44,6 +47,7 @@
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
import org.apache.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.metadata.entities.ViewDetails;
import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.ADateTime;
@@ -58,15 +62,18 @@
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.AUnorderedList;
import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnorderedListType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.compression.CompressionManager;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -200,7 +207,7 @@
break;
}
- case EXTERNAL:
+ case EXTERNAL: {
ARecord datasetDetailsRecord = (ARecord) datasetRecord
.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX);
String adapter = ((AString) datasetDetailsRecord
@@ -229,6 +236,121 @@
.getIntegerValue()];
datasetDetails = new ExternalDatasetDetails(adapter, properties, timestamp, state);
+ break;
+ }
+ case VIEW: {
+ int datasetDetailsFieldPos =
+ datasetRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_VIEW_DETAILS);
+ ARecord datasetDetailsRecord = (ARecord) datasetRecord.getValueByPos(datasetDetailsFieldPos);
+
+ // Definition
+ int definitionFieldPos =
+ datasetDetailsRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_DEFINITION);
+ String definition = ((AString) datasetDetailsRecord.getValueByPos(definitionFieldPos)).getStringValue();
+
+ // Dependencies
+ List<List<Triple<DataverseName, String, String>>> dependencies = Collections.emptyList();
+ int dependenciesFieldPos =
+ datasetDetailsRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_DEPENDENCIES);
+ if (dependenciesFieldPos >= 0) {
+ dependencies = new ArrayList<>();
+ IACursor dependenciesCursor =
+ ((AOrderedList) datasetDetailsRecord.getValueByPos(dependenciesFieldPos)).getCursor();
+ while (dependenciesCursor.next()) {
+ List<Triple<DataverseName, String, String>> dependencyList = new ArrayList<>();
+ IACursor qualifiedDependencyCursor = ((AOrderedList) dependenciesCursor.get()).getCursor();
+ while (qualifiedDependencyCursor.next()) {
+ Triple<DataverseName, String, String> dependency =
+ getDependency((AOrderedList) qualifiedDependencyCursor.get());
+ dependencyList.add(dependency);
+ }
+ dependencies.add(dependencyList);
+ }
+ }
+
+ // Default Null
+ Boolean defaultNull = null;
+ int defaultFieldPos =
+ datasetDetailsRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_DEFAULT);
+ if (defaultFieldPos >= 0) {
+ IAObject defaultValue = datasetDetailsRecord.getValueByPos(defaultFieldPos);
+ defaultNull = defaultValue.getType().getTypeTag() == ATypeTag.NULL;
+ }
+
+ // Primary Key
+ List<String> primaryKeyFields = null;
+ int primaryKeyFieldPos =
+ datasetDetailsRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_PRIMARY_KEY);
+ if (primaryKeyFieldPos >= 0) {
+ AOrderedList primaryKeyFieldList =
+ ((AOrderedList) datasetDetailsRecord.getValueByPos(primaryKeyFieldPos));
+ int n = primaryKeyFieldList.size();
+ primaryKeyFields = new ArrayList<>(n);
+ for (int i = 0; i < n; i++) {
+ AOrderedList list = (AOrderedList) primaryKeyFieldList.getItem(i);
+ if (list.size() != 1) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, list.toJSON());
+ }
+ AString str = (AString) list.getItem(0);
+ primaryKeyFields.add(str.getStringValue());
+ }
+ }
+
+ // Foreign Keys
+ List<ViewDetails.ForeignKey> foreignKeys = null;
+ int foreignKeysFieldPos =
+ datasetDetailsRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_FOREIGN_KEYS);
+ if (foreignKeysFieldPos >= 0) {
+ AOrderedList foreignKeyRecordsList =
+ ((AOrderedList) datasetDetailsRecord.getValueByPos(foreignKeysFieldPos));
+ int nForeignKeys = foreignKeyRecordsList.size();
+ foreignKeys = new ArrayList<>(nForeignKeys);
+ for (int i = 0; i < nForeignKeys; i++) {
+ ARecord foreignKeyRecord = (ARecord) foreignKeyRecordsList.getItem(i);
+ // 'ForeignKey'
+ int foreignKeyFieldPos =
+ foreignKeyRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_FOREIGN_KEY);
+ AOrderedList foreignKeyFieldList =
+ ((AOrderedList) foreignKeyRecord.getValueByPos(foreignKeyFieldPos));
+ int nForeignKeyFields = foreignKeyFieldList.size();
+ List<String> foreignKeyFields = new ArrayList<>(nForeignKeyFields);
+ for (int j = 0; j < nForeignKeyFields; j++) {
+ AOrderedList list = (AOrderedList) foreignKeyFieldList.getItem(j);
+ if (list.size() != 1) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, list.toJSON());
+ }
+ AString str = (AString) list.getItem(0);
+ foreignKeyFields.add(str.getStringValue());
+ }
+
+ // 'RefDataverseName'
+ int refDataverseNameFieldPos = foreignKeyRecord.getType()
+ .getFieldIndex(MetadataRecordTypes.FIELD_NAME_REF_DATAVERSE_NAME);
+ String refDataverseCanonicalName =
+ ((AString) foreignKeyRecord.getValueByPos(refDataverseNameFieldPos)).getStringValue();
+ DataverseName refDataverseName =
+ DataverseName.createFromCanonicalForm(refDataverseCanonicalName);
+
+ // 'RefDatasetName'
+ int refDatasetNameFieldPos = foreignKeyRecord.getType()
+ .getFieldIndex(MetadataRecordTypes.FIELD_NAME_REF_DATASET_NAME);
+ String refDatasetName =
+ ((AString) foreignKeyRecord.getValueByPos(refDatasetNameFieldPos)).getStringValue();
+
+ foreignKeys.add(new ViewDetails.ForeignKey(foreignKeyFields,
+ new DatasetFullyQualifiedName(refDataverseName, refDatasetName)));
+ }
+ }
+
+ // Format fields
+ Triple<String, String, String> dateTimeFormats = getDateTimeFormats(datasetDetailsRecord);
+ String datetimeFormat = dateTimeFormats.first;
+ String dateFormat = dateTimeFormats.second;
+ String timeFormat = dateTimeFormats.third;
+ datasetDetails = new ViewDetails(definition, dependencies, defaultNull, primaryKeyFields, foreignKeys,
+ datetimeFormat, dateFormat, timeFormat);
+ break;
+ }
}
Map<String, String> hints = getDatasetHints(datasetRecord);
@@ -359,8 +481,19 @@
dataset.getCompactionPolicyProperties(), listBuilder, itemValue);
// write field 8/9
- fieldValue.reset();
- writeDatasetDetailsRecordType(recordBuilder, dataset, fieldValue.getDataOutput());
+ switch (dataset.getDatasetType()) {
+ case INTERNAL:
+ fieldValue.reset();
+ dataset.getDatasetDetails().writeDatasetDetailsRecordType(fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX, fieldValue);
+ break;
+ case EXTERNAL:
+ fieldValue.reset();
+ dataset.getDatasetDetails().writeDatasetDetailsRecordType(fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX, fieldValue);
+ break;
+ // VIEW details are written later by {@code writeOpenFields()}
+ }
// write field 10
UnorderedListBuilder uListBuilder = new UnorderedListBuilder();
@@ -441,6 +574,19 @@
writeMetaPart(dataset);
writeRebalanceCount(dataset);
writeBlockLevelStorageCompression(dataset);
+ writeOpenDetails(dataset);
+ }
+
+ private void writeOpenDetails(Dataset dataset) throws HyracksDataException {
+ if (dataset.getDatasetType() == DatasetType.VIEW) {
+ // write ViewDetails field
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_VIEW_DETAILS);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ dataset.getDatasetDetails().writeDatasetDetailsRecordType(fieldValue.getDataOutput());
+ recordBuilder.addField(fieldName, fieldValue);
+ }
}
private void writeMetaPart(Dataset dataset) throws HyracksDataException {
@@ -500,20 +646,6 @@
}
}
- protected void writeDatasetDetailsRecordType(IARecordBuilder recordBuilder, Dataset dataset, DataOutput dataOutput)
- throws HyracksDataException {
- dataset.getDatasetDetails().writeDatasetDetailsRecordType(dataOutput);
- switch (dataset.getDatasetType()) {
- case INTERNAL:
- recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX, fieldValue);
- break;
- case EXTERNAL:
- recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX, fieldValue);
- break;
- }
-
- }
-
protected Map<String, String> getDatasetHints(ARecord datasetRecord) {
Map<String, String> hints = new HashMap<>();
String key;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
index ccae8a4..ef9c143 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FunctionTupleTranslator.java
@@ -244,21 +244,6 @@
return new TypeSignature(typeDataverseName, typeName);
}
- private Triple<DataverseName, String, String> getDependency(AOrderedList dependencySubnames)
- throws AlgebricksException {
- String dataverseCanonicalName = ((AString) dependencySubnames.getItem(0)).getStringValue();
- DataverseName dataverseName = DataverseName.createFromCanonicalForm(dataverseCanonicalName);
- String second = null, third = null;
- int ln = dependencySubnames.size();
- if (ln > 1) {
- second = ((AString) dependencySubnames.getItem(1)).getStringValue();
- if (ln > 2) {
- third = ((AString) dependencySubnames.getItem(2)).getStringValue();
- }
- }
- return new Triple<>(dataverseName, second, third);
- }
-
private Map<String, String> getResources(ARecord functionRecord, String resourcesFieldName) {
Map<String, String> adaptorConfiguration = null;
final ARecordType functionType = functionRecord.getType();
@@ -381,11 +366,14 @@
dependenciesListBuilder.reset((AOrderedListType) MetadataRecordTypes.FUNCTION_RECORDTYPE
.getFieldTypes()[MetadataRecordTypes.FUNCTION_ARECORD_FUNCTION_DEPENDENCIES_FIELD_INDEX]);
List<List<Triple<DataverseName, String, String>>> dependenciesList = function.getDependencies();
+ List<String> subNames = new ArrayList<>();
for (List<Triple<DataverseName, String, String>> dependencies : dependenciesList) {
dependencyListBuilder.reset(listOfLists);
for (Triple<DataverseName, String, String> dependency : dependencies) {
dependencyNameListBuilder.reset(stringList);
- for (String subName : getDependencySubNames(dependency)) {
+ subNames.clear();
+ getDependencySubNames(dependency, subNames);
+ for (String subName : subNames) {
itemValue.reset();
aString.setValue(subName);
stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -611,18 +599,6 @@
propertyRecordBuilder.write(out, true);
}
- private List<String> getDependencySubNames(Triple<DataverseName, String, String> dependency) {
- dependencySubnames.clear();
- dependencySubnames.add(dependency.first.getCanonicalForm());
- if (dependency.second != null) {
- dependencySubnames.add(dependency.second);
- }
- if (dependency.third != null) {
- dependencySubnames.add(dependency.third);
- }
- return dependencySubnames;
- }
-
// back-compat
private static List<String> decodeExternalIdentifierBackCompat(String encodedValue,
ExternalFunctionLanguage language) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 32764b2..bb51748 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -19,6 +19,9 @@
package org.apache.asterix.metadata.entitytupletranslators;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FIELD_NAME_CAST;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FIELD_NAME_DEFAULT;
+
import java.util.ArrayList;
import java.util.Calendar;
import java.util.Collections;
@@ -65,6 +68,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.OptionalBoolean;
import com.google.common.base.Strings;
@@ -81,6 +85,7 @@
public static final String FULL_TEXT_CONFIG_FIELD_NAME = "FullTextConfig";
public static final String INDEX_SEARCHKEY_TYPE_FIELD_NAME = "SearchKeyType";
public static final String INDEX_ISENFORCED_FIELD_NAME = "IsEnforced";
+ public static final String INDEX_EXCLUDE_UNKNOWN_FIELD_NAME = "ExcludeUnknownKey";
public static final String INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME = "SearchKeySourceIndicator";
public static final String INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME = "SearchKeyElements";
public static final String COMPLEXSEARCHKEY_UNNEST_FIELD_NAME = "UnnestList";
@@ -94,6 +99,7 @@
protected OrderedListBuilder primaryKeyListBuilder;
protected OrderedListBuilder complexSearchKeyNameListBuilder;
protected IARecordBuilder complexSearchKeyNameRecordBuilder;
+ protected IARecordBuilder castRecordBuilder;
protected AOrderedListType stringList;
protected AOrderedListType int8List;
protected ArrayBackedValueStorage nameValue;
@@ -112,6 +118,7 @@
innerListBuilder = new OrderedListBuilder();
primaryKeyListBuilder = new OrderedListBuilder();
complexSearchKeyNameRecordBuilder = new RecordBuilder();
+ castRecordBuilder = new RecordBuilder();
complexSearchKeyNameListBuilder = new OrderedListBuilder();
stringList = new AOrderedListType(BuiltinType.ASTRING, null);
int8List = new AOrderedListType(BuiltinType.AINT8, null);
@@ -138,6 +145,9 @@
IndexType indexType = IndexType.valueOf(
((AString) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_INDEXSTRUCTURE_FIELD_INDEX))
.getStringValue());
+ boolean isPrimaryIndex =
+ ((ABoolean) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
+ .getBoolean();
// Read key names
List<Pair<List<List<String>>, List<List<String>>>> searchElements = new ArrayList<>();
@@ -377,8 +387,46 @@
List<List<String>> keyFieldNames =
searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
List<IAType> keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+
+ OptionalBoolean excludeUnknownKey = OptionalBoolean.empty();
+ OptionalBoolean castDefaultNull = OptionalBoolean.empty();
+ String datetimeFormat = null, dateFormat = null, timeFormat = null;
+ boolean isBtreeIdx = indexType == IndexType.BTREE && !isPrimaryIndex && !keyFieldNames.isEmpty();
+ if (isBtreeIdx) {
+ // exclude unknown key value; default to always include unknowns for normal b-trees
+ excludeUnknownKey = OptionalBoolean.FALSE();
+ int excludeUnknownKeyPos = indexRecord.getType().getFieldIndex(INDEX_EXCLUDE_UNKNOWN_FIELD_NAME);
+ if (excludeUnknownKeyPos >= 0) {
+ excludeUnknownKey = OptionalBoolean
+ .of(((ABoolean) indexRecord.getValueByPos(excludeUnknownKeyPos)).getBoolean());
+ }
+ // cast record
+ int castPos = indexRecord.getType().getFieldIndex(FIELD_NAME_CAST);
+ if (castPos >= 0) {
+ IAObject recValue = indexRecord.getValueByPos(castPos);
+ if (recValue.getType().getTypeTag() == ATypeTag.OBJECT) {
+ ARecord castRec = (ARecord) recValue;
+ ARecordType castRecType = castRec.getType();
+ // cast default value
+ int defaultFieldPos = castRecType.getFieldIndex(FIELD_NAME_DEFAULT);
+ if (defaultFieldPos >= 0) {
+ IAObject defaultVal = castRec.getValueByPos(defaultFieldPos);
+ if (defaultVal.getType().getTypeTag() == ATypeTag.NULL) {
+ castDefaultNull = OptionalBoolean.TRUE();
+
+ // Format fields
+ Triple<String, String, String> dateTimeFormats = getDateTimeFormats(castRec);
+ datetimeFormat = dateTimeFormats.first;
+ dateFormat = dateTimeFormats.second;
+ timeFormat = dateTimeFormats.third;
+ }
+ }
+ }
+ }
+ }
indexDetails = new Index.ValueIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
- isOverridingKeyTypes);
+ isOverridingKeyTypes, excludeUnknownKey, castDefaultNull, datetimeFormat, dateFormat,
+ timeFormat);
break;
case TEXT:
keyFieldNames =
@@ -413,9 +461,6 @@
if (isEnforcedFieldPos > 0) {
isEnforcingKeys = ((ABoolean) indexRecord.getValueByPos(isEnforcedFieldPos)).getBoolean();
}
- Boolean isPrimaryIndex =
- ((ABoolean) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
- .getBoolean();
int pendingOp = ((AInt32) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX))
.getIntegerValue();
@@ -549,6 +594,8 @@
writeSearchKeyType(index);
writeEnforced(index);
writeSearchKeySourceIndicator(index);
+ writeExcludeUnknownKey(index);
+ writeCast(index);
}
private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
@@ -748,4 +795,64 @@
recordBuilder.addField(nameValue, fieldValue);
}
}
+
+ private void writeExcludeUnknownKey(Index index) throws HyracksDataException {
+ switch (index.getIndexType()) {
+ case BTREE:
+ if (!index.isPrimaryIndex() && !index.isPrimaryKeyIndex()) {
+ OptionalBoolean excludeUnknown =
+ ((Index.ValueIndexDetails) index.getIndexDetails()).getExcludeUnknownKey();
+ ABoolean bVal = excludeUnknown.isEmpty() ? ABoolean.FALSE : ABoolean.valueOf(excludeUnknown.get());
+ fieldValue.reset();
+ nameValue.reset();
+ aString.setValue(INDEX_EXCLUDE_UNKNOWN_FIELD_NAME);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ booleanSerde.serialize(bVal, fieldValue.getDataOutput());
+ recordBuilder.addField(nameValue, fieldValue);
+ }
+ break;
+
+ case ARRAY:
+ // TODO: This value is written for back-compatibility, and is currently always assumed to be true.
+ fieldValue.reset();
+ nameValue.reset();
+ aString.setValue(INDEX_EXCLUDE_UNKNOWN_FIELD_NAME);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ booleanSerde.serialize(ABoolean.TRUE, fieldValue.getDataOutput());
+ recordBuilder.addField(nameValue, fieldValue);
+ break;
+ }
+ }
+
+ private void writeCast(Index index) throws HyracksDataException {
+ if (index.getIndexType() == IndexType.BTREE && !index.isPrimaryIndex() && !index.isPrimaryKeyIndex()) {
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ boolean defaultNull = indexDetails.getCastDefaultNull().getOrElse(false);
+ // write record field 'Cast'
+ if (defaultNull) {
+ // write field 'Default'
+ castRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+ fieldValue.reset();
+ nameValue.reset();
+ aString.setValue(FIELD_NAME_DEFAULT);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ nullSerde.serialize(ANull.NULL, fieldValue.getDataOutput());
+ castRecordBuilder.addField(nameValue, fieldValue);
+
+ // write field 'Format'
+ String datetimeFormat = indexDetails.getCastDatetimeFormat();
+ String dateFormat = indexDetails.getCastDateFormat();
+ String timeFormat = indexDetails.getCastTimeFormat();
+ writeDateTimeFormats(datetimeFormat, dateFormat, timeFormat, castRecordBuilder, aString, nullSerde,
+ stringSerde, nameValue, fieldValue, itemValue);
+
+ nameValue.reset();
+ fieldValue.reset();
+ aString.setValue(FIELD_NAME_CAST);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ castRecordBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(nameValue, fieldValue);
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
index f4dfe56..1abf300 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
@@ -19,6 +19,7 @@
package org.apache.asterix.metadata.utils;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -39,14 +40,18 @@
public class ArrayIndexUtil {
/**
- * @deprecated Use the project + unnest scheme instead of array indicators.
+ * Similar function to Index's "getSubFieldType", but accounts for array fields as well.
*/
- public static IAType getSubFieldInArrayType(ARecordType recordType, List<String> subFieldName,
- List<Integer> arrayDepthIndicators) throws AlgebricksException {
- IAType subType = recordType.getFieldType(subFieldName.get(0));
- for (int i = 1; i < subFieldName.size(); i++) {
+ public static IAType getSubFieldType(ARecordType recordType, List<List<String>> unnestList,
+ List<String> projectList) throws AlgebricksException {
+ List<String> flattenedFieldName = ArrayIndexUtil.getFlattenedKeyFieldNames(unnestList, projectList);
+ List<Boolean> unnestFlags = ArrayIndexUtil.getUnnestFlags(unnestList, projectList);
+ IAType subType = recordType.getFieldType(flattenedFieldName.get(0));
+
+ for (int i = 1; i < flattenedFieldName.size(); i++) {
if (subType == null) {
return null;
+
} else if (subType.getTypeTag().equals(ATypeTag.UNION)) {
// Support enforced types here.
subType = ((AUnionType) subType).getActualType();
@@ -56,31 +61,30 @@
"Field accessor is not defined for values of type " + subType.getTypeTag());
}
}
- if (subType.getTypeTag().equals(ATypeTag.OBJECT) && arrayDepthIndicators.get(i - 1) == 0) {
- subType = ((ARecordType) subType).getFieldType(subFieldName.get(i));
+
+ if (subType.getTypeTag().equals(ATypeTag.OBJECT) && !unnestFlags.get(i - 1)) {
+ subType = ((ARecordType) subType).getFieldType(flattenedFieldName.get(i));
+
} else if ((subType.getTypeTag().equals(ATypeTag.ARRAY) || subType.getTypeTag().equals(ATypeTag.MULTISET))
- && arrayDepthIndicators.get(i - 1) > 0) {
- for (int j = 0; j < arrayDepthIndicators.get(i - 1); j++) {
- subType = TypeComputeUtils.extractListItemType(subType);
- }
- subType = (subType != null) ? ((ARecordType) subType).getFieldType(subFieldName.get(i)) : null;
+ && unnestFlags.get(i - 1)) {
+ subType = TypeComputeUtils.extractListItemType(subType);
+ subType = (subType != null) ? ((ARecordType) subType).getFieldType(flattenedFieldName.get(i)) : null;
+
} else {
throw new AsterixException(ErrorCode.COMPILATION_ERROR,
- (arrayDepthIndicators.get(i - 1) > 0)
- ? "Object type given, but array depth indicator is " + "non-zero."
- : "Array/multiset type given, but array depth indicator is zero.");
+ unnestFlags.get(i - 1) ? "Object type given, but unnest flag is also raised."
+ : "Array/multiset type given, but unnest flag is lowered.");
}
}
- if (subType != null && arrayDepthIndicators.get(arrayDepthIndicators.size() - 1) > 0) {
+
+ if (subType != null && unnestFlags.get(unnestFlags.size() - 1)) {
// If the end field is an array, we must extract the list item here as well.
- for (int j = 0; j < arrayDepthIndicators.get(arrayDepthIndicators.size() - 1); j++) {
- if (subType instanceof AbstractCollectionType) {
- subType = TypeComputeUtils.extractListItemType(subType);
- } else {
- throw new AsterixException(ErrorCode.COMPILATION_ERROR,
- "Array type expected for last term, but given: "
- + ((subType != null) ? subType.getTypeTag() : "null"));
- }
+ if (subType instanceof AbstractCollectionType) {
+ subType = TypeComputeUtils.extractListItemType(subType);
+
+ } else {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+ "Array type expected for last term, but given: " + subType.getTypeTag());
}
}
return subType;
@@ -88,15 +92,18 @@
/**
* Given a path of complex types (i.e. lists + records), determine the nullability of the field.
- * @deprecated Use the project + unnest scheme instead of array indicators.
*/
- public static boolean isSubFieldNullable(ARecordType recordType, List<String> subFieldName,
- List<Integer> arrayIndicators) throws AlgebricksException {
- IAType subType = recordType.getFieldType(subFieldName.get(0));
- for (int i = 1; i < subFieldName.size(); i++) {
+ public static boolean isSubFieldNullable(ARecordType recordType, List<List<String>> unnestList,
+ List<String> projectList) throws AlgebricksException {
+ List<String> flattenedFieldName = ArrayIndexUtil.getFlattenedKeyFieldNames(unnestList, projectList);
+ List<Boolean> unnestFlags = ArrayIndexUtil.getUnnestFlags(unnestList, projectList);
+ IAType subType = recordType.getFieldType(flattenedFieldName.get(0));
+
+ for (int i = 1; i < flattenedFieldName.size(); i++) {
if (subType == null) {
return true;
}
+
if (subType.getTypeTag().equals(ATypeTag.UNION)) {
if (NonTaggedFormatUtil.isOptional(subType)) {
return true;
@@ -109,12 +116,12 @@
}
if (subType instanceof ARecordType) {
- subType = ((ARecordType) subType).getFieldType(subFieldName.get(i));
- } else if (subType instanceof AbstractCollectionType && arrayIndicators.get(i - 1) > 0) {
- for (int j = 0; j < arrayIndicators.get(i - 1); j++) {
- subType = TypeComputeUtils.extractListItemType(subType);
- }
- subType = (subType != null) ? ((ARecordType) subType).getFieldType(subFieldName.get(i)) : null;
+ subType = ((ARecordType) subType).getFieldType(flattenedFieldName.get(i));
+
+ } else if (subType instanceof AbstractCollectionType && unnestFlags.get(i - 1)) {
+ subType = TypeComputeUtils.extractListItemType(subType);
+ subType = (subType != null) ? ((ARecordType) subType).getFieldType(flattenedFieldName.get(i)) : null;
+
} else {
throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
"Illegal field type " + subType.getTypeTag() + " when checking field nullability");
@@ -125,32 +132,37 @@
/**
* Similar function to Index's "getNonNullableOpenFieldType", but accounts for array fields as well.
- * @deprecated Use the project + unnest scheme instead of array indicators.
*/
- public static Pair<IAType, Boolean> getNonNullableOpenFieldType(IAType fieldType, List<String> fieldName,
- ARecordType recType, List<Integer> arrayIndicators) throws AlgebricksException {
+ public static Pair<IAType, Boolean> getNonNullableOpenFieldType(IAType fieldType, List<List<String>> unnestList,
+ List<String> projectList, ARecordType recType) throws AlgebricksException {
Pair<IAType, Boolean> keyPairType = null;
IAType subType = recType;
boolean nullable = false;
- for (int i = 0; i < fieldName.size(); i++) {
+
+ List<String> flattenedFieldName = ArrayIndexUtil.getFlattenedKeyFieldNames(unnestList, projectList);
+ List<Boolean> unnestFlags = ArrayIndexUtil.getUnnestFlags(unnestList, projectList);
+ for (int i = 0; i < flattenedFieldName.size(); i++) {
if (subType instanceof AUnionType) {
nullable = nullable || ((AUnionType) subType).isUnknownableType();
subType = ((AUnionType) subType).getActualType();
}
if (subType instanceof ARecordType) {
- subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+ subType = ((ARecordType) subType).getFieldType(flattenedFieldName.get(i));
+
} else if ((subType instanceof AOrderedListType || subType instanceof AUnorderedListType)
- && arrayIndicators.get(i - 1) > 0) {
- for (int j = 0; j < arrayIndicators.get(i - 1); j++) {
- subType = TypeComputeUtils.extractListItemType(subType);
- }
+ && unnestFlags.get(i - 1)) {
+ subType = TypeComputeUtils.extractListItemType(subType);
if (subType instanceof ARecordType) {
- subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+ subType = ((ARecordType) subType).getFieldType(flattenedFieldName.get(i));
+
} else {
- throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+ throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Unexpected type " + subType + ", expected record.");
}
+
} else {
- throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+ throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Unexpected type " + subType + ", expected record, array, or multi-set.");
}
if (subType == null) {
@@ -158,18 +170,20 @@
break;
}
}
+
if (subType != null) {
- IAType keyType = ArrayIndexUtil.getSubFieldInArrayType(recType, fieldName, arrayIndicators);
+ IAType keyType = ArrayIndexUtil.getSubFieldType(recType, unnestList, projectList);
Pair<IAType, Boolean> pair = Index.getNonNullableType(keyType);
- pair.second = pair.second || ArrayIndexUtil.isSubFieldNullable(recType, fieldName, arrayIndicators);
+ pair.second = pair.second || ArrayIndexUtil.isSubFieldNullable(recType, unnestList, projectList);
keyPairType = pair;
}
+
keyPairType.second = keyPairType.second || nullable;
return keyPairType;
}
/**
- * @deprecated Use new unnestList and projectList scheme.
+ * @return The concatenation of the unnest list fields and the project field (for use in creating a unique name).
*/
public static List<String> getFlattenedKeyFieldNames(List<List<String>> unnestList, List<String> projectList) {
if (unnestList == null) {
@@ -188,94 +202,82 @@
}
/**
- * @deprecated Use new unnestList and projectList scheme.
+ * @return Mapping to the flattened key field names, determine where the UNNESTs occur.
*/
- public static List<Integer> getArrayDepthIndicator(List<List<String>> unnestList, List<String> projectList) {
- if (unnestList == null) {
- // A simple element has a flat set of depth indicators.
- List<Integer> depthIndicator = new ArrayList<>();
+ public static List<Boolean> getUnnestFlags(List<List<String>> unnestList, List<String> projectList) {
+ if (unnestList.isEmpty()) {
+ // A simple element has no UNNEST flags raised..
+ List<Boolean> unnestFlags = new ArrayList<>();
for (String ignored : projectList) {
- depthIndicator.add(0);
+ unnestFlags.add(false);
}
- return depthIndicator;
+ return unnestFlags;
} else {
- List<Integer> depthIndicatorPrefix = new ArrayList<>();
+ List<Boolean> unnestFlagsPrefix = new ArrayList<>();
for (List<String> unnestField : unnestList) {
for (int i = 0; i < unnestField.size() - 1; i++) {
- depthIndicatorPrefix.add(0);
+ unnestFlagsPrefix.add(false);
}
- depthIndicatorPrefix.add(1);
+ unnestFlagsPrefix.add(true);
}
if (projectList == null) {
- // Stop here. The prefix is the indicator itself.
- return depthIndicatorPrefix;
+ // Stop here. The prefix is the flag vector itself.
+ return unnestFlagsPrefix;
} else {
- List<Integer> depthIndicator = new ArrayList<>(depthIndicatorPrefix);
+ List<Boolean> unnestFlags = new ArrayList<>(unnestFlagsPrefix);
for (int i = 0; i < projectList.size(); i++) {
- depthIndicator.add(0);
+ unnestFlags.add(false);
}
- return depthIndicator;
+ return unnestFlags;
}
}
}
/**
- * @deprecated Use new unnestList and projectList scheme.
- * @return The record paths and non-zero depth indicators associated each record of fields from an array index.
+ * Traverse each distinct record path and invoke the appropriate commands for each scenario. Here, we keep track
+ * of the record/list type at each step and give this to each command.
*/
- public static Pair<List<List<String>>, List<Integer>> unnestComplexRecordPath(List<String> fieldName,
- List<Integer> depthIndicators) {
- List<List<String>> resultantPaths = new ArrayList<>();
- List<Integer> resultantArrayIndicators = new ArrayList<>();
- List<String> workingRecordPath = new ArrayList<>();
- for (int i = 0; i < depthIndicators.size(); i++) {
- workingRecordPath.add(fieldName.get(i));
-
- if (i == depthIndicators.size() - 1 || depthIndicators.get(i) > 0) {
- resultantArrayIndicators.add(depthIndicators.get(i));
- resultantPaths.add(workingRecordPath);
- workingRecordPath = new ArrayList<>();
- }
- }
- return new Pair<>(resultantPaths, resultantArrayIndicators);
- }
-
- /**
- * Given the {@code Index}'s representation of an array path (i.e. a concatenation of record paths, with array
- * steps specified in depths corresponding to an index in the aforementioned record path array), traverse each
- * distinct record path and invoke the appropriate commands for each scenario.
- * <p>
- * Here, we keep track of the record/list type at each step and give this to each command.
- */
- public static void walkArrayPath(ARecordType baseRecordType, List<String> flattenedFieldName,
- List<Integer> flattenedDepthIndicators, TypeTrackerCommandExecutor commandExecutor)
- throws AlgebricksException {
- ArrayPath arrayPath = new ArrayPath(flattenedFieldName, flattenedDepthIndicators).invoke();
+ public static void walkArrayPath(Index index, ARecordType baseRecordType, List<String> flattenedFieldName,
+ List<Boolean> unnestFlags, TypeTrackerCommandExecutor commandExecutor) throws AlgebricksException {
+ ArrayPath arrayPath = new ArrayPath(flattenedFieldName, unnestFlags).invoke();
List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
- List<Integer> depthOfArraySteps = arrayPath.depthOfArraySteps;
+ List<Boolean> unnestFlagsPerArray = arrayPath.unnestFlagsPerArray;
+ boolean requiresOnlyOneUnnest = unnestFlags.stream().filter(f -> f).count() == 1;
- // If we are given no base record type, then we do not need to keep track of the record type. We are solely
+ // If we are given no base record type, then we do not need to keep track of the record type. We are solely
// using this walk for its flags.
boolean isTrackingType = baseRecordType != null;
-
IAType workingType = baseRecordType;
+
for (int i = 0; i < fieldNamesPerArray.size(); i++) {
- ARecordType startingStepRecordType = null;
+ ARecordType startingStepRecordType = (isTrackingType) ? (ARecordType) workingType : null;
if (isTrackingType) {
if (!workingType.getTypeTag().equals(ATypeTag.OBJECT)) {
throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Mismatched record type to depth-"
+ "indicators. Expected record type, but got: " + workingType.getTypeTag());
}
- startingStepRecordType = (ARecordType) workingType;
- workingType = Index.getNonNullableOpenFieldType(
- startingStepRecordType.getSubFieldType(fieldNamesPerArray.get(i)), fieldNamesPerArray.get(i),
- startingStepRecordType).first;
+
+ ARecordType intermediateRecordType = startingStepRecordType;
+ List<String> fieldName = fieldNamesPerArray.get(i);
+ for (String fieldPart : fieldName) {
+ // Determine whether we have an open field or not. Extract the type appropriately.
+ isTrackingType = isTrackingType && intermediateRecordType.doesFieldExist(fieldPart);
+ if (isTrackingType) {
+ workingType =
+ Index.getNonNullableOpenFieldType(index, intermediateRecordType.getFieldType(fieldPart),
+ Collections.singletonList(fieldPart), intermediateRecordType).first;
+ if (workingType instanceof ARecordType) {
+ // We have an intermediate step, set our record step for the next loop iteration.
+ intermediateRecordType = (ARecordType) workingType;
+ }
+ }
+ }
}
- for (int j = 0; j < depthOfArraySteps.get(i); j++) {
+ if (unnestFlagsPerArray.get(i)) {
if (isTrackingType) {
workingType = TypeComputeUtils.extractListItemType(workingType);
if (workingType == null) {
@@ -284,17 +286,13 @@
}
}
boolean isFirstArrayStep = i == 0;
- boolean isFirstUnnestInStep = j == 0;
- boolean isLastUnnestInIntermediateStep =
- j == depthOfArraySteps.get(i) - 1 && i < fieldNamesPerArray.size() - 1;
+ boolean isLastUnnestInIntermediateStep = i < fieldNamesPerArray.size() - 1;
commandExecutor.executeActionOnEachArrayStep(startingStepRecordType, workingType,
- fieldNamesPerArray.get(i), isFirstArrayStep, isFirstUnnestInStep,
- isLastUnnestInIntermediateStep);
+ fieldNamesPerArray.get(i), isFirstArrayStep, isLastUnnestInIntermediateStep);
}
if (i == fieldNamesPerArray.size() - 1) {
- boolean requiresOnlyOneUnnest = depthOfArraySteps.stream().reduce(0, Integer::sum).equals(1);
- boolean isNonArrayStep = depthOfArraySteps.get(i) == 0;
+ boolean isNonArrayStep = !unnestFlagsPerArray.get(i);
commandExecutor.executeActionOnFinalArrayStep(startingStepRecordType, fieldNamesPerArray.get(i),
isNonArrayStep, requiresOnlyOneUnnest);
}
@@ -302,28 +300,25 @@
}
/**
- * Given the {@code Index}'s representation of an array path (i.e. a concatenation of record paths, with array
- * steps specified in depths corresponding to an index in the aforementioned record path array), traverse each
- * distinct record path and invoke the appropriate commands for each scenario.
- * <p>
- * Here, we keep track of the total number of actions performed and give this to each command.
+ * Traverse each distinct record path and invoke the appropriate commands for each scenario. Here, we keep track
+ * of the total number of actions performed and give this to each command.
*/
- public static void walkArrayPath(List<String> flattenedFieldName, List<Integer> flattenedDepthIndicators,
+ public static void walkArrayPath(List<String> flattenedFieldName, List<Boolean> unnestFlags,
ActionCounterCommandExecutor commandExecutor) throws AlgebricksException {
- ArrayPath arrayPath = new ArrayPath(flattenedFieldName, flattenedDepthIndicators).invoke();
+ ArrayPath arrayPath = new ArrayPath(flattenedFieldName, unnestFlags).invoke();
List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
- List<Integer> depthOfArraySteps = arrayPath.depthOfArraySteps;
+ List<Boolean> unnestFlagsPerArray = arrayPath.unnestFlagsPerArray;
int numberOfActionsPerformed = 0;
for (int i = 0; i < fieldNamesPerArray.size(); i++) {
- int unnestLevel = depthOfArraySteps.get(i);
+ boolean isUnnestFlagRaised = unnestFlagsPerArray.get(i);
if (i == 0) {
commandExecutor.executeActionOnFirstArrayStep();
numberOfActionsPerformed++;
- unnestLevel--;
+ isUnnestFlagRaised = false;
}
- for (int j = 0; j < unnestLevel; j++) {
+ if (isUnnestFlagRaised) {
commandExecutor.executeActionOnIntermediateArrayStep(numberOfActionsPerformed++);
}
@@ -343,8 +338,8 @@
public interface TypeTrackerCommandExecutor {
void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
- List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
- boolean isLastUnnestInIntermediateStep) throws AlgebricksException;
+ List<String> fieldName, boolean isFirstArrayStep, boolean isLastUnnestInIntermediateStep)
+ throws AlgebricksException;
void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException;
@@ -352,24 +347,24 @@
private static class ArrayPath {
private final List<String> flattenedFieldName;
- private final List<Integer> flattenedDepthIndicators;
+ private final List<Boolean> unnestFlags;
private List<List<String>> fieldNamesPerArray;
- private List<Integer> depthOfArraySteps;
+ private List<Boolean> unnestFlagsPerArray;
- public ArrayPath(List<String> flattenedFieldName, List<Integer> flattenedDepthIndicators) {
+ public ArrayPath(List<String> flattenedFieldName, List<Boolean> unnestFlags) {
this.flattenedFieldName = flattenedFieldName;
- this.flattenedDepthIndicators = flattenedDepthIndicators;
+ this.unnestFlags = unnestFlags;
}
public ArrayPath invoke() {
fieldNamesPerArray = new ArrayList<>();
- depthOfArraySteps = new ArrayList<>();
+ unnestFlagsPerArray = new ArrayList<>();
List<String> workingRecordPath = new ArrayList<>();
- for (int i = 0; i < flattenedDepthIndicators.size(); i++) {
+ for (int i = 0; i < unnestFlags.size(); i++) {
workingRecordPath.add(flattenedFieldName.get(i));
- if (i == flattenedDepthIndicators.size() - 1 || flattenedDepthIndicators.get(i) > 0) {
- depthOfArraySteps.add(flattenedDepthIndicators.get(i));
+ if (i == unnestFlags.size() - 1 || unnestFlags.get(i)) {
+ unnestFlagsPerArray.add(unnestFlags.get(i));
fieldNamesPerArray.add(workingRecordPath);
workingRecordPath = new ArrayList<>();
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 2e10d77..6529794 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -19,7 +19,6 @@
package org.apache.asterix.metadata.utils;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
-import static org.apache.asterix.common.utils.IdentifierUtil.dataverse;
import java.io.DataOutput;
import java.util.ArrayList;
@@ -39,6 +38,7 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.external.indexing.IndexingConstants;
@@ -369,8 +369,7 @@
DataverseName dataverseName = dataverse.getDataverseName();
Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
if (dataset == null) {
- throw new AsterixException(
- "Could not find " + dataset() + " " + datasetName + " in " + dataverse() + " " + dataverseName);
+ throw new AsterixException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, datasetName, dataverseName);
}
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
@@ -417,7 +416,7 @@
metadataProvider.getStorageComponentProvider().getStorageManager(), primaryFileSplitProvider);
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec,
dataset.getPrimaryRecordDescriptor(metadataProvider), lowKeyFields, highKeyFields, true, true,
- indexHelperFactory, false, false, null, searchCallbackFactory, null, null, false);
+ indexHelperFactory, false, false, null, searchCallbackFactory, null, null, false, null);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
primaryPartitionConstraint);
return primarySearchOp;
@@ -475,9 +474,9 @@
IDataFormat dataFormat = metadataProvider.getDataFormat();
int f = 0;
- // add the upsert indicator var
- outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(BuiltinType.ABOOLEAN);
- outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(BuiltinType.ABOOLEAN);
+ // add the upsert operation var
+ outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(BuiltinType.AINT8);
+ outputTypeTraits[f] = dataFormat.getTypeTraitProvider().getTypeTrait(BuiltinType.AINT8);
f++;
// add the previous record
outputSerDes[f] = dataFormat.getSerdeProvider().getSerializerDeserializer(itemType);
@@ -621,4 +620,11 @@
return nodeGroup;
}
+ public static String getDatasetTypeDisplayName(DatasetType datasetType) {
+ return datasetType == DatasetType.VIEW ? "view" : dataset();
+ }
+
+ public static boolean isNotView(Dataset dataset) {
+ return dataset.getDatasetType() != DatasetType.VIEW;
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index f5b2697..ad92803 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -33,12 +33,19 @@
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.job.IJobletEventListenerFactory;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.util.OptionalBoolean;
public class IndexUtil {
@@ -58,9 +65,8 @@
public static Index getPrimaryIndex(Dataset dataset) {
InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
- return new Index(dataset.getDataverseName(), dataset.getDatasetName(), dataset.getDatasetName(),
- DatasetConfig.IndexType.BTREE, id.getPartitioningKey(), id.getKeySourceIndicator(),
- id.getPrimaryKeyType(), false, false, true, dataset.getPendingOp());
+ return Index.createPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName(), id.getPartitioningKey(),
+ id.getKeySourceIndicator(), id.getPrimaryKeyType(), dataset.getPendingOp());
}
public static int[] getBtreeFieldsIfFiltered(Dataset dataset, Index index) throws AlgebricksException {
@@ -180,4 +186,58 @@
spec.setJobletEventListenerFactory(jobEventListenerFactory);
}
+ public static boolean castDefaultNull(Index index) {
+ return Index.IndexCategory.of(index.getIndexType()) == Index.IndexCategory.VALUE
+ && ((Index.ValueIndexDetails) index.getIndexDetails()).getCastDefaultNull().getOrElse(false);
+ }
+
+ public static Pair<FunctionIdentifier, IAObject> getTypeConstructorDefaultNull(Index index, IAType type,
+ SourceLocation srcLoc) throws CompilationException {
+ Triple<String, String, String> temporalFormats = getTemporalFormats(index);
+ String format = temporalFormats != null ? TypeUtil.getTemporalFormat(type, temporalFormats) : null;
+ boolean withFormat = format != null;
+ FunctionIdentifier typeConstructorFun = TypeUtil.getTypeConstructorDefaultNull(type, withFormat);
+ if (typeConstructorFun == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_TYPE_UNSUPPORTED, srcLoc, "index", type.getTypeName());
+ }
+ return new Pair<>(typeConstructorFun, withFormat ? new AString(format) : null);
+ }
+
+ private static Triple<String, String, String> getTemporalFormats(Index index) {
+ if (Index.IndexCategory.of(index.getIndexType()) != Index.IndexCategory.VALUE) {
+ return null;
+ }
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ String datetimeFormat = indexDetails.getCastDatetimeFormat();
+ String dateFormat = indexDetails.getCastDateFormat();
+ String timeFormat = indexDetails.getCastTimeFormat();
+ if (datetimeFormat != null || dateFormat != null || timeFormat != null) {
+ return new Triple<>(datetimeFormat, dateFormat, timeFormat);
+ } else {
+ return null;
+ }
+ }
+
+ public static boolean includesUnknowns(Index index) {
+ return !index.isPrimaryKeyIndex() && secondaryIndexIncludesUnknowns(index);
+ }
+
+ private static boolean secondaryIndexIncludesUnknowns(Index index) {
+ if (Index.IndexCategory.of(index.getIndexType()) != Index.IndexCategory.VALUE) {
+ // other types of indexes do not include unknowns
+ return false;
+ }
+ OptionalBoolean excludeUnknownKey = ((Index.ValueIndexDetails) index.getIndexDetails()).getExcludeUnknownKey();
+ if (index.getIndexType() == DatasetConfig.IndexType.BTREE) {
+ // by default, Btree includes unknowns
+ return excludeUnknownKey.isEmpty() || !excludeUnknownKey.get();
+ } else {
+ // by default, others exclude unknowns
+ return !excludeUnknownKey.isEmpty() && !excludeUnknownKey.get();
+ }
+ }
+
+ public static boolean excludesUnknowns(Index index) {
+ return !includesUnknowns(index);
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index 8b69652..a75a75d 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -31,15 +31,18 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
import org.apache.asterix.metadata.api.IResourceFactoryProvider;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.data.std.accessors.ShortBinaryComparatorFactory;
@@ -129,13 +132,15 @@
FullTextUtil.fetchFilterAndCreateConfigEvaluator(mdProvider, index.getDataverseName(),
indexDetails.getFullTextConfigName());
+ ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
- bloomFilterFalsePositiveRate);
+ bloomFilterFalsePositiveRate, typeTraitProvider.getTypeTrait(BuiltinType.ANULL),
+ NullIntrospector.INSTANCE);
}
// Returns an array of the type traits of the inverted list elements
@@ -180,8 +185,8 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
- indexDetails.getKeyFieldNames().get(0), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
+ indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), sourceType);
IAType secondaryKeyType = keyTypePair.first;
int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
@@ -217,8 +222,8 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
- indexDetails.getKeyFieldNames().get(0), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
+ indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), sourceType);
IAType secondaryKeyType = keyTypePair.first;
// Comparators and type traits for tokens.
int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
@@ -253,8 +258,8 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
- indexDetails.getKeyFieldNames().get(0), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
+ indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), sourceType);
IAType secondaryKeyType = keyTypePair.first;
// Set tokenizer factory.
// TODO: We might want to expose the hashing option at the AQL level,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
index b2026d7..61effa8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
@@ -133,7 +133,7 @@
List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
List<IAType> indexKeyTypes = new ArrayList<>();
for (int i = 0; i < indexDetails.getKeyFieldNames().size(); i++) {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i),
chooseSource(keySourceIndicators, i, recordType, metaRecordType));
indexKeyTypes.add(keyPairType.first);
@@ -153,9 +153,7 @@
ARecordType sourceType =
(e.getSourceIndicator() == Index.RECORD_INDICATOR) ? recordType : metaRecordType;
Pair<IAType, Boolean> keyPairType = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
- ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)),
- sourceType,
- ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(i)));
+ e.getUnnestList(), e.getProjectList().get(i), sourceType);
indexKeyTypes.add(keyPairType.first);
}
}
@@ -180,8 +178,8 @@
List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
List<IAType> indexKeyTypes = new ArrayList<>();
ARecordType targetRecType = chooseSource(keySourceIndicators, 0, recordType, metaRecordType);
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
- indexDetails.getKeyFieldNames().get(0), targetRecType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index,
+ indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), targetRecType);
IAType keyType = keyPairType.first;
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
int numKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, targetRecType, metaRecordType);
@@ -219,7 +217,7 @@
case RTREE:
Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0),
chooseSource(keySourceIndicators, 0, recordType, metaRecordType));
IAType keyType = keyPairType.first;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index e00d7b6..463cffb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -28,6 +28,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
import org.apache.asterix.metadata.api.IResourceFactoryProvider;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
@@ -35,6 +36,7 @@
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.commons.lang3.StringUtils;
@@ -76,7 +78,7 @@
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD,
indexDetails.getKeyFieldNames().size(), index.getIndexType(), 1);
}
- IAType spatialType = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+ IAType spatialType = Index.getNonNullableOpenFieldType(index, indexDetails.getKeyFieldTypes().get(0),
indexDetails.getKeyFieldNames().get(0), recordType).first;
if (spatialType == null) {
throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
@@ -155,6 +157,7 @@
dataset.getDatasetType() == DatasetType.EXTERNAL ? IndexingConstants.getBuddyBtreeComparatorFactories()
: getComparatorFactoriesForDeletedKeyBTree(secondaryTypeTraits, primaryComparatorFactories,
secondaryComparatorFactories);
+ ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
AsterixVirtualBufferCacheProvider vbcProvider =
new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
@@ -162,14 +165,16 @@
filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, valueProviderFactories, rTreePolicyType,
- linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories);
+ linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories,
+ typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
} else {
return new ExternalRTreeLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
mergePolicyProperties, true, btreeCompFactories, valueProviderFactories, rTreePolicyType,
linearizeCmpFactory, rtreeFields, new int[] { numNestedSecondaryKeyFields }, isPointMBR,
- mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate());
+ mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
+ typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
}
}
@@ -206,7 +211,7 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), sourceType);
IAType spatialType = spatialTypePair.first;
if (spatialType == null) {
@@ -243,7 +248,7 @@
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), sourceType);
IAType spatialType = spatialTypePair.first;
if (spatialType == null) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
index fdb21a2..c8a7ee1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
@@ -22,11 +22,13 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.Comparator;
import java.util.Deque;
import java.util.LinkedList;
import java.util.List;
+import java.util.Objects;
import java.util.Queue;
-import java.util.Stack;
+import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
@@ -57,7 +59,6 @@
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
-import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
import org.apache.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
@@ -78,14 +79,13 @@
public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndexOperationsHelper {
private final int numAtomicSecondaryKeys, numArraySecondaryKeys, numTotalSecondaryKeys;
- private final Index.ArrayIndexDetails arrayIndexDetails;
private final EvalFactoryAndRecDescStackBuilder evalFactoryAndRecDescStackBuilder =
new EvalFactoryAndRecDescStackBuilder();
- // TODO (GLENN): Phase these out and use the UNNEST / PROJECT scheme instead.
+ private final Index.ArrayIndexDetails arrayIndexDetails;
private final List<List<String>> flattenedFieldNames;
private final List<IAType> flattenedKeyTypes;
- private final List<List<Integer>> depthIndicators;
+ private final List<List<Boolean>> unnestFlags;
protected SecondaryArrayIndexBTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
SourceLocation sourceLoc) throws AlgebricksException {
@@ -94,19 +94,19 @@
flattenedFieldNames = new ArrayList<>();
flattenedKeyTypes = new ArrayList<>();
- depthIndicators = new ArrayList<>();
+ unnestFlags = new ArrayList<>();
for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
if (e.getUnnestList().isEmpty()) {
flattenedFieldNames.add(e.getProjectList().get(0));
flattenedKeyTypes.add(e.getTypeList().get(0));
- depthIndicators
- .add(ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(0)));
+ unnestFlags.add(ArrayIndexUtil.getUnnestFlags(e.getUnnestList(), e.getProjectList().get(0)));
+
} else {
for (int i = 0; i < e.getProjectList().size(); i++) {
List<String> project = e.getProjectList().get(i);
flattenedFieldNames.add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
- depthIndicators.add(ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
flattenedKeyTypes.add(e.getTypeList().get(i));
+ unnestFlags.add(ArrayIndexUtil.getUnnestFlags(e.getUnnestList(), project));
}
}
}
@@ -127,7 +127,7 @@
numArraySecondaryKeys = numTotalSecondaryKeys - numAtomicSecondaryKeys;
}
- private int findPosOfArrayIndex() throws AsterixException {
+ private int findPosOfArrayIndexElement() throws AsterixException {
for (int i = 0; i < arrayIndexDetails.getElementList().size(); i++) {
if (!arrayIndexDetails.getElementList().get(i).getUnnestList().isEmpty()) {
return i;
@@ -154,18 +154,13 @@
ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
IBinaryComparatorFactoryProvider comparatorFactoryProvider =
metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
- // Record column is 0 for external datasets, numPrimaryKeys for internal ones
- int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
boolean isOverridingKeyFieldTypes = arrayIndexDetails.isOverridingKeyFieldTypes();
int flattenedListPos = 0;
for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
for (int i = 0; i < e.getProjectList().size(); i++) {
- ARecordType sourceType = (e.getSourceIndicator() == 0) ? itemType : metaType;
- addSKEvalFactories(isOverridingKeyFieldTypes ? enforcedItemType : sourceType, flattenedListPos, false);
+ addSKEvalFactories(isOverridingKeyFieldTypes ? enforcedItemType : itemType, flattenedListPos, false);
Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
- ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)),
- sourceType,
- ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(i)));
+ e.getUnnestList(), e.getProjectList().get(i), itemType);
IAType keyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
@@ -242,20 +237,21 @@
return;
}
- List<Integer> arrayDepthIndicators = depthIndicators.get(fieldPos);
- List<String> fieldNames = flattenedFieldNames.get(fieldPos);
- if (arrayDepthIndicators.stream().noneMatch(b -> b > 0)) {
+ List<String> flattenedFieldName = flattenedFieldNames.get(fieldPos);
+ List<Boolean> workingUnnestFlags = unnestFlags.get(fieldPos);
+ if (workingUnnestFlags.stream().noneMatch(b -> b)) {
addAtomicFieldToBuilder(recordType, fieldPos);
+
} else {
EvalFactoryAndRecDescInvoker commandExecutor =
new EvalFactoryAndRecDescInvoker(!evalFactoryAndRecDescStackBuilder.isUnnestEvalPopulated());
- ArrayIndexUtil.walkArrayPath(recordType, fieldNames, arrayDepthIndicators, commandExecutor);
+ ArrayIndexUtil.walkArrayPath(index, recordType, flattenedFieldName, workingUnnestFlags, commandExecutor);
}
}
/**
- * The following job spec is produced: (key provider) -> (PIDX scan) -> (cast)? -> ((unnest) -> (assign))* ->
- * (select)? -> (sort)? -> (bulk load) -> (sink)
+ * The following job spec is produced: (key provider) -> (PIDX scan) -> (cast)? -> (assign)? ->
+ * ((unnest) -> (assign))* -> (select)? -> (sort)? -> (bulk load) -> (sink)
*/
@Override
public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
@@ -278,19 +274,38 @@
sourceOp = targetOp;
}
- // TODO (GLENN): Refactor to use UNNEST + PROJECT scheme.
+ // We do not index meta fields. Project away meta fields if they exist.
+ if (dataset.hasMetaPart()) {
+ int[] outColumns = new int[] { primaryRecDesc.getFieldCount() };
+ int[] projectionList = new int[primaryRecDesc.getFieldCount() - 1];
+ for (int i = 0; i < projectionList.length - 1; i++) {
+ projectionList[i] = i;
+ }
+ projectionList[projectionList.length - 1] = primaryRecDesc.getFieldCount() - 2;
+ ISerializerDeserializer[] fields = new ISerializerDeserializer[primaryRecDesc.getFieldCount() - 1];
+ ITypeTraits[] typeTraits = new ITypeTraits[primaryRecDesc.getFieldCount() - 1];
+ for (int i = 0; i < primaryRecDesc.getFieldCount() - 1; i++) {
+ fields[i] = primaryRecDesc.getFields()[i];
+ typeTraits[i] = primaryRecDesc.getTypeTraits()[i];
+ }
+ targetOp = createGenericAssignOp(spec, new ArrayList<>(), new RecordDescriptor(fields, typeTraits),
+ outColumns, projectionList);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+ }
+
// Perform the unnest work.
final Mutable<IOperatorDescriptor> sourceOpRef = new MutableObject<>(sourceOp);
final Mutable<IOperatorDescriptor> targetOpRef = new MutableObject<>(targetOp);
LoadingJobBuilder jobBuilder = new LoadingJobBuilder(spec, sourceOpRef, targetOpRef);
- int posOfArrayIndex = findPosOfArrayIndex();
- ArrayIndexUtil.walkArrayPath(flattenedFieldNames.get(posOfArrayIndex), depthIndicators.get(posOfArrayIndex),
+ int posOfArrayElement = findPosOfArrayIndexElement();
+ ArrayIndexUtil.walkArrayPath(flattenedFieldNames.get(posOfArrayElement), unnestFlags.get(posOfArrayElement),
jobBuilder);
sourceOp = sourceOpRef.getValue();
if (anySecondaryKeyIsNullable || arrayIndexDetails.isOverridingKeyFieldTypes()) {
// If any of the secondary fields are nullable, then we need to filter out the nulls.
- targetOp = createFilterNullsSelectOp(spec, numTotalSecondaryKeys, secondaryRecDesc);
+ targetOp = createFilterAnyUnknownSelectOp(spec, numTotalSecondaryKeys, secondaryRecDesc);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
}
@@ -450,8 +465,7 @@
: inputWidth + numTotalSecondaryKeys + numFilterFields).toArray();
for (int i = 0; i < numTotalSecondaryKeys; i++) {
int sizeOfFieldNamesForI = flattenedFieldNames.get(i).size();
- if (depthIndicators.get(i).get(sizeOfFieldNamesForI - 1) != 0
- && (depthIndicators.get(i).stream().anyMatch(b -> b > 0))) {
+ if (unnestFlags.get(i).get(sizeOfFieldNamesForI - 1)) {
projectionList[i] = numPrimaryKeys + 1;
} else {
projectionList[i] = outColumns[outColumnsCursor++];
@@ -469,9 +483,9 @@
outColumns = IntStream.range(inputWidth, inputWidth + numArraySecondaryKeys).toArray();
for (int i = 0; i < numTotalSecondaryKeys; i++) {
int sizeOfFieldNamesForI = flattenedFieldNames.get(i).size();
- if (depthIndicators.get(i).stream().noneMatch(b -> b > 0)) {
+ if (unnestFlags.get(i).stream().noneMatch(b -> b)) {
projectionList[i] = numPrimaryKeys + atomicSKCursor++;
- } else if (depthIndicators.get(i).get(sizeOfFieldNamesForI - 1) == 0) {
+ } else if (!unnestFlags.get(i).get(sizeOfFieldNamesForI - 1)) {
projectionList[i] = outColumns[arraySKCursor++];
} else {
projectionList[i] = numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields + 1;
@@ -501,7 +515,7 @@
}
private void addAtomicFieldToBuilder(ARecordType recordType, int indexPos) throws AlgebricksException {
- IAType workingType = Index.getNonNullableOpenFieldType(flattenedKeyTypes.get(indexPos),
+ IAType workingType = Index.getNonNullableOpenFieldType(index, flattenedKeyTypes.get(indexPos),
flattenedFieldNames.get(indexPos), recordType).first;
IScalarEvaluatorFactory sef =
metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(),
@@ -513,7 +527,7 @@
IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
metadataProvider.getFunctionManager(), recordType, filterFieldName, numPrimaryKeys, sourceLoc);
evalFactoryAndRecDescStackBuilder.addFilter(sef,
- Index.getNonNullableKeyFieldType(filterFieldName, itemType).first);
+ Index.getNonNullableKeyFieldType(filterFieldName, recordType).first);
}
class EvalFactoryAndRecDescInvoker implements ArrayIndexUtil.TypeTrackerCommandExecutor {
@@ -525,25 +539,20 @@
@Override
public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
- List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
- boolean isLastUnnestInIntermediateStep) throws AlgebricksException {
+ List<String> fieldName, boolean isFirstArrayStep, boolean isLastUnnestInIntermediateStep)
+ throws AlgebricksException {
if (!this.isFirstWalk) {
// We have already added the appropriate UNNESTs.
return;
}
int sourceColumnForNestedArrays = numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields;
- if (isFirstUnnestInStep) {
- int sourceColumnForFirstUnnestInAtomicPath =
- isFirstArrayStep ? numPrimaryKeys : sourceColumnForNestedArrays;
- IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
- metadataProvider.getFunctionManager(), startingStepRecordType, fieldName,
- sourceColumnForFirstUnnestInAtomicPath, sourceLoc);
- evalFactoryAndRecDescStackBuilder.addUnnest(sef, workingType);
- } else {
- IScalarEvaluatorFactory sef = new ColumnAccessEvalFactory(sourceColumnForNestedArrays);
- evalFactoryAndRecDescStackBuilder.addUnnest(sef, workingType);
- }
+ int sourceColumnForFirstUnnestInAtomicPath =
+ isFirstArrayStep ? numPrimaryKeys : sourceColumnForNestedArrays;
+ IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+ metadataProvider.getFunctionManager(), startingStepRecordType, fieldName,
+ sourceColumnForFirstUnnestInAtomicPath, sourceLoc);
+ evalFactoryAndRecDescStackBuilder.addUnnest(sef, workingType);
}
@Override
@@ -564,8 +573,8 @@
}
class LoadingJobBuilder implements ArrayIndexUtil.ActionCounterCommandExecutor {
- private final Stack<RecordDescriptor> recDescStack = evalFactoryAndRecDescStackBuilder.buildRecDescStack();
- private final Stack<List<IScalarEvaluatorFactory>> sefStack =
+ private final Deque<RecordDescriptor> recDescStack = evalFactoryAndRecDescStackBuilder.buildRecDescStack();
+ private final Deque<List<IScalarEvaluatorFactory>> sefStack =
evalFactoryAndRecDescStackBuilder.buildEvalFactoryStack();
private final JobSpecification spec;
@@ -588,9 +597,9 @@
@Override
public void executeActionOnFirstArrayStep() throws AlgebricksException {
+ IScalarEvaluatorFactory sef = sefStack.pop().get(0);
nextRecDesc = recDescStack.pop();
- targetOpRef
- .setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sefStack.pop().get(0), nextRecDesc));
+ targetOpRef.setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sef, nextRecDesc));
connectAndMoveToNextOp();
}
@@ -604,45 +613,57 @@
workingRecDesc.getFieldCount(), sefStack.pop(), nextRecDesc));
connectAndMoveToNextOp();
+ IScalarEvaluatorFactory sef = sefStack.pop().get(0);
nextRecDesc = recDescStack.pop();
- targetOpRef
- .setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sefStack.pop().get(0), nextRecDesc));
+ targetOpRef.setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sef, nextRecDesc));
connectAndMoveToNextOp();
}
@Override
- public void executeActionOnFinalArrayStep(int numberOfActionsAlreadyPerformed) {
+ public void executeActionOnFinalArrayStep(int numberOfActionsAlreadyPerformed) throws AlgebricksException {
+ nextRecDesc = recDescStack.pop();
targetOpRef.setValue(createFinalAssignOp(spec, numberOfActionsAlreadyPerformed < 2,
- workingRecDesc.getFieldCount(), sefStack.pop(), recDescStack.pop()));
+ workingRecDesc.getFieldCount(), sefStack.pop(), nextRecDesc));
connectAndMoveToNextOp();
}
}
class EvalFactoryAndRecDescStackBuilder {
- private final Stack<IScalarEvaluatorFactory> unnestEvalFactories = new Stack<>();
- private final List<IScalarEvaluatorFactory> atomicSKEvalFactories = new ArrayList<>();
- private final List<IScalarEvaluatorFactory> finalArraySKEvalFactories = new ArrayList<>();
+ class EvalFactoryAndPosition {
+ final IScalarEvaluatorFactory scalarEvaluatorFactory;
+ final int position;
+
+ EvalFactoryAndPosition(IScalarEvaluatorFactory scalarEvaluatorFactory) {
+ this.scalarEvaluatorFactory = scalarEvaluatorFactory;
+ this.position = workingPosition++;
+ }
+ }
+
+ private final Deque<EvalFactoryAndPosition> unnestEvalFactories = new ArrayDeque<>();
+ private final List<EvalFactoryAndPosition> atomicSKEvalFactories = new ArrayList<>();
+ private final List<EvalFactoryAndPosition> finalArraySKEvalFactories = new ArrayList<>();
private final Queue<IAType> unnestEvalTypes = new LinkedList<>();
private final List<IAType> atomicSKEvalTypes = new ArrayList<>();
- private IScalarEvaluatorFactory filterEvalFactory = null;
+ private EvalFactoryAndPosition filterEvalFactory = null;
private IAType filterEvalType = null;
+ private int workingPosition = 0;
public void addAtomicSK(IScalarEvaluatorFactory sef, IAType type) {
- atomicSKEvalFactories.add(sef);
+ atomicSKEvalFactories.add(new EvalFactoryAndPosition(sef));
atomicSKEvalTypes.add(type);
}
public void addFilter(IScalarEvaluatorFactory sef, IAType type) {
- filterEvalFactory = sef;
+ filterEvalFactory = new EvalFactoryAndPosition(sef);
filterEvalType = type;
}
public void addFinalArraySK(IScalarEvaluatorFactory sef) {
- finalArraySKEvalFactories.add(sef);
+ finalArraySKEvalFactories.add(new EvalFactoryAndPosition(sef));
}
public void addUnnest(IScalarEvaluatorFactory sef, IAType type) {
- unnestEvalFactories.push(sef);
+ unnestEvalFactories.push(new EvalFactoryAndPosition(sef));
unnestEvalTypes.add(type);
}
@@ -662,8 +683,8 @@
* [ final ASSIGN SEFs -- array SKs (record accessors) ---------------- ]
* </pre>
*/
- public Stack<List<IScalarEvaluatorFactory>> buildEvalFactoryStack() {
- Stack<List<IScalarEvaluatorFactory>> resultant = new Stack<>();
+ public Deque<List<IScalarEvaluatorFactory>> buildEvalFactoryStack() {
+ Deque<List<EvalFactoryAndPosition>> resultant = new ArrayDeque<>();
resultant.push(finalArraySKEvalFactories);
int initialUnnestEvalFactorySize = unnestEvalFactories.size();
for (int i = 0; i < initialUnnestEvalFactorySize - 1; i++) {
@@ -675,12 +696,22 @@
resultant.push(new ArrayList<>());
}
}
+
+ // Sort the SEFs according to the index order.
resultant.peek().addAll(atomicSKEvalFactories);
+ List<EvalFactoryAndPosition> reorderedSEFs = new ArrayList<>(Objects.requireNonNull(resultant.peek()));
+ reorderedSEFs.sort(Comparator.comparingInt(s -> s.position));
+ resultant.pop();
+ resultant.push(reorderedSEFs);
+
+ // Append our filter eval factory last.
if (filterEvalFactory != null) {
resultant.peek().add(filterEvalFactory);
}
resultant.push(Collections.singletonList(unnestEvalFactories.pop()));
- return resultant;
+ return resultant.stream()
+ .map(l -> l.stream().map(s -> s.scalarEvaluatorFactory).collect(Collectors.toList()))
+ .collect(Collectors.toCollection(ArrayDeque::new));
}
/**
@@ -697,18 +728,26 @@
* [ secondary record descriptor ------------------------------------- ]
* </pre>
*/
- public Stack<RecordDescriptor> buildRecDescStack() throws AlgebricksException {
+ public Deque<RecordDescriptor> buildRecDescStack() throws AlgebricksException {
int initialUnnestEvalTypesSize = unnestEvalTypes.size();
- Deque<RecordDescriptor> resultantAsDeque = new ArrayDeque<>();
- resultantAsDeque.addFirst(primaryRecDesc);
- resultantAsDeque.addFirst(createUnnestRecDesc(primaryRecDesc, unnestEvalTypes.remove()));
- for (int i = 0; i < initialUnnestEvalTypesSize - 1; i++) {
- resultantAsDeque.addFirst(createAssignRecDesc(resultantAsDeque.getFirst(), i == 0));
- resultantAsDeque.addFirst(createUnnestRecDesc(resultantAsDeque.getFirst(), unnestEvalTypes.remove()));
+ Deque<RecordDescriptor> resultant = new ArrayDeque<>();
+ RecordDescriptor recDescBeforeFirstUnnest = primaryRecDesc;
+ if (dataset.hasMetaPart()) {
+ ISerializerDeserializer[] fields = new ISerializerDeserializer[primaryRecDesc.getFieldCount() - 1];
+ ITypeTraits[] typeTraits = new ITypeTraits[primaryRecDesc.getFieldCount() - 1];
+ for (int i = 0; i < primaryRecDesc.getFieldCount() - 1; i++) {
+ fields[i] = primaryRecDesc.getFields()[i];
+ typeTraits[i] = primaryRecDesc.getTypeTraits()[i];
+ }
+ recDescBeforeFirstUnnest = new RecordDescriptor(fields, typeTraits);
}
- resultantAsDeque.addFirst(secondaryRecDesc);
- Stack<RecordDescriptor> resultant = new Stack<>();
- resultant.addAll(resultantAsDeque);
+ resultant.addLast(recDescBeforeFirstUnnest);
+ resultant.addLast(createUnnestRecDesc(recDescBeforeFirstUnnest, unnestEvalTypes.remove()));
+ for (int i = 0; i < initialUnnestEvalTypesSize - 1; i++) {
+ resultant.addLast(createAssignRecDesc(resultant.getLast(), i == 0));
+ resultant.addLast(createUnnestRecDesc(resultant.getLast(), unnestEvalTypes.remove()));
+ }
+ resultant.addLast(secondaryRecDesc);
return resultant;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index 33f5b62..3d4fb16 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -48,7 +48,6 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -66,10 +65,10 @@
public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
- boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
int[] fieldPermutation = createFieldPermutationForBulkLoadOp(indexDetails.getKeyFieldNames().size());
IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
+ boolean excludeUnknown = excludeUnknownKeys(index, indexDetails, anySecondaryKeyIsNullable);
if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
/*
* In case of external data,
@@ -81,20 +80,15 @@
ExternalScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
// Assign op.
- AbstractOperatorDescriptor sourceOp = primaryScanOp;
- if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
- sourceOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
- spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
- }
AlgebricksMetaOperatorDescriptor asterixAssignOp =
createExternalAssignOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
// If any of the secondary fields are nullable, then add a select op that filters nulls.
AlgebricksMetaOperatorDescriptor selectOp = null;
- if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
- selectOp = createFilterNullsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
+ if (excludeUnknown) {
+ selectOp =
+ createFilterAllUnknownsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
}
-
// Sort by secondary keys.
ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
// Create secondary BTree bulk load op.
@@ -116,8 +110,8 @@
metaOp.setSourceLocation(sourceLoc);
spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
root = metaOp;
- spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
- if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+ if (excludeUnknown) {
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
} else {
@@ -129,7 +123,7 @@
return spec;
} else {
// job spec:
- // key provider -> primary idx -> (cast assign)? -> assign -> (select)? -> (sort)? -> bulk load -> sink
+ // key provider -> primary idx scan -> cast assign -> (select)? -> (sort)? -> bulk load -> sink
IndexUtil.bindJobEventListener(spec, metadataProvider);
// dummy key provider ----> primary index scan
@@ -138,35 +132,28 @@
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
- if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
- // primary index scan ----> cast assign
- targetOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
- spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
- sourceOp = targetOp;
- }
- // primary index OR cast assign ----> assign op
+ // primary index ----> cast assign op (produces the secondary index entry)
targetOp = createAssignOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
- if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
+ if (excludeUnknown) {
// if any of the secondary fields are nullable, then add a select op that filters nulls.
// assign op ----> select op
- targetOp = createFilterNullsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
+ targetOp =
+ createFilterAllUnknownsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
}
-
// no need to sort if the index is secondary primary index
if (!indexDetails.getKeyFieldNames().isEmpty()) {
- // sort by secondary keys.
- // assign op OR select op ----> sort op
+ // sort by <SKs,PKs>. cast assign op OR select op ----> sort op
targetOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
}
- // assign op OR select op OR sort op ----> bulk load op
+ // cast assign op OR select op OR sort op ----> bulk load op
targetOp = createTreeIndexBulkLoadOp(spec, fieldPermutation, dataflowHelperFactory,
StorageConstants.DEFAULT_TREE_FILL_FACTOR);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
@@ -210,12 +197,12 @@
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
- int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
+ int numSecondaryKeys = getNumSecondaryKeys();
secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
ISerializerDeserializer[] secondaryRecFields =
- new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys + numFilterFields];
+ new ISerializerDeserializer[numSecondaryKeys + numPrimaryKeys + numFilterFields];
ISerializerDeserializer[] enforcedRecFields =
new ISerializerDeserializer[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
ITypeTraits[] enforcedTypeTraits =
@@ -230,24 +217,28 @@
boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
+ ARecordType enforcedType;
int sourceColumn;
List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = itemType;
sourceColumn = recordColumn;
+ enforcedType = enforcedItemType;
} else {
sourceType = metaType;
sourceColumn = recordColumn + 1;
+ enforcedType = enforcedMetaType;
}
- secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
- metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : sourceType,
- indexDetails.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
- indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
+ List<String> secFieldName = indexDetails.getKeyFieldNames().get(i);
+ IAType secFieldType = indexDetails.getKeyFieldTypes().get(i);
+ Pair<IAType, Boolean> keyTypePair =
+ Index.getNonNullableOpenFieldType(index, secFieldType, secFieldName, sourceType);
IAType keyType = keyTypePair.first;
+ IScalarEvaluatorFactory secFieldAccessor = createFieldAccessor(sourceType, sourceColumn, secFieldName);
+ secondaryFieldAccessEvalFactories[i] =
+ createFieldCast(secFieldAccessor, isOverridingKeyFieldTypes, enforcedType, sourceType, keyType);
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
- ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
- secondaryRecFields[i] = keySerde;
+ secondaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
secondaryBloomFilterKeyFields[i] = i;
@@ -279,20 +270,29 @@
}
if (numFilterFields > 0) {
- ARecordType filterItemType =
- ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterSourceIndicator() == 0 ? itemType
- : metaType;
- secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getDataFormat()
- .getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(), filterItemType,
- filterFieldName, numPrimaryKeys, sourceLoc);
- Pair<IAType, Boolean> keyTypePair;
- keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, filterItemType);
- IAType type = keyTypePair.first;
- ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
+ Integer filterSourceIndicator =
+ ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterSourceIndicator();
+ ARecordType sourceType;
+ ARecordType enforcedType;
+ int sourceColumn;
+ if (filterSourceIndicator == null || filterSourceIndicator == 0) {
+ sourceType = itemType;
+ sourceColumn = recordColumn;
+ enforcedType = enforcedItemType;
+ } else {
+ sourceType = metaType;
+ sourceColumn = recordColumn + 1;
+ enforcedType = enforcedMetaType;
+ }
+ IAType filterType = Index.getNonNullableKeyFieldType(filterFieldName, sourceType).first;
+ IScalarEvaluatorFactory filterAccessor = createFieldAccessor(sourceType, sourceColumn, filterFieldName);
+ secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+ createFieldCast(filterAccessor, isOverridingKeyFieldTypes, enforcedType, sourceType, filterType);
+ ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(filterType);
secondaryRecFields[numPrimaryKeys + numSecondaryKeys] = serde;
enforcedRecFields[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] = serde;
enforcedTypeTraits[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] =
- typeTraitProvider.getTypeTrait(type);
+ typeTraitProvider.getTypeTrait(filterType);
}
secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
enforcedRecDesc = new RecordDescriptor(enforcedRecFields, enforcedTypeTraits);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index aca953b..c596137 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -67,8 +67,7 @@
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
- boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
-
+ boolean excludeUnknown = excludeUnknownKeys(index, indexDetails, anySecondaryKeyIsNullable);
assert dataset.getDatasetType() == DatasetType.INTERNAL;
// only handle internal datasets
@@ -83,18 +82,13 @@
getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)));
// Assign op.
- IOperatorDescriptor sourceOp = primaryScanOp;
- if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
- sourceOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
- spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
- }
RecordDescriptor taggedSecondaryRecDesc = getTaggedRecordDescriptor(secondaryRecDesc);
AlgebricksMetaOperatorDescriptor asterixAssignOp =
createAssignOp(spec, indexDetails.getKeyFieldNames().size(), taggedSecondaryRecDesc);
// Generate compensate tuples for upsert
- IOperatorDescriptor processorOp =
- createTupleProcessorOp(spec, taggedSecondaryRecDesc, getNumSecondaryKeys(), numPrimaryKeys, false);
+ IOperatorDescriptor processorOp = createTupleProcessorOp(spec, taggedSecondaryRecDesc, getNumSecondaryKeys(),
+ numPrimaryKeys, false, excludeUnknown, false);
ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
getTaggedSecondaryComparatorFactories(secondaryComparatorFactories), taggedSecondaryRecDesc);
@@ -111,7 +105,7 @@
metaOp.setSourceLocation(sourceLoc);
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, asterixAssignOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, processorOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), processorOp, 0, sortOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
@@ -129,12 +123,12 @@
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
- int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
+ int numSecondaryKeys = getNumSecondaryKeys();
secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
ISerializerDeserializer[] secondaryRecFields =
- new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys + numFilterFields];
+ new ISerializerDeserializer[numSecondaryKeys + numPrimaryKeys + numFilterFields];
ISerializerDeserializer[] enforcedRecFields =
new ISerializerDeserializer[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
ITypeTraits[] enforcedTypeTraits =
@@ -144,29 +138,32 @@
ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
IBinaryComparatorFactoryProvider comparatorFactoryProvider =
metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
- // Record column is 0 for external datasets, numPrimaryKeys for internal ones
int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
boolean isOverridingKeyTypes = indexDetails.isOverridingKeyFieldTypes();
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
+ ARecordType enforcedType;
int sourceColumn;
List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = itemType;
sourceColumn = recordColumn;
+ enforcedType = enforcedItemType;
} else {
sourceType = metaType;
sourceColumn = recordColumn + 1;
+ enforcedType = enforcedMetaType;
}
- secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
- metadataProvider.getFunctionManager(), isOverridingKeyTypes ? enforcedItemType : sourceType,
- indexDetails.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
- indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
+ List<String> secFieldName = indexDetails.getKeyFieldNames().get(i);
+ IAType secFieldType = indexDetails.getKeyFieldTypes().get(i);
+ Pair<IAType, Boolean> keyTypePair =
+ Index.getNonNullableOpenFieldType(index, secFieldType, secFieldName, sourceType);
IAType keyType = keyTypePair.first;
+ IScalarEvaluatorFactory secFieldAccessor = createFieldAccessor(sourceType, sourceColumn, secFieldName);
+ secondaryFieldAccessEvalFactories[i] =
+ createFieldCast(secFieldAccessor, isOverridingKeyTypes, enforcedType, sourceType, keyType);
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
- ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
- secondaryRecFields[i] = keySerde;
+ secondaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
secondaryBloomFilterKeyFields[i] = i;
@@ -188,20 +185,29 @@
}
if (numFilterFields > 0) {
- ARecordType filterItemType =
- ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterSourceIndicator() == 0 ? itemType
- : metaType;
- secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getDataFormat()
- .getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(), filterItemType,
- filterFieldName, recordColumn, sourceLoc);
- Pair<IAType, Boolean> keyTypePair;
- keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, filterItemType);
- IAType type = keyTypePair.first;
- ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
+ Integer filterSourceIndicator =
+ ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterSourceIndicator();
+ ARecordType sourceType;
+ ARecordType enforcedType;
+ int sourceColumn;
+ if (filterSourceIndicator == null || filterSourceIndicator == 0) {
+ sourceType = itemType;
+ sourceColumn = recordColumn;
+ enforcedType = enforcedItemType;
+ } else {
+ sourceType = metaType;
+ sourceColumn = recordColumn + 1;
+ enforcedType = enforcedMetaType;
+ }
+ IAType filterType = Index.getNonNullableKeyFieldType(filterFieldName, sourceType).first;
+ IScalarEvaluatorFactory filterAccessor = createFieldAccessor(sourceType, sourceColumn, filterFieldName);
+ secondaryFieldAccessEvalFactories[numSecondaryKeys] =
+ createFieldCast(filterAccessor, isOverridingKeyTypes, enforcedType, sourceType, filterType);
+ ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(filterType);
secondaryRecFields[numPrimaryKeys + numSecondaryKeys] = serde;
enforcedRecFields[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] = serde;
enforcedTypeTraits[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] =
- typeTraitProvider.getTypeTrait(type);
+ typeTraitProvider.getTypeTrait(filterType);
}
secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
enforcedRecDesc = new RecordDescriptor(enforcedRecFields, enforcedTypeTraits);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index c111f0e..00cc595 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -116,7 +116,7 @@
secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
indexDetails.getKeyFieldNames().get(0), recordColumn, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), itemType);
secondaryKeyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
@@ -239,8 +239,8 @@
createAssignOp(spec, numSecondaryKeys, taggedSecondaryRecDesc);
// Generate compensate tuples for upsert
- IOperatorDescriptor processorOp =
- createTupleProcessorOp(spec, taggedSecondaryRecDesc, numSecondaryKeys, numPrimaryKeys, true);
+ IOperatorDescriptor processorOp = createTupleProcessorOp(spec, taggedSecondaryRecDesc, numSecondaryKeys,
+ numPrimaryKeys, true, true, true);
// Create a tokenizer op.
AbstractOperatorDescriptor tokenizerOp = createTokenizerOp(spec);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index 82c67d7..e26aab3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -83,7 +83,7 @@
throw AsterixException.create(ErrorCode.INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED, sourceLoc,
numSecondaryKeys);
}
- Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
IAType spatialType = spatialTypePair.first;
anySecondaryKeyIsNullable = spatialTypePair.second;
@@ -207,7 +207,7 @@
// Generate compensate tuples for upsert
IOperatorDescriptor processorOp = createTupleProcessorOp(spec, secondaryRecDescConsideringPointMBR,
- numNestedSecondaryKeFieldsConsideringPointMBR, numPrimaryKeys, false);
+ numNestedSecondaryKeFieldsConsideringPointMBR, numPrimaryKeys, false, true, true);
ExternalSortOperatorDescriptor sortOp = createSortOp(spec,
getTaggedSecondaryComparatorFactories(new IBinaryComparatorFactory[] {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index 06df020..1a65869 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -247,10 +247,11 @@
}
protected IOperatorDescriptor createTupleProcessorOp(JobSpecification spec, RecordDescriptor taggedSecondaryRecDesc,
- int numSecondaryKeyFields, int numPrimaryKeyFields, boolean hasBuddyBTree) {
+ int numSecondaryKeyFields, int numPrimaryKeyFields, boolean hasBuddyBTree, boolean excludeUnknownKeys,
+ boolean forAnyUnknownKey) {
IOperatorDescriptor op = new LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor(spec,
taggedSecondaryRecDesc, MissingWriterFactory.INSTANCE, NUM_TAG_FIELDS, numSecondaryKeyFields,
- numPrimaryKeyFields, hasBuddyBTree);
+ numPrimaryKeyFields, hasBuddyBTree, excludeUnknownKeys, forAnyUnknownKey);
op.setSourceLocation(sourceLoc);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, primaryPartitionConstraint);
return op;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index a6e3087..9e0d650 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -23,6 +23,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.function.Supplier;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
@@ -34,6 +35,7 @@
import org.apache.asterix.external.operators.ExternalIndexBulkLoadOperatorDescriptor;
import org.apache.asterix.external.operators.ExternalIndexBulkModifyOperatorDescriptor;
import org.apache.asterix.external.operators.ExternalScanOperatorDescriptor;
+import org.apache.asterix.formats.base.IDataFormat;
import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -43,19 +45,27 @@
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.AbstractFunctionDescriptor;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionManager;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
import org.apache.asterix.runtime.evaluators.functions.IsUnknownDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NotDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor;
import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -279,6 +289,63 @@
primaryRecDesc = new RecordDescriptor(primaryRecFields, primaryTypeTraits);
}
+ protected IScalarEvaluatorFactory createFieldAccessor(ARecordType recordType, int recordColumn,
+ List<String> fieldName) throws AlgebricksException {
+ IFunctionManager funManger = metadataProvider.getFunctionManager();
+ IDataFormat dataFormat = metadataProvider.getDataFormat();
+ return dataFormat.getFieldAccessEvaluatorFactory(funManger, recordType, fieldName, recordColumn, sourceLoc);
+ }
+
+ protected IScalarEvaluatorFactory createFieldCast(IScalarEvaluatorFactory fieldEvalFactory,
+ boolean isOverridingKeyFieldTypes, IAType enforcedRecordType, ARecordType recordType, IAType targetType)
+ throws AlgebricksException {
+ IFunctionManager funManger = metadataProvider.getFunctionManager();
+ IDataFormat dataFormat = metadataProvider.getDataFormat();
+
+ // check IndexUtil.castDefaultNull(index), too, because we always want to cast even if the overriding type is
+ // the same as the overridden type (this is for the case where overriding the type of closed field is allowed)
+ // e.g. field "a" is a string in the dataset ds; CREATE INDEX .. ON ds(a:string) CAST (DEFAULT NULL)
+ boolean castIndexedField = isOverridingKeyFieldTypes
+ && (!enforcedRecordType.equals(recordType) || IndexUtil.castDefaultNull(index));
+ if (!castIndexedField) {
+ return fieldEvalFactory;
+ }
+
+ IScalarEvaluatorFactory castFieldEvalFactory;
+ if (IndexUtil.castDefaultNull(index)) {
+ castFieldEvalFactory = createConstructorFunction(funManger, dataFormat, fieldEvalFactory, targetType);
+ } else if (index.isEnforced()) {
+ IScalarEvaluatorFactory[] castArg = new IScalarEvaluatorFactory[] { fieldEvalFactory };
+ castFieldEvalFactory =
+ createCastFunction(targetType, BuiltinType.ANY, true, sourceLoc).createEvaluatorFactory(castArg);
+ } else {
+ IScalarEvaluatorFactory[] castArg = new IScalarEvaluatorFactory[] { fieldEvalFactory };
+ castFieldEvalFactory =
+ createCastFunction(targetType, BuiltinType.ANY, false, sourceLoc).createEvaluatorFactory(castArg);
+ }
+ return castFieldEvalFactory;
+ }
+
+ protected IScalarEvaluatorFactory createConstructorFunction(IFunctionManager funManager, IDataFormat dataFormat,
+ IScalarEvaluatorFactory fieldEvalFactory, IAType fieldType) throws AlgebricksException {
+ IAType targetType = TypeComputeUtils.getActualType(fieldType);
+ Pair<FunctionIdentifier, IAObject> constructorWithFmt =
+ IndexUtil.getTypeConstructorDefaultNull(index, targetType, sourceLoc);
+ FunctionIdentifier typeConstructorFun = constructorWithFmt.first;
+ IFunctionDescriptor typeConstructor = funManager.lookupFunction(typeConstructorFun, sourceLoc);
+ IScalarEvaluatorFactory[] args;
+ // add the format argument if specified
+ if (constructorWithFmt.second != null) {
+ IScalarEvaluatorFactory fmtEvalFactory =
+ dataFormat.getConstantEvalFactory(new AsterixConstantValue(constructorWithFmt.second));
+ args = new IScalarEvaluatorFactory[] { fieldEvalFactory, fmtEvalFactory };
+ } else {
+ args = new IScalarEvaluatorFactory[] { fieldEvalFactory };
+ }
+ typeConstructor.setSourceLocation(sourceLoc);
+ return typeConstructor.createEvaluatorFactory(args);
+ }
+
protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec, int numSecondaryKeyFields,
RecordDescriptor secondaryRecDesc) throws AlgebricksException {
int[] outColumns = new int[numSecondaryKeyFields + numFilterFields];
@@ -340,11 +407,17 @@
new RecordDescriptor[] { enforcedRecDesc });
}
- IFunctionDescriptor createCastFunction(boolean strictCast, SourceLocation sourceLoc) throws AlgebricksException {
+ protected IFunctionDescriptor createCastFunction(boolean strictCast, SourceLocation sourceLoc)
+ throws AlgebricksException {
+ return createCastFunction(enforcedItemType, itemType, strictCast, sourceLoc);
+ }
+
+ protected IFunctionDescriptor createCastFunction(IAType targetType, IAType inputType, boolean strictCast,
+ SourceLocation sourceLoc) throws AlgebricksException {
IFunctionDescriptor castFuncDesc = metadataProvider.getFunctionManager()
.lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX, sourceLoc);
castFuncDesc.setSourceLocation(sourceLoc);
- castFuncDesc.setImmutableStates(enforcedItemType, itemType);
+ castFuncDesc.setImmutableStates(targetType, inputType);
return castFuncDesc;
}
@@ -361,6 +434,11 @@
return sortOp;
}
+ protected static boolean excludeUnknownKeys(Index index, Index.ValueIndexDetails details,
+ boolean anySecKeyIsNullable) {
+ return IndexUtil.excludesUnknowns(index) && (anySecKeyIsNullable || details.isOverridingKeyFieldTypes());
+ }
+
protected LSMIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
int[] fieldPermutation, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor) {
IndexDataflowHelperFactory primaryIndexDataflowHelperFactory = new IndexDataflowHelperFactory(
@@ -386,9 +464,20 @@
return treeIndexBulkLoadOp;
}
- public AlgebricksMetaOperatorDescriptor createFilterNullsSelectOp(JobSpecification spec, int numSecondaryKeyFields,
- RecordDescriptor secondaryRecDesc) {
- IScalarEvaluatorFactory[] andArgsEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeyFields];
+ public AlgebricksMetaOperatorDescriptor createFilterAllUnknownsSelectOp(JobSpecification spec,
+ int numSecondaryKeyFields, RecordDescriptor secondaryRecDesc) throws AlgebricksException {
+ return createFilterSelectOp(spec, numSecondaryKeyFields, secondaryRecDesc, OrDescriptor::new);
+ }
+
+ public AlgebricksMetaOperatorDescriptor createFilterAnyUnknownSelectOp(JobSpecification spec,
+ int numSecondaryKeyFields, RecordDescriptor secondaryRecDesc) throws AlgebricksException {
+ return createFilterSelectOp(spec, numSecondaryKeyFields, secondaryRecDesc, AndDescriptor::new);
+ }
+
+ private AlgebricksMetaOperatorDescriptor createFilterSelectOp(JobSpecification spec, int numSecondaryKeyFields,
+ RecordDescriptor secondaryRecDesc, Supplier<AbstractFunctionDescriptor> predicatesCombinerFuncSupplier)
+ throws AlgebricksException {
+ IScalarEvaluatorFactory[] predicateArgsEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeyFields];
NotDescriptor notDesc = new NotDescriptor();
notDesc.setSourceLocation(sourceLoc);
IsUnknownDescriptor isUnknownDesc = new IsUnknownDescriptor();
@@ -400,17 +489,15 @@
isUnknownDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { columnAccessEvalFactory });
IScalarEvaluatorFactory notEvalFactory =
notDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { isUnknownEvalFactory });
- andArgsEvalFactories[i] = notEvalFactory;
+ predicateArgsEvalFactories[i] = notEvalFactory;
}
IScalarEvaluatorFactory selectCond;
if (numSecondaryKeyFields > 1) {
- // Create conjunctive condition where all secondary index keys must
- // satisfy 'is not null'.
- AndDescriptor andDesc = new AndDescriptor();
- andDesc.setSourceLocation(sourceLoc);
- selectCond = andDesc.createEvaluatorFactory(andArgsEvalFactories);
+ AbstractFunctionDescriptor predicatesCombiner = predicatesCombinerFuncSupplier.get();
+ predicatesCombiner.setSourceLocation(sourceLoc);
+ selectCond = predicatesCombiner.createEvaluatorFactory(predicateArgsEvalFactories);
} else {
- selectCond = andArgsEvalFactories[0];
+ selectCond = predicateArgsEvalFactories[0];
}
StreamSelectRuntimeFactory select =
new StreamSelectRuntimeFactory(selectCond, null, BinaryBooleanInspector.FACTORY, false, -1, null);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 2d5a4f8..a84454f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -120,7 +120,7 @@
secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
indexDetails.getKeyFieldNames().get(0), numPrimaryKeys, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), itemType);
secondaryKeyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
@@ -236,7 +236,7 @@
// that filters nulls.
AlgebricksMetaOperatorDescriptor selectOp = null;
if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
- selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys, secondaryRecDesc);
+ selectOp = createFilterAnyUnknownSelectOp(spec, numSecondaryKeys, secondaryRecDesc);
}
// Create a tokenizer op.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index dbbd723..e36e71f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -89,7 +89,7 @@
throw new AsterixException("Cannot use " + numSecondaryKeys + " fields as a key for the R-tree index. "
+ "There can be only one field as a key for the R-tree index.");
}
- Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index,
indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
IAType spatialType = spatialTypePair.first;
anySecondaryKeyIsNullable = spatialTypePair.second;
@@ -222,7 +222,7 @@
// If any of the secondary fields are nullable, then add a select op that filters nulls.
AlgebricksMetaOperatorDescriptor selectOp = null;
if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
- selectOp = createFilterNullsSelectOp(spec, numNestedSecondaryKeFieldsConsideringPointMBR,
+ selectOp = createFilterAnyUnknownSelectOp(spec, numNestedSecondaryKeFieldsConsideringPointMBR,
secondaryRecDescConsideringPointMBR);
}
@@ -273,7 +273,7 @@
// If any of the secondary fields are nullable, then add a select op that filters nulls.
AlgebricksMetaOperatorDescriptor selectOp = null;
if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
- selectOp = createFilterNullsSelectOp(spec, numNestedSecondaryKeFieldsConsideringPointMBR,
+ selectOp = createFilterAnyUnknownSelectOp(spec, numNestedSecondaryKeFieldsConsideringPointMBR,
secondaryRecDescConsideringPointMBR);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
index dd303fc..5201a6a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -33,18 +33,27 @@
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeSignature;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
/**
* Provider utility methods for data types
@@ -57,12 +66,124 @@
private static final String FUNCTION_INLINE_TYPE_PREFIX = "$f$t$";
+ public static final String DATETIME_PARAMETER_NAME = BuiltinType.ADATETIME.getTypeName();
+
+ public static final String DATE_PARAMETER_NAME = BuiltinType.ADATE.getTypeName();
+
+ public static final String TIME_PARAMETER_NAME = BuiltinType.ATIME.getTypeName();
+
private TypeUtil() {
}
+ /**
+ * @param type type
+ * @param withFormat true when needing a constructor with format
+ *
+ * @return a type constructor that produces NULL for MISSING input
+ */
+ public static FunctionIdentifier getTypeConstructorDefaultNull(IAType type, boolean withFormat) {
+ switch (type.getTypeTag()) {
+ case TINYINT:
+ return BuiltinFunctions.INT8_DEFAULT_NULL_CONSTRUCTOR;
+ case SMALLINT:
+ return BuiltinFunctions.INT16_DEFAULT_NULL_CONSTRUCTOR;
+ case INTEGER:
+ return BuiltinFunctions.INT32_DEFAULT_NULL_CONSTRUCTOR;
+ case BIGINT:
+ return BuiltinFunctions.INT64_DEFAULT_NULL_CONSTRUCTOR;
+ case FLOAT:
+ return BuiltinFunctions.FLOAT_DEFAULT_NULL_CONSTRUCTOR;
+ case DOUBLE:
+ return BuiltinFunctions.DOUBLE_DEFAULT_NULL_CONSTRUCTOR;
+ case BOOLEAN:
+ return BuiltinFunctions.BOOLEAN_DEFAULT_NULL_CONSTRUCTOR;
+ case STRING:
+ return BuiltinFunctions.STRING_DEFAULT_NULL_CONSTRUCTOR;
+ case DATE:
+ return withFormat ? BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT
+ : BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR;
+ case TIME:
+ return withFormat ? BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT
+ : BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR;
+ case DATETIME:
+ return withFormat ? BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT
+ : BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR;
+ case YEARMONTHDURATION:
+ return BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR;
+ case DAYTIMEDURATION:
+ return BuiltinFunctions.DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR;
+ case DURATION:
+ return BuiltinFunctions.DURATION_DEFAULT_NULL_CONSTRUCTOR;
+ case UUID:
+ return BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR;
+ case BINARY:
+ return BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR;
+ default:
+ return null;
+ }
+ }
+
+ public static Map<String, String> validateConfiguration(Map<String, String> config, SourceLocation sourceLoc)
+ throws CompilationException {
+ if (config == null) {
+ return Collections.emptyMap();
+ }
+ for (Map.Entry<String, String> me : config.entrySet()) {
+ String name = me.getKey();
+ String value = me.getValue();
+ if (DATETIME_PARAMETER_NAME.equals(name) || DATE_PARAMETER_NAME.equals(name)
+ || TIME_PARAMETER_NAME.equals(name)) {
+ if (value == null) {
+ throw new CompilationException(ErrorCode.INVALID_REQ_PARAM_VAL, sourceLoc, name, "null");
+ }
+ } else {
+ throw new CompilationException(ErrorCode.ILLEGAL_SET_PARAMETER, sourceLoc, name);
+ }
+ }
+ return config;
+ }
+
+ public static String getDatetimeFormat(Map<String, String> config) {
+ return config.get(DATETIME_PARAMETER_NAME);
+ }
+
+ public static String getDateFormat(Map<String, String> config) {
+ return config.get(DATE_PARAMETER_NAME);
+ }
+
+ public static String getTimeFormat(Map<String, String> config) {
+ return config.get(TIME_PARAMETER_NAME);
+ }
+
+ public static String getTemporalFormat(IAType targetType, Triple<String, String, String> temporalFormatByType) {
+ switch (targetType.getTypeTag()) {
+ case DATETIME:
+ return temporalFormatByType.first;
+ case DATE:
+ return temporalFormatByType.second;
+ case TIME:
+ return temporalFormatByType.third;
+ default:
+ return null;
+ }
+ }
+
+ public static IAObject getTemporalFormatArg(AbstractFunctionCallExpression funExpr) {
+ FunctionIdentifier funId = funExpr.getFunctionIdentifier();
+ if (BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT.equals(funId)
+ || BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT.equals(funId)
+ || BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT.equals(funId)) {
+ List<Mutable<ILogicalExpression>> arguments = funExpr.getArguments();
+ if (arguments.size() > 1) {
+ return ConstantExpressionUtil.getConstantIaObject(arguments.get(1).getValue(), null);
+ }
+ }
+ return null;
+ }
+
private static class EnforcedTypeBuilder {
- private final Deque<Triple<IAType, String, Integer>> typeStack = new ArrayDeque<>();
- private List<Integer> keyDepthIndicators;
+ private final Deque<Triple<IAType, String, Boolean>> typeStack = new ArrayDeque<>();
+ private List<Boolean> keyUnnestFlags;
private List<String> keyFieldNames;
private ARecordType baseRecordType;
private IAType keyFieldType;
@@ -71,13 +192,15 @@
private String bridgeNameFoundFromOpenTypeBuild;
private IAType endOfOpenTypeBuild;
private int indexOfOpenPart;
+ private boolean castDefaultNull;
- public void reset(ARecordType baseRecordType, List<String> keyFieldNames, List<Integer> keyDepthIndicators,
- IAType keyFieldType) {
+ public void reset(ARecordType baseRecordType, List<String> keyFieldNames, List<Boolean> keyUnnestFlags,
+ IAType keyFieldType, boolean castDefaultNull) {
this.baseRecordType = baseRecordType;
this.keyFieldNames = keyFieldNames;
- this.keyDepthIndicators = keyDepthIndicators;
+ this.keyUnnestFlags = keyUnnestFlags;
this.keyFieldType = keyFieldType;
+ this.castDefaultNull = castDefaultNull;
}
public ARecordType build() throws AlgebricksException {
@@ -90,21 +213,19 @@
IAType typeIntermediate = baseRecordType;
List<String> subFieldName = new ArrayList<>();
for (int i = 0; i < keyFieldNames.size() - 1; i++) {
- typeStack.push(new Triple<>(typeIntermediate, keyFieldNames.get(i),
- (i == 0) ? 0 : keyDepthIndicators.get(i - 1)));
+ typeStack.push(
+ new Triple<>(typeIntermediate, keyFieldNames.get(i), i != 0 && keyUnnestFlags.get(i - 1)));
bridgeNameFoundFromOpenTypeBuild = typeIntermediate.getTypeName();
- if (i == 0 || keyDepthIndicators.get(i - 1) == 0) {
+ if (i == 0 || !keyUnnestFlags.get(i - 1)) {
subFieldName.add(keyFieldNames.get(i));
} else {
- // We have a multi-valued intermediate. Traverse the array first, then add our field name.
- for (int j = 0; j < keyDepthIndicators.get(i - 1); j++) {
- typeIntermediate = TypeComputeUtils.extractListItemType(typeIntermediate);
- if (typeIntermediate == null) {
- String fName = String.join(".", subFieldName);
- throw new AsterixException(ErrorCode.COMPILATION_ERROR,
- "Wrong level of array nesting for field: " + fName);
- }
+ // We have a multi-valued intermediate. Perform our UNNEST then add our field name.
+ typeIntermediate = TypeComputeUtils.extractListItemType(typeIntermediate);
+ if (typeIntermediate == null) {
+ String fName = String.join(".", subFieldName);
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+ "No list item type found. Wrong type given from field " + fName);
}
subFieldName.add(keyFieldNames.get(i));
}
@@ -133,33 +254,36 @@
}
private IAType buildNewForOpenType() {
- int depthOfOpenType = keyDepthIndicators.subList(indexOfOpenPart + 1, keyDepthIndicators.size()).stream()
- .filter(i -> i != 0).findFirst().orElse(0);
- IAType resultant = nestArrayType(keyFieldType, depthOfOpenType);
+ // Walk backwards through our flags and construct the desired type.
+ List<Boolean> unnestFlagsForOpenType = keyUnnestFlags.subList(indexOfOpenPart, keyUnnestFlags.size());
+ List<String> fieldNamesForOpenType = keyFieldNames.subList(indexOfOpenPart, keyFieldNames.size());
+ IAType resultant = keyFieldType;
+ for (int i = unnestFlagsForOpenType.size() - 1; i >= 0; i--) {
+ // Construct the type name.
+ StringBuilder recordTypeNameBuilder = new StringBuilder();
+ recordTypeNameBuilder.append(baseRecordType.getTypeName());
+ for (int j = 0; j < i + indexOfOpenPart; j++) {
+ recordTypeNameBuilder.append("_").append(keyFieldNames.get(j));
+ if (keyUnnestFlags.get(j)) {
+ recordTypeNameBuilder.append("_Item");
+ }
+ }
- // Build the type (list or record) that holds the type (list or record) above.
- resultant = nestArrayType(
- new ARecordType(keyFieldNames.get(keyFieldNames.size() - 2),
- new String[] { keyFieldNames.get(keyFieldNames.size() - 1) },
- new IAType[] { AUnionType.createUnknownableType(resultant) }, true),
- keyDepthIndicators.get(indexOfOpenPart));
-
- // Create open part of the nested field.
- for (int i = keyFieldNames.size() - 3; i > (indexOfOpenPart - 1); i--) {
- resultant = nestArrayType(
- new ARecordType(keyFieldNames.get(i), new String[] { keyFieldNames.get(i + 1) },
- new IAType[] { AUnionType.createUnknownableType(resultant) }, true),
- keyDepthIndicators.get(i));
+ // Construct the type itself and account for any array steps.
+ resultant = nestArrayType(resultant, unnestFlagsForOpenType.get(i));
+ resultant =
+ new ARecordType(recordTypeNameBuilder.toString(), new String[] { fieldNamesForOpenType.get(i) },
+ new IAType[] { AUnionType.createUnknownableType(resultant) }, true);
}
// Now update the parent to include this optional field, accounting for intermediate arrays.
- Triple<IAType, String, Integer> gapTriple = this.typeStack.pop();
+ Triple<IAType, String, Boolean> gapTriple = this.typeStack.pop();
ARecordType parentRecord =
(ARecordType) unnestArrayType(TypeComputeUtils.getActualType(gapTriple.first), gapTriple.third);
IAType[] parentFieldTypes = ArrayUtils.addAll(parentRecord.getFieldTypes().clone(),
- AUnionType.createUnknownableType(resultant));
+ ((ARecordType) resultant).getFieldTypes().clone());
resultant = new ARecordType(bridgeNameFoundFromOpenTypeBuild,
- ArrayUtils.addAll(parentRecord.getFieldNames(), resultant.getTypeName()), parentFieldTypes, true);
+ ArrayUtils.addAll(parentRecord.getFieldNames(), gapTriple.second), parentFieldTypes, true);
resultant = keepUnknown(gapTriple.first, nestArrayType(resultant, gapTriple.third));
return resultant;
@@ -168,9 +292,9 @@
private IAType buildNewForFullyClosedType() throws AsterixException {
// The schema is closed all the way to the field itself.
IAType typeIntermediate = TypeComputeUtils.getActualType(endOfOpenTypeBuild);
- int depthOfOpenType = (indexOfOpenPart == 0) ? 0 : keyDepthIndicators.get(indexOfOpenPart - 1);
- int depthOfKeyType = keyDepthIndicators.get(indexOfOpenPart);
- ARecordType lastNestedRecord = (ARecordType) unnestArrayType(typeIntermediate, depthOfOpenType);
+ boolean isOpenTypeWithUnnest = indexOfOpenPart != 0 && keyUnnestFlags.get(indexOfOpenPart - 1);
+ boolean isKeyTypeWithUnnest = keyUnnestFlags.get(indexOfOpenPart);
+ ARecordType lastNestedRecord = (ARecordType) unnestArrayType(typeIntermediate, isOpenTypeWithUnnest);
Map<String, IAType> recordNameTypesMap = createRecordNameTypeMap(lastNestedRecord);
// If an enforced field already exists, verify that the type is correct.
@@ -179,29 +303,34 @@
&& ((AUnionType) enforcedFieldType).isUnknownableType()) {
enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
}
- if (enforcedFieldType != null
- && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(), this.keyFieldType.getTypeTag())) {
- throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Cannot enforce field \""
- + String.join(".", this.keyFieldNames) + "\" to have type " + this.keyFieldType);
- }
- if (enforcedFieldType == null) {
+ if (enforcedFieldType != null) {
+ // choose the type specified in the DDL over the type in the dataset schema if CAST is used
+ if (castDefaultNull) {
+ recordNameTypesMap.put(keyFieldNames.get(keyFieldNames.size() - 1),
+ AUnionType.createNullableType(nestArrayType(keyFieldType, isKeyTypeWithUnnest)));
+ } else if (!ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(), this.keyFieldType.getTypeTag())) {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Cannot enforce field \""
+ + String.join(".", this.keyFieldNames) + "\" to have type " + this.keyFieldType);
+ }
+ } else {
recordNameTypesMap.put(keyFieldNames.get(keyFieldNames.size() - 1),
- AUnionType.createUnknownableType(nestArrayType(keyFieldType, depthOfKeyType)));
+ AUnionType.createUnknownableType(nestArrayType(keyFieldType, isKeyTypeWithUnnest)));
}
// Build the nested record, and account for the wrapping array.
IAType resultant = nestArrayType(
new ARecordType(lastNestedRecord.getTypeName(), recordNameTypesMap.keySet().toArray(new String[0]),
recordNameTypesMap.values().toArray(new IAType[0]), lastNestedRecord.isOpen()),
- depthOfOpenType);
+ isOpenTypeWithUnnest);
return keepUnknown(endOfOpenTypeBuild, resultant);
}
private ARecordType buildRestOfRecord(IAType newTypeToAdd) {
IAType resultant = TypeComputeUtils.getActualType(newTypeToAdd);
while (!typeStack.isEmpty()) {
- Triple<IAType, String, Integer> typeFromStack = typeStack.pop();
+ Triple<IAType, String, Boolean> typeFromStack = typeStack.pop();
IAType typeIntermediate = unnestArrayType(typeFromStack.first, typeFromStack.third);
+ typeIntermediate = TypeComputeUtils.getActualType(typeIntermediate);
ARecordType recordType = (ARecordType) typeIntermediate;
IAType[] fieldTypes = recordType.getFieldTypes().clone();
fieldTypes[recordType.getFieldIndex(typeFromStack.second)] = resultant;
@@ -228,18 +357,21 @@
return updatedRecordType;
}
- private static IAType nestArrayType(IAType originalType, int depthOfArrays) {
- IAType resultant = originalType;
- for (int i = 0; i < depthOfArrays; i++) {
- resultant =
- new AOrderedListType(resultant, (i == depthOfArrays - 1) ? originalType.getTypeName() : null);
+ private static IAType nestArrayType(IAType originalType, boolean isWithinArray) {
+ if (isWithinArray) {
+ String newTypeName = originalType.getTypeName().endsWith("_Item")
+ ? originalType.getTypeName().substring(0, originalType.getTypeName().length() - 5)
+ : originalType.getTypeName();
+ return new AOrderedListType(originalType, newTypeName + "_Array");
+
+ } else {
+ return originalType;
}
- return resultant;
}
- private static IAType unnestArrayType(IAType originalType, int depthOfArrays) {
+ private static IAType unnestArrayType(IAType originalType, boolean isWithinArray) {
IAType resultant = originalType;
- for (int i = 0; i < depthOfArrays; i++) {
+ if (isWithinArray) {
resultant = TypeComputeUtils.extractListItemType(resultant);
if (resultant != null) {
resultant = TypeComputeUtils.getActualType(resultant);
@@ -299,7 +431,8 @@
"Indexing an open field is only supported on the record part");
}
enforcedTypeBuilder.reset(enforcedRecordType, keyFieldNames.get(i),
- Collections.nCopies(keyFieldNames.get(i).size(), 0), keyFieldTypes.get(i));
+ Collections.nCopies(keyFieldNames.get(i).size(), false), keyFieldTypes.get(i),
+ valueIndexDetails.getCastDefaultNull().getOrElse(false));
validateRecord(enforcedRecordType);
enforcedRecordType = enforcedTypeBuilder.build();
}
@@ -319,7 +452,7 @@
"Indexing an open field is only supported on the record part");
}
enforcedTypeBuilder.reset(enforcedRecordType, keyFieldNames.get(i),
- Collections.nCopies(keyFieldNames.get(i).size(), 0), keyFieldTypes.get(i));
+ Collections.nCopies(keyFieldNames.get(i).size(), false), keyFieldTypes.get(i), false);
validateRecord(enforcedRecordType);
enforcedRecordType = enforcedTypeBuilder.build();
}
@@ -342,7 +475,7 @@
List<String> project = projectList.get(i);
enforcedTypeBuilder.reset(enforcedRecordType,
ArrayIndexUtil.getFlattenedKeyFieldNames(unnestList, project),
- ArrayIndexUtil.getArrayDepthIndicator(unnestList, project), typeList.get(i));
+ ArrayIndexUtil.getUnnestFlags(unnestList, project), typeList.get(i), false);
validateRecord(enforcedRecordType);
enforcedRecordType = enforcedTypeBuilder.build();
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index c9fb39c..9b19bb5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -27,6 +27,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
@@ -42,8 +43,9 @@
private byte[] tupleBytes;
private ByteBuffer buf;
- public TupleCopyValueExtractor(ITypeTraits[] typeTraits) {
- this.tupleWriter = new TypeAwareTupleWriter(typeTraits);
+ public TupleCopyValueExtractor(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ this.tupleWriter = new TypeAwareTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
this.tupleReference = tupleWriter.createTupleReference();
}
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index f690018..edb360f 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -44,6 +44,7 @@
import org.apache.asterix.runtime.compression.CompressionManager;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.util.OptionalBoolean;
import org.junit.Assert;
import org.junit.Test;
@@ -73,7 +74,7 @@
Index index = new Index(dvTest, "d1", "i1", IndexType.BTREE,
Collections.singletonList(Collections.singletonList("row_id")),
indicator == null ? null : Collections.singletonList(indicator),
- Collections.singletonList(BuiltinType.AINT64), false, false, false, 0);
+ Collections.singletonList(BuiltinType.AINT64), false, false, false, 0, OptionalBoolean.of(false));
MetadataNode mockMetadataNode = mock(MetadataNode.class);
when(mockMetadataNode.getDatatype(any(), any(DataverseName.class), anyString())).thenReturn(new Datatype(
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/projectedfieldnames/TestFieldNamesEquals.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/projectedfieldnames/TestFieldNamesEquals.java
deleted file mode 100644
index d19a8bb..0000000
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/projectedfieldnames/TestFieldNamesEquals.java
+++ /dev/null
@@ -1,98 +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.
- */
-package org.apache.asterix.metadata.projectedfieldnames;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.asterix.metadata.declared.ExternalDataProjectionInfo;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestFieldNamesEquals {
-
- @Test
- public void testEqual() {
- ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
- ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
- setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
- setFieldNames(p2, new String[] { "a.b.c", "d.e.f", "g.h.i" });
- Assert.assertEquals(p1, p2);
- }
-
- @Test
- public void testReversed() {
- ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
- ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
- setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
- setFieldNames(p2, new String[] { "g.h.i", "d.e.f", "a.b.c" });
- Assert.assertEquals(p1, p2);
- }
-
- @Test
- public void testDifferentPermutations() {
- ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
- ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
- setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
- setFieldNames(p2, new String[] { "d.e.f", "g.h.i", "a.b.c" });
- Assert.assertEquals(p1, p2);
-
- p1 = new ExternalDataProjectionInfo();
- p2 = new ExternalDataProjectionInfo();
- setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
- setFieldNames(p2, new String[] { "g.h.i", "d.e.f", "a.b.c" });
- Assert.assertEquals(p1, p2);
-
- p1 = new ExternalDataProjectionInfo();
- p2 = new ExternalDataProjectionInfo();
- setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
- setFieldNames(p2, new String[] { "g.h.i", "a.b.c", "d.e.f" });
- Assert.assertEquals(p1, p2);
- }
-
- @Test
- public void testDifferentLengths() {
- ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
- ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
- setFieldNames(p1, new String[] { "a.b.c", "d.e.f" });
- setFieldNames(p2, new String[] { "d.e.f", "a.b.c", "g" });
- Assert.assertNotEquals(p1, p2);
- }
-
- @Test
- public void testEqualSubPath() {
- ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
- ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
- setFieldNames(p1, new String[] { "a.b.c", "d.e.f.g" });
- setFieldNames(p2, new String[] { "d.e.f", "a.b.c" });
- Assert.assertNotEquals(p1, p2);
- }
-
- private static void setFieldNames(ExternalDataProjectionInfo p, String[] fieldNames) {
- List<List<String>> fieldNamesList = p.getProjectionInfo();
- for (String fnString : fieldNames) {
- List<String> fnList = new ArrayList<>();
- String[] fn = fnString.split("[.]");
- Collections.addAll(fnList, fn);
- fieldNamesList.add(fnList);
- }
- }
-
-}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
index 91b5d19..b51ebb4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
@@ -19,10 +19,11 @@
package org.apache.asterix.dataflow.data.common;
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
public class MissableTypeComputer implements IMissableTypeComputer {
@@ -33,9 +34,10 @@
}
@Override
- public IAType makeMissableType(Object type) throws AlgebricksException {
+ public IAType makeMissableType(Object type) {
IAType t = (IAType) type;
- return AUnionType.createMissableType(t);
+ return TypeHelper.canBeMissing(t) ? t
+ : t.getTypeTag() == ATypeTag.NULL ? BuiltinType.ANY : AUnionType.createMissableType(t);
}
@Override
@@ -45,8 +47,45 @@
}
@Override
+ public Object makeNullableType(Object type) {
+ IAType t = (IAType) type;
+ return TypeHelper.canBeNull(t) ? t
+ : t.getTypeTag() == ATypeTag.MISSING ? BuiltinType.ANY : AUnionType.createNullableType(t);
+ }
+
+ @Override
+ public boolean canBeNull(Object type) {
+ IAType t = (IAType) type;
+ return TypeHelper.canBeNull(t);
+ }
+
+ @Override
public Object getNonOptionalType(Object type) {
IAType t = (IAType) type;
return TypeComputeUtils.getActualType(t);
}
+
+ @Override
+ public Object getNonMissableType(Object type) {
+ IAType t = (IAType) type;
+ if (t.getTypeTag() == ATypeTag.UNION) {
+ AUnionType ut = ((AUnionType) t);
+ IAType primeType = ut.getActualType();
+ return ut.isNullableType() ? AUnionType.createNullableType(primeType) : primeType;
+ } else {
+ return t;
+ }
+ }
+
+ @Override
+ public Object getNonNullableType(Object type) {
+ IAType t = (IAType) type;
+ if (t.getTypeTag() == ATypeTag.UNION) {
+ AUnionType ut = ((AUnionType) t);
+ IAType primeType = ut.getActualType();
+ return ut.isMissableType() ? AUnionType.createMissableType(primeType) : primeType;
+ } else {
+ return t;
+ }
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
index 3cc9e3d..9d52a85 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/TypeResolverUtil.java
@@ -21,9 +21,12 @@
import java.util.ArrayList;
import java.util.HashSet;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
+import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
+import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
@@ -168,9 +171,38 @@
generalizeRecordFields(leftType, rightType, allPossibleAdditionalFieldNames, fieldNames, fieldTypes);
boolean rightAllMatched =
generalizeRecordFields(rightType, leftType, allPossibleAdditionalFieldNames, fieldNames, fieldTypes);
- return new ARecordType("generalized-record-type", fieldNames.toArray(new String[fieldNames.size()]),
- fieldTypes.toArray(new IAType[fieldTypes.size()]), !(canBeClosed && leftAllMatched && rightAllMatched),
- knowsAdditonalFieldNames ? allPossibleAdditionalFieldNames : null);
+ boolean resultTypeIsOpen = !(canBeClosed && leftAllMatched && rightAllMatched);
+ String[] fieldNamesArray = fieldNames.toArray(new String[0]);
+ IAType[] fieldTypesArray = fieldTypes.toArray(new IAType[0]);
+ ARecordType resultType;
+ if (resultTypeIsOpen && knowsAdditonalFieldNames) {
+ resultType = new ARecordType("generalized-record-type", fieldNamesArray, fieldTypesArray, resultTypeIsOpen,
+ allPossibleAdditionalFieldNames);
+ LinkedHashSet<String> resultFieldOrder = generalizeRecordFieldOrderHint(leftType, rightType);
+ if (resultFieldOrder != null) {
+ resultType.getAnnotations().add(new RecordFieldOrderAnnotation(resultFieldOrder));
+ }
+ } else {
+ resultType = new ARecordType("generalized-record-type", fieldNamesArray, fieldTypesArray, resultTypeIsOpen);
+ }
+ return resultType;
+ }
+
+ private static LinkedHashSet<String> generalizeRecordFieldOrderHint(ARecordType leftType, ARecordType rightType) {
+ IRecordTypeAnnotation leftFieldOrderHint =
+ leftType.findAnnotation(IRecordTypeAnnotation.Kind.RECORD_FIELD_ORDER);
+ if (leftFieldOrderHint == null) {
+ return null;
+ }
+ IRecordTypeAnnotation rightFieldOrderHint =
+ rightType.findAnnotation(IRecordTypeAnnotation.Kind.RECORD_FIELD_ORDER);
+ if (rightFieldOrderHint == null) {
+ return null;
+ }
+ LinkedHashSet<String> resultFieldOrder = new LinkedHashSet<>();
+ resultFieldOrder.addAll(((RecordFieldOrderAnnotation) leftFieldOrderHint).getFieldNames());
+ resultFieldOrder.addAll(((RecordFieldOrderAnnotation) rightFieldOrderHint).getFieldNames());
+ return resultFieldOrder;
}
// Generates closed fields and possible additional fields of a generalized type of two record types.
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java
new file mode 100644
index 0000000..0a5ef58
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.dataflow.data.nontagged;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.dataflow.value.IMissingWriter;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public final class NullWriterFactory implements IMissingWriterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final NullWriterFactory INSTANCE = new NullWriterFactory();
+
+ private NullWriterFactory() {
+ }
+
+ @Override
+ public IMissingWriter createMissingWriter() {
+ return NullWriterFactory::writeNull;
+ }
+
+ private static void writeNull(DataOutput out) throws HyracksDataException {
+ try {
+ out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
index 93ccaa3..4c51b0d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
@@ -38,7 +38,6 @@
import org.apache.asterix.om.util.container.ListObjectPool;
import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.RawBinaryComparatorFactory;
public final class LogicalComplexBinaryComparator implements ILogicalBinaryComparator {
@@ -189,9 +188,7 @@
if (!isEquality) {
return Result.INCOMPARABLE;
}
- return ILogicalBinaryComparator
- .asResult(RawBinaryComparatorFactory.compare(left.getByteArray(), left.getStartOffset(),
- left.getLength(), right.getByteArray(), right.getStartOffset(), right.getLength()));
+ return compareArrays(leftType, left, rightType, right);
}
private Result compareRecords(IAType leftType, TaggedValueReference left, IAType rightType,
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java
index 0ccfa20..449ce13 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/PrintTools.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.dataflow.data.nontagged.printers;
+import java.io.DataOutput;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
@@ -43,8 +44,8 @@
long chrononTime = AInt32SerializerDeserializer.getInt(b, s + 1) * CHRONON_OF_DAY;
try {
- gCalInstance.getExtendStringRepUntilField(chrononTime, 0, ps, GregorianCalendarSystem.Fields.YEAR,
- GregorianCalendarSystem.Fields.DAY, false);
+ gCalInstance.getExtendStringRepUntilField(chrononTime, ps, GregorianCalendarSystem.Fields.YEAR,
+ GregorianCalendarSystem.Fields.DAY);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -54,8 +55,8 @@
long chrononTime = AInt64SerializerDeserializer.getLong(b, s + 1);
try {
- gCalInstance.getExtendStringRepUntilField(chrononTime, 0, ps, GregorianCalendarSystem.Fields.YEAR,
- GregorianCalendarSystem.Fields.MILLISECOND, true);
+ gCalInstance.getExtendStringRepUntilField(chrononTime, ps, GregorianCalendarSystem.Fields.YEAR,
+ GregorianCalendarSystem.Fields.MILLISECOND);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -212,8 +213,8 @@
int time = AInt32SerializerDeserializer.getInt(b, s + 1);
try {
- gCalInstance.getExtendStringRepUntilField(time, 0, ps, GregorianCalendarSystem.Fields.HOUR,
- GregorianCalendarSystem.Fields.MILLISECOND, true);
+ gCalInstance.getExtendStringRepUntilField(time, ps, GregorianCalendarSystem.Fields.HOUR,
+ GregorianCalendarSystem.Fields.MILLISECOND);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -285,17 +286,50 @@
if (c == '"') {
os.write('"');
}
- os.write(c);
- position += sz;
+ if (Character.isHighSurrogate(c)) {
+ position += writeSupplementaryChar(os, b, maxPosition, position, c, sz);
+ continue;
+ }
+ while (sz > 0) {
+ os.write(b[position]);
+ ++position;
+ --sz;
+ }
+ break;
}
os.write('"');
}
- public static void writeUTF8StringAsJSON(byte[] b, int s, int l, OutputStream os) throws IOException {
+ public static void writeUTF8StringRaw(byte[] b, int s, int l, DataOutput os) throws IOException {
int utfLength = UTF8StringUtil.getUTFLength(b, s);
int position = s + UTF8StringUtil.getNumBytesToStoreLength(utfLength); // skip 2 bytes containing string size
int maxPosition = position + utfLength;
+ while (position < maxPosition) {
+ char c = UTF8StringUtil.charAt(b, position);
+ int sz = UTF8StringUtil.charSize(b, position);
+ if (Character.isHighSurrogate(c)) {
+ position += writeSupplementaryChar(os, b, maxPosition, position, c, sz);
+ continue;
+ }
+ while (sz > 0) {
+ os.write(b[position]);
+ ++position;
+ --sz;
+ }
+ }
+ }
+
+ public static void writeUTF8StringAsJSON(byte[] b, int s, int l, OutputStream os) throws IOException {
+ int utfLength = UTF8StringUtil.getUTFLength(b, s);
os.write('"');
+ writeUTF8StringAsJSONUnquoted(b, s, l, utfLength, os);
+ os.write('"');
+ }
+
+ public static void writeUTF8StringAsJSONUnquoted(byte[] b, int s, int l, int utfLength, OutputStream os)
+ throws IOException {
+ int position = s + UTF8StringUtil.getNumBytesToStoreLength(utfLength); // skip 2 bytes containing string size
+ int maxPosition = position + utfLength;
while (position < maxPosition) {
char c = UTF8StringUtil.charAt(b, position);
int sz = UTF8StringUtil.charSize(b, position);
@@ -385,7 +419,6 @@
break;
}
}
- os.write('\"');
}
private static void writeUEscape(OutputStream os, char c) throws IOException {
@@ -415,4 +448,17 @@
return highSurrogateSize + lowSurrogateSize;
}
+ //TODO: some way to dedupe this?
+ private static int writeSupplementaryChar(DataOutput os, byte[] src, int limit, int highSurrogatePos,
+ char highSurrogate, int highSurrogateSize) throws IOException {
+ final int lowSurrogatePos = highSurrogatePos + highSurrogateSize;
+ if (lowSurrogatePos >= limit) {
+ throw new IllegalStateException("malformed utf8 input");
+ }
+ final char lowSurrogate = UTF8StringUtil.charAt(src, lowSurrogatePos);
+ final int lowSurrogateSize = UTF8StringUtil.charSize(src, lowSurrogatePos);
+ os.write(new String(new char[] { highSurrogate, lowSurrogate }).getBytes(StandardCharsets.UTF_8));
+ return highSurrogateSize + lowSurrogateSize;
+ }
+
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java
index 41da3bd..739b653 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java
@@ -19,11 +19,13 @@
package org.apache.asterix.dataflow.data.nontagged.printers.adm;
+import java.io.IOException;
import java.io.PrintStream;
import org.apache.asterix.om.base.AUUID;
import org.apache.hyracks.algebricks.data.IPrinter;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class AUUIDPrinterFactory implements IPrinterFactory {
@@ -32,10 +34,13 @@
public static final AUUIDPrinterFactory INSTANCE = new AUUIDPrinterFactory();
public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
- StringBuilder buf = new StringBuilder(AUUID.UUID_CHARS + 8);
- buf.append("uuid(\"");
- AUUID.appendLiteralOnly(b, s + 1, buf).append("\")");
- ps.print(buf.toString());
+ try {
+ ps.append("uuid(\"");
+ AUUID.appendLiteralOnly(b, s + 1, ps);
+ ps.append("\")");
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
};
@Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AUUIDPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AUUIDPrinterFactory.java
index e4ecfea..0c8ae87 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AUUIDPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/AUUIDPrinterFactory.java
@@ -19,11 +19,13 @@
package org.apache.asterix.dataflow.data.nontagged.printers.csv;
+import java.io.IOException;
import java.io.PrintStream;
import org.apache.asterix.om.base.AUUID;
import org.apache.hyracks.algebricks.data.IPrinter;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class AUUIDPrinterFactory implements IPrinterFactory {
@@ -32,10 +34,13 @@
public static final AUUIDPrinterFactory INSTANCE = new AUUIDPrinterFactory();
public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
- StringBuilder buf = new StringBuilder(AUUID.UUID_CHARS + 2);
- buf.append('"');
- AUUID.appendLiteralOnly(b, s + 1, buf).append('"');
- ps.print(buf.toString());
+ try {
+ ps.append('"');
+ AUUID.appendLiteralOnly(b, s + 1, ps);
+ ps.append('"');
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
};
@Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java
index f1c853f..b364be9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java
@@ -19,11 +19,13 @@
package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
+import java.io.IOException;
import java.io.PrintStream;
import org.apache.asterix.om.base.AUUID;
import org.apache.hyracks.algebricks.data.IPrinter;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class AUUIDPrinterFactory implements IPrinterFactory {
@@ -32,10 +34,13 @@
public static final AUUIDPrinterFactory INSTANCE = new AUUIDPrinterFactory();
public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
- StringBuilder buf = new StringBuilder(AUUID.UUID_CHARS + 2);
- buf.append('"');
- AUUID.appendLiteralOnly(b, s + 1, buf).append('"');
- ps.print(buf.toString());
+ try {
+ ps.append('"');
+ AUUID.appendLiteralOnly(b, s + 1, ps);
+ ps.append('"');
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
};
@Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/AUUIDPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/AUUIDPrinterFactory.java
index 2366a98..388fc913 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/AUUIDPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/AUUIDPrinterFactory.java
@@ -19,11 +19,13 @@
package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
+import java.io.IOException;
import java.io.PrintStream;
import org.apache.asterix.om.base.AUUID;
import org.apache.hyracks.algebricks.data.IPrinter;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class AUUIDPrinterFactory implements IPrinterFactory {
@@ -32,10 +34,13 @@
public static final AUUIDPrinterFactory INSTANCE = new AUUIDPrinterFactory();
public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
- StringBuilder buf = new StringBuilder(AUUID.UUID_CHARS + 2);
- buf.append('"');
- AUUID.appendLiteralOnly(b, s + 1, buf).append('"');
- ps.print(buf.toString());
+ try {
+ ps.append('"');
+ AUUID.appendLiteralOnly(b, s + 1, ps);
+ ps.append('"');
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
};
@Override
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ABinaryPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ABinaryPrinterFactory.java
new file mode 100644
index 0000000..3c18792
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ABinaryPrinterFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
+import org.apache.hyracks.util.bytes.Base64Printer;
+
+public class ABinaryPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ABinaryPrinterFactory INSTANCE = new ABinaryPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ int validLength = ByteArrayPointable.getContentLength(b, s + 1);
+ int start = s + 1 + ByteArrayPointable.getNumberBytesToStoreMeta(validLength);
+ Base64Printer.printBase64Binary(b, start, validLength, ps);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ABooleanPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ABooleanPrinterFactory.java
new file mode 100644
index 0000000..8984cad
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ABooleanPrinterFactory.java
@@ -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.
+ */
+package org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+/**
+ * Boolean value is printed as JSON boolean.
+ */
+public class ABooleanPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ABooleanPrinterFactory INSTANCE = new ABooleanPrinterFactory();
+
+ public static final IPrinter PRINTER =
+ (byte[] b, int s, int l, PrintStream ps) -> ps.print(ABooleanSerializerDeserializer.getBoolean(b, s + 1));
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ACirclePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ACirclePrinterFactory.java
new file mode 100644
index 0000000..323c5d6
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ACirclePrinterFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ACirclePrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ACirclePrinterFactory INSTANCE = new ACirclePrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ int offsetX = ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X);
+ int offsetY = ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y);
+ int offsetR = ACircleSerializerDeserializer.getRadiusOffset();
+ printDouble(b, s + 1 + offsetX, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetR, ps);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADatePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADatePrinterFactory.java
new file mode 100644
index 0000000..590ae4f
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADatePrinterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ADatePrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ADatePrinterFactory INSTANCE = new ADatePrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ ps.print(ADateSerializerDeserializer.getChronon(b, s + 1));
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADateTimePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADateTimePrinterFactory.java
new file mode 100644
index 0000000..4ba055b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADateTimePrinterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ADateTimePrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ADateTimePrinterFactory INSTANCE = new ADateTimePrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ ps.print(ADateTimeSerializerDeserializer.getChronon(b, s + 1));
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADayTimeDurationPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADayTimeDurationPrinterFactory.java
new file mode 100644
index 0000000..b660afa
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADayTimeDurationPrinterFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ADayTimeDurationPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ADayTimeDurationPrinterFactory INSTANCE = new ADayTimeDurationPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, PrintStream ps) {
+ ps.print(ADayTimeDurationSerializerDeserializer.getDayTime(b, s + 1));
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADoublePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADoublePrinterFactory.java
new file mode 100644
index 0000000..af851b1
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADoublePrinterFactory.java
@@ -0,0 +1,41 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ADoublePrinterFactory implements IPrinterFactory {
+
+ public static final ADoublePrinterFactory INSTANCE = new ADoublePrinterFactory();
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ printDouble(b, s + 1, ps);
+ }
+ };
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADurationPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADurationPrinterFactory.java
new file mode 100644
index 0000000..838d431
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ADurationPrinterFactory.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ADurationPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ADurationPrinterFactory INSTANCE = new ADurationPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ int yearMonth = ADurationSerializerDeserializer.getYearMonth(b, s + 1);
+ long dayTime = ADurationSerializerDeserializer.getDayTime(b, s + 1);
+ ps.print(yearMonth);
+ printDelimiter(ps);
+ ps.print(dayTime);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AFloatPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AFloatPrinterFactory.java
new file mode 100644
index 0000000..5121830
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AFloatPrinterFactory.java
@@ -0,0 +1,39 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class AFloatPrinterFactory implements IPrinterFactory {
+
+ public static final AFloatPrinterFactory INSTANCE = new AFloatPrinterFactory();
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ printFloat(b, s + 1, ps);
+ }
+ };
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt16PrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt16PrinterFactory.java
new file mode 100644
index 0000000..34884ec
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt16PrinterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class AInt16PrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AInt16PrinterFactory INSTANCE = new AInt16PrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ ps.print(AInt16SerializerDeserializer.getShort(b, s + 1));
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt32PrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt32PrinterFactory.java
new file mode 100644
index 0000000..84986ba
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt32PrinterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class AInt32PrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AInt32PrinterFactory INSTANCE = new AInt32PrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ ps.print(AInt32SerializerDeserializer.getInt(b, s + 1));
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt64PrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt64PrinterFactory.java
new file mode 100644
index 0000000..2a067fd
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt64PrinterFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+/**
+ * Int64 value is printed as JSON number.
+ */
+public class AInt64PrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AInt64PrinterFactory INSTANCE = new AInt64PrinterFactory();
+
+ public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+ ps.print(AInt64SerializerDeserializer.getLong(b, s + 1));
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt8PrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt8PrinterFactory.java
new file mode 100644
index 0000000..dcfac5e
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AInt8PrinterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class AInt8PrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AInt8PrinterFactory INSTANCE = new AInt8PrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ ps.print(AInt8SerializerDeserializer.getByte(b, s + 1));
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ALinePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ALinePrinterFactory.java
new file mode 100644
index 0000000..4669922
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ALinePrinterFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ALineSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ALinePrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ALinePrinterFactory INSTANCE = new ALinePrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ int offsetX1 = ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X);
+ int offsetY1 = ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y);
+ int offsetX2 = ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X);
+ int offsetY2 = ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y);
+ printDouble(b, s + 1 + offsetX1, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY1, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetX2, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY2, ps);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AMissingPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AMissingPrinterFactory.java
new file mode 100644
index 0000000..3018d6a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AMissingPrinterFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Missing value is printed as standalone type code.
+ */
+public class AMissingPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AMissingPrinterFactory INSTANCE = new AMissingPrinterFactory();
+
+ public static final IPrinter PRINTER = (b, s, l, ps) -> {
+ try {
+ ps.print('"');
+ ATaggedValuePrinter.printTypeTag(b, s, ps);
+ ps.print('"');
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ANullPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ANullPrinterFactory.java
new file mode 100644
index 0000000..9dad25d
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ANullPrinterFactory.java
@@ -0,0 +1,41 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+/**
+ * Null value is printed as JSON null.
+ */
+public class ANullPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ANullPrinterFactory INSTANCE = new ANullPrinterFactory();
+
+ public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> ps.print("null");
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AObjectPrinterFactory.java
new file mode 100644
index 0000000..01c3da0
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AObjectPrinterFactory.java
@@ -0,0 +1,161 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.printer.IPrintVisitor;
+import org.apache.asterix.om.pointables.printer.json.losslessadm.APrintVisitor;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public final class AObjectPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AObjectPrinterFactory INSTANCE = new AObjectPrinterFactory();
+
+ public static boolean printFlatValue(ATypeTag typeTag, byte[] b, int s, int l, PrintStream ps)
+ throws HyracksDataException {
+ switch (typeTag) {
+ case TINYINT:
+ AInt8PrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case SMALLINT:
+ AInt16PrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case INTEGER:
+ AInt32PrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case BIGINT:
+ AInt64PrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case MISSING:
+ AMissingPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case NULL:
+ ANullPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case BOOLEAN:
+ ABooleanPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case FLOAT:
+ AFloatPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case DOUBLE:
+ ADoublePrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case DATE:
+ ADatePrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case TIME:
+ ATimePrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case DATETIME:
+ ADateTimePrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case DURATION:
+ ADurationPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case YEARMONTHDURATION:
+ AYearMonthDurationPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case DAYTIMEDURATION:
+ ADayTimeDurationPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case STRING:
+ AStringPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case BINARY:
+ ABinaryPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case UUID:
+ AUUIDPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case POINT:
+ APointPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case POINT3D:
+ APoint3DPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case CIRCLE:
+ ACirclePrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case LINE:
+ ALinePrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case RECTANGLE:
+ ARectanglePrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case POLYGON:
+ APolygonPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
+ case INTERVAL:
+ case GEOMETRY:
+ // NYI
+ default:
+ return false;
+ }
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ final ARecordVisitablePointable rPointable =
+ new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+ final AListVisitablePointable olPointable =
+ new AListVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE);
+ final AListVisitablePointable ulPointable =
+ new AListVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE);
+ final Pair<PrintStream, ATypeTag> streamTag = new Pair<>(null, null);
+
+ final IPrintVisitor visitor = new APrintVisitor();
+
+ return (byte[] b, int s, int l, PrintStream ps) -> {
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b[s]);
+ if (!printFlatValue(typeTag, b, s, l, ps)) {
+ streamTag.first = ps;
+ streamTag.second = typeTag;
+ switch (typeTag) {
+ case OBJECT:
+ rPointable.set(b, s, l);
+ visitor.visit(rPointable, streamTag);
+ break;
+ case ARRAY:
+ olPointable.set(b, s, l);
+ visitor.visit(olPointable, streamTag);
+ break;
+ case MULTISET:
+ ulPointable.set(b, s, l);
+ visitor.visit(ulPointable, streamTag);
+ break;
+ default:
+ throw new HyracksDataException("No printer for type " + typeTag);
+ }
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AOptionalFieldPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AOptionalFieldPrinterFactory.java
new file mode 100644
index 0000000..25c77070
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AOptionalFieldPrinterFactory.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.formats.nontagged.LosslessADMJSONPrinterFactoryProvider;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class AOptionalFieldPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final AUnionType unionType;
+
+ public AOptionalFieldPrinterFactory(AUnionType unionType) {
+ this.unionType = unionType;
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ return new IPrinter() {
+ private IPrinter missingPrinter;
+ private IPrinter nullPrinter;
+ private IPrinter fieldPrinter;
+
+ @Override
+ public void init() throws HyracksDataException {
+ missingPrinter =
+ (LosslessADMJSONPrinterFactoryProvider.INSTANCE.getPrinterFactory(BuiltinType.AMISSING))
+ .createPrinter();
+ nullPrinter = (LosslessADMJSONPrinterFactoryProvider.INSTANCE.getPrinterFactory(BuiltinType.ANULL))
+ .createPrinter();
+ fieldPrinter =
+ (LosslessADMJSONPrinterFactoryProvider.INSTANCE.getPrinterFactory(unionType.getActualType()))
+ .createPrinter();
+ }
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
+ fieldPrinter.init();
+ if (b[s] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+ missingPrinter.print(b, s, l, ps);
+ } else if (b[s] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+ nullPrinter.print(b, s, l, ps);
+ } else {
+ fieldPrinter.print(b, s, l, ps);
+ }
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AOrderedlistPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AOrderedlistPrinterFactory.java
new file mode 100644
index 0000000..58988f7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AOrderedlistPrinterFactory.java
@@ -0,0 +1,68 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.printer.json.losslessadm.APrintVisitor;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class AOrderedlistPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final AOrderedListType orderedlistType;
+
+ public AOrderedlistPrinterFactory(AOrderedListType orderedlistType) {
+ this.orderedlistType = orderedlistType;
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ final PointableAllocator allocator = new PointableAllocator();
+ final IAType inputType = orderedlistType == null ? DefaultOpenFieldType.getDefaultOpenFieldType(ATypeTag.ARRAY)
+ : orderedlistType;
+ final IVisitablePointable listAccessor = allocator.allocateListValue(inputType);
+ final APrintVisitor printVisitor = new APrintVisitor();
+ final Pair<PrintStream, ATypeTag> arg = new Pair<>(null, null);
+
+ return new IPrinter() {
+ @Override
+ public void init() {
+ arg.second = inputType.getTypeTag();
+ }
+
+ @Override
+ public void print(byte[] b, int start, int l, PrintStream ps) throws HyracksDataException {
+ listAccessor.set(b, start, l);
+ arg.first = ps;
+ listAccessor.accept(printVisitor, arg);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APoint3DPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APoint3DPrinterFactory.java
new file mode 100644
index 0000000..8367bdb
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APoint3DPrinterFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.APoint3DSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class APoint3DPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final APoint3DPrinterFactory INSTANCE = new APoint3DPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ int offsetX = APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.X);
+ int offsetY = APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.Y);
+ int offsetZ = APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.Z);
+ printDouble(b, s + 1 + offsetX, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetZ, ps);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APointPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APointPrinterFactory.java
new file mode 100644
index 0000000..c6fa6f5
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APointPrinterFactory.java
@@ -0,0 +1,50 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class APointPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final APointPrinterFactory INSTANCE = new APointPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ int offsetX = APointSerializerDeserializer.getCoordinateOffset(Coordinate.X);
+ int offsetY = APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y);
+ printDouble(b, s + 1 + offsetX, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY, ps);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APolygonPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APolygonPrinterFactory.java
new file mode 100644
index 0000000..ad18b61
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/APolygonPrinterFactory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class APolygonPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final APolygonPrinterFactory INSTANCE = new APolygonPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ int numOfPoints = AInt16SerializerDeserializer.getShort(b,
+ s + 1 + APolygonSerializerDeserializer.getNumberOfPointsOffset());
+ ps.print(numOfPoints);
+ for (int i = 0; i < numOfPoints; i++) {
+ int offsetX = APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.X);
+ int offsetY = APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.Y);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetX, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY, ps);
+ }
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ARecordPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ARecordPrinterFactory.java
new file mode 100644
index 0000000..9c715a9
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ARecordPrinterFactory.java
@@ -0,0 +1,68 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.printer.json.losslessadm.APrintVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ARecordPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final ARecordType recType;
+
+ public ARecordPrinterFactory(ARecordType recType) {
+ this.recType = recType;
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ final PointableAllocator allocator = new PointableAllocator();
+ final IAType inputType =
+ recType == null ? DefaultOpenFieldType.getDefaultOpenFieldType(ATypeTag.OBJECT) : recType;
+ final IVisitablePointable recAccessor = allocator.allocateRecordValue(inputType);
+ final APrintVisitor printVisitor = new APrintVisitor();
+ final Pair<PrintStream, ATypeTag> arg = new Pair<>(null, null);
+
+ return new IPrinter() {
+ @Override
+ public void init() {
+ arg.second = inputType.getTypeTag();
+ }
+
+ @Override
+ public void print(byte[] b, int start, int l, PrintStream ps) throws HyracksDataException {
+ recAccessor.set(b, start, l);
+ arg.first = ps;
+ recAccessor.accept(printVisitor, arg);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ARectanglePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ARectanglePrinterFactory.java
new file mode 100644
index 0000000..59fa919
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ARectanglePrinterFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ARectanglePrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ARectanglePrinterFactory INSTANCE = new ARectanglePrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ int offsetX1 = ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X);
+ int offsetY1 = ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y);
+ int offsetX2 = ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X);
+ int offsetY2 = ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y);
+ printDouble(b, s + 1 + offsetX1, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY1, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetX2, ps);
+ printDelimiter(ps);
+ printDouble(b, s + 1 + offsetY2, ps);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AStringPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AStringPrinterFactory.java
new file mode 100644
index 0000000..c381336
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AStringPrinterFactory.java
@@ -0,0 +1,59 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+/*
+ * String value printed without type code.
+ * Empty string is printed as empty string.
+ */
+public class AStringPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AStringPrinterFactory INSTANCE = new AStringPrinterFactory();
+
+ public static final IPrinter PRINTER = (b, s, l, ps) -> {
+ try {
+ int utfLength = UTF8StringUtil.getUTFLength(b, s + 1);
+ ps.print('"');
+ if (utfLength > 0) {
+ ATaggedValuePrinter.printDelimiter(ps);
+ PrintTools.writeUTF8StringAsJSONUnquoted(b, s + 1, l - 1, utfLength, ps);
+ }
+ ps.print('"');
+
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ATaggedValuePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ATaggedValuePrinter.java
new file mode 100644
index 0000000..62990b2
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ATaggedValuePrinter.java
@@ -0,0 +1,67 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.bytes.HexPrinter;
+
+public abstract class ATaggedValuePrinter implements IPrinter {
+
+ public static final char DELIMITER = ':';
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
+ try {
+ ps.print("\"");
+ printTypeTag(b, s, ps);
+ printDelimiter(ps);
+ printNonTaggedValue(b, s, l, ps);
+ ps.print("\"");
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ static void printTypeTag(byte[] b, int s, PrintStream ps) throws IOException {
+ HexPrinter.printHexString(b, s, 1, ps);
+ }
+
+ static void printDelimiter(PrintStream ps) {
+ ps.print(DELIMITER);
+ }
+
+ static void printFloat(byte[] b, int s, PrintStream ps) {
+ int bits = AFloatSerializerDeserializer.getIntBits(b, s);
+ ps.print(bits);
+ }
+
+ static void printDouble(byte[] b, int s, PrintStream ps) {
+ long bits = ADoubleSerializerDeserializer.getLongBits(b, s);
+ ps.print(bits);
+ }
+
+ protected abstract void printNonTaggedValue(byte[] b, int s, int l, PrintStream ps) throws IOException;
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ATimePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ATimePrinterFactory.java
new file mode 100644
index 0000000..d0e551f
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/ATimePrinterFactory.java
@@ -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.
+ */
+package org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class ATimePrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final ATimePrinterFactory INSTANCE = new ATimePrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ int chronon = ATimeSerializerDeserializer.getChronon(b, s + 1);
+ ps.print(chronon);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUUIDPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUUIDPrinterFactory.java
new file mode 100644
index 0000000..601d7fd
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUUIDPrinterFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.base.AUUID;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class AUUIDPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AUUIDPrinterFactory INSTANCE = new AUUIDPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) throws IOException {
+ AUUID.appendLiteralOnly(b, s + 1, ps);
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUnionPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUnionPrinterFactory.java
new file mode 100644
index 0000000..1f01326
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUnionPrinterFactory.java
@@ -0,0 +1,74 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+import java.util.List;
+
+import org.apache.asterix.formats.nontagged.LosslessADMJSONPrinterFactoryProvider;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class AUnionPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final AUnionType unionType;
+
+ public AUnionPrinterFactory(AUnionType unionType) {
+ this.unionType = unionType;
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ return new IPrinter() {
+ private IPrinter[] printers;
+ private List<IAType> unionList;
+
+ @Override
+ public void init() throws HyracksDataException {
+ unionList = unionType.getUnionList();
+ printers = new IPrinter[unionType.getUnionList().size()];
+ for (int i = 0; i < printers.length; i++) {
+ printers[i] = (LosslessADMJSONPrinterFactoryProvider.INSTANCE
+ .getPrinterFactory(unionType.getUnionList().get(i))).createPrinter();
+ printers[i].init();
+ }
+ }
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
+ ATypeTag tag = unionList.get(b[s + 1]).getTypeTag();
+ if (tag == ATypeTag.UNION) {
+ printers[b[s + 1]].print(b, s + 1, l, ps);
+ } else {
+ if (tag == ATypeTag.ANY) {
+ printers[b[s + 1]].print(b, s + 2, l, ps);
+ } else {
+ printers[b[s + 1]].print(b, s + 1, l, ps);
+ }
+ }
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUnorderedlistPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUnorderedlistPrinterFactory.java
new file mode 100644
index 0000000..9b5db41
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AUnorderedlistPrinterFactory.java
@@ -0,0 +1,72 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.printer.json.losslessadm.APrintVisitor;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class AUnorderedlistPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final AUnorderedListType unorderedlistType;
+
+ public AUnorderedlistPrinterFactory(AUnorderedListType unorderedlistType) {
+ this.unorderedlistType = unorderedlistType;
+ }
+
+ @Override
+ public IPrinter createPrinter() {
+ PointableAllocator allocator = new PointableAllocator();
+ final IAType inputType = unorderedlistType == null
+ ? DefaultOpenFieldType.getDefaultOpenFieldType(ATypeTag.MULTISET) : unorderedlistType;
+ final IVisitablePointable listAccessor = allocator.allocateListValue(inputType);
+ final APrintVisitor printVisitor = new APrintVisitor();
+ final Pair<PrintStream, ATypeTag> arg = new Pair<>(null, null);
+
+ return new IPrinter() {
+ @Override
+ public void init() {
+ arg.second = inputType.getTypeTag();
+ }
+
+ @Override
+ public void print(byte[] b, int start, int l, PrintStream ps) throws HyracksDataException {
+ try {
+ listAccessor.set(b, start, l);
+ arg.first = ps;
+ listAccessor.accept(printVisitor, arg);
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AYearMonthDurationPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AYearMonthDurationPrinterFactory.java
new file mode 100644
index 0000000..7453a12
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/losslessadm/AYearMonthDurationPrinterFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dataflow.data.nontagged.printers.json.losslessadm;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class AYearMonthDurationPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AYearMonthDurationPrinterFactory INSTANCE = new AYearMonthDurationPrinterFactory();
+
+ public static final IPrinter PRINTER = new ATaggedValuePrinter() {
+ @Override
+ protected void printNonTaggedValue(byte[] b, int s, int l, java.io.PrintStream ps) {
+ ps.print(AYearMonthDurationSerializerDeserializer.getYearMonth(b, s + 1));
+ }
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
index 016a632..c7b8340 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ABinarySerializerDeserializer.java
@@ -25,6 +25,7 @@
import org.apache.asterix.om.base.ABinary;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
import org.apache.hyracks.dataflow.common.data.marshalling.ByteArraySerializerDeserializer;
public class ABinarySerializerDeserializer implements ISerializerDeserializer<ABinary> {
@@ -44,4 +45,12 @@
public void serialize(ABinary binary, DataOutput out) throws HyracksDataException {
ByteArraySerializerDeserializer.serialize(binary.getBytes(), binary.getStart(), binary.getLength(), out);
}
+
+ public static int getContentLength(byte[] bytes, int offset) {
+ return ByteArrayPointable.getContentLength(bytes, offset);
+ }
+
+ public static int getMetaLength(int contentLength) {
+ return ByteArrayPointable.getNumberBytesToStoreMeta(contentLength);
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADoubleSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADoubleSerializerDeserializer.java
index 95a4156..be4dcf8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADoubleSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ADoubleSerializerDeserializer.java
@@ -24,6 +24,7 @@
import org.apache.asterix.om.base.ADouble;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.DoublePointable;
import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
public class ADoubleSerializerDeserializer implements ISerializerDeserializer<ADouble> {
@@ -46,10 +47,10 @@
}
public static double getDouble(byte[] bytes, int offset) {
- return Double.longBitsToDouble(getLongBits(bytes, offset));
+ return DoublePointable.getDouble(bytes, offset);
}
public static long getLongBits(byte[] bytes, int offset) {
- return AInt64SerializerDeserializer.getLong(bytes, offset);
+ return DoublePointable.getLongBits(bytes, offset);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AFloatSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AFloatSerializerDeserializer.java
index 95f5e14..970fb6d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AFloatSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AFloatSerializerDeserializer.java
@@ -49,4 +49,8 @@
public static float getFloat(byte[] bytes, int offset) {
return FloatPointable.getFloat(bytes, offset);
}
+
+ public static int getIntBits(byte[] bytes, int offset) {
+ return FloatPointable.getIntBits(bytes, offset);
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index 57f3449..06c2d96 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -27,6 +27,7 @@
import org.apache.asterix.om.base.ACircle;
import org.apache.asterix.om.base.ADate;
import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.ADayTimeDuration;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.ADuration;
import org.apache.asterix.om.base.AFloat;
@@ -46,7 +47,9 @@
import org.apache.asterix.om.base.ARectangle;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.base.AUUID;
import org.apache.asterix.om.base.AUnorderedList;
+import org.apache.asterix.om.base.AYearMonthDuration;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.IAType;
@@ -123,6 +126,8 @@
return AUnorderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
case GEOMETRY:
return AGeometrySerializerDeserializer.INSTANCE.deserialize(in);
+ case UUID:
+ return AUUIDSerializerDeserializer.INSTANCE.deserialize(in);
default:
throw new NotImplementedException("No serializer/deserializer implemented for type " + typeTag + " .");
}
@@ -180,6 +185,12 @@
case DATETIME:
ADateTimeSerializerDeserializer.INSTANCE.serialize((ADateTime) instance, out);
break;
+ case YEARMONTHDURATION:
+ AYearMonthDurationSerializerDeserializer.INSTANCE.serialize((AYearMonthDuration) instance, out);
+ break;
+ case DAYTIMEDURATION:
+ ADayTimeDurationSerializerDeserializer.INSTANCE.serialize((ADayTimeDuration) instance, out);
+ break;
case DURATION:
ADurationSerializerDeserializer.INSTANCE.serialize((ADuration) instance, out);
break;
@@ -219,6 +230,9 @@
case GEOMETRY:
AGeometrySerializerDeserializer.INSTANCE.serialize((AGeometry) instance, out);
break;
+ case UUID:
+ AUUIDSerializerDeserializer.INSTANCE.serialize((AUUID) instance, out);
+ break;
default:
throw new HyracksDataException(
"No serializer/deserializer implemented for type " + t.getTypeTag() + " .");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
index cab8c22..82af52e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
@@ -65,8 +65,12 @@
public IPrinterFactoryProvider getCleanJSONPrinterFactoryProvider();
+ public IPrinterFactoryProvider getLosslessADMJSONPrinterFactoryProvider();
+
public IMissingWriterFactory getMissingWriterFactory();
+ public IMissingWriterFactory getNullWriterFactory();
+
public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory();
public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/LosslessADMJSONPrinterFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/LosslessADMJSONPrinterFactoryProvider.java
new file mode 100644
index 0000000..e70b6b1
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/LosslessADMJSONPrinterFactoryProvider.java
@@ -0,0 +1,226 @@
+/*
+ * 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.formats.nontagged;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ABinaryPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ABooleanPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ACirclePrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ADatePrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ADateTimePrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ADayTimeDurationPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ADoublePrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ADurationPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AFloatPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AInt16PrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AInt32PrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AInt64PrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AInt8PrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ALinePrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AMissingPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ANullPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AObjectPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AOptionalFieldPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AOrderedlistPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.APoint3DPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.APointPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.APolygonPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ARecordPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ARectanglePrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AStringPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.ATimePrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AUUIDPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AUnionPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AUnorderedlistPrinterFactory;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AYearMonthDurationPrinterFactory;
+import org.apache.asterix.om.base.ABinary;
+import org.apache.asterix.om.base.ACircle;
+import org.apache.asterix.om.base.ADate;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.ADayTimeDuration;
+import org.apache.asterix.om.base.ADuration;
+import org.apache.asterix.om.base.ALine;
+import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.om.base.APoint3D;
+import org.apache.asterix.om.base.APolygon;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.base.AUUID;
+import org.apache.asterix.om.base.AYearMonthDuration;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.algebricks.data.IPrinterFactoryProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Printer for the Lossless-ADM-JSON format.
+ * <p/>
+ * Format specification:
+ * <p/>
+ * <ul>
+ * <li>Record is printed as JSON object
+ * <li>Array or multiset is printed as JSON array
+ * <li>Primitive types are printed as JSON strings in the following format: {@code "TC:encoded_value"}. <br/>
+ * TC is a two-character type code returned by {@link ATypeTag#serialize()} printed in hexadecimal format, <br/>
+ * ':' is a separator character, <br/>
+ * {@code encoded_value} is a type-specific text. <br/>
+ * This is a generic primitive type encoding that works for all primitive types.
+ * Null, Boolean, Int64, and String types types are printed in their special simplified form describe below.
+ * Lossless-ADM-JSON parser is supposed to read both forms for these types: generic and simplified.
+ * </ul>
+ *
+ * <p/>
+ * encoded_value format in the generic form:
+ * <ul>
+ * <li>Int8, Int16, Int32: integer number
+ * <li>Float, Double: integer number corresponding to IEEE 754 floating-point layout</li>
+ * <li>Date: integer number returned by {@link ADate#getChrononTimeInDays()}</li>
+ * <li>Time: integer number returned by {@link ATime#getChrononTime()}</li>
+ * <li>DateTime: integer number returned by {@link ADateTime#getChrononTime()} </li>
+ * <li>YearMonthDuration: integer number returned by {@link AYearMonthDuration#getMonths()}</li>
+ * <li>DayTimeDuration: integer number returned by {@link ADayTimeDuration#getMilliseconds()}</li>
+ * <li>Duration: int1:int2, where int1 is {@link ADuration#getMonths()} and int2 is {@link ADuration#getMilliseconds()}</li>
+ * <li>Binary: {@link ABinary#getBytes()} encoded as Base64</li>
+ * <li>UUID: textual form produced by {@link AUUID#appendLiteralOnly(Appendable)}</li>
+ * <li>Missing: N/A. The two character type code (TC) is sufficient to encode this type. ':' separator character is not printed</li>
+ * <li>Point: int1:int2, where int1 and int2 are integer numbers corresponding to IEEE 754 floating-point layout of
+ * {@link APoint#getX()} and {@link APoint#getY()}</li>
+ * <li>Point3D: int1:int2:int3, where int1 and int2, and int3 are integer numbers corresponding to IEEE 754 floating-point layouts of
+ * {@link APoint3D#getX()} and {@link APoint3D#getY()}, and {@link APoint3D#getZ()}</li>
+ * <li>Circle: int1:int2:int3 - integer numbers corresponding to IEEE 754 floating-point layouts of {@link ACircle#getP()} (getX()/getY()) and {@link ACircle#getRadius()}</li>
+ * <li>Line: int1:int2:int3:int4 - integer numbers corresponding to IEEE 754 floating-point layouts of {@link ALine#getP1()} (getX(), getY()) and {@link ALine#getP2()} (getX(), getY())</li>
+ * <li>Rectangle: int1:int2:int3:int4 - integer numbers corresponding to IEEE 754 floating-point layouts of {@link ARectangle#getP1()} (getX(), getY()) and {@link ARectangle#getP2()} (getX(), getY())</li></li>
+ * <li>Polygon: int0:int1:int2:..intN - integer numbers where int0 is the number of points and each subsequent integer pair corresponds to IEEE 754 floating-point layouts of each point returned by {@link APolygon#getPoints()}</li>
+ * </ul>
+ *
+ * Simplified form for {@code Null}, {@code Boolean}, {@code Int64}, and {@code String} types:
+ * <ul>
+ * <li>Null is printed as JSON null
+ * <li>Boolean value is printed as JSON boolean
+ * <li>Int64 value is printed as JSON number
+ * <li>String value is printed as ":string_text". Empty string is printed as "".
+ * </ul>
+ *
+ * Generic encoded_value form for {@code Null}, {@code Boolean}, {@code Int64}, and {@code String} types
+ * <ul>
+ * <li>Null: N/A. The two character type code (TC) is sufficient to encode this type. ':' separator character is not printed</li>
+ * <li>Boolean: integer number. 0=false, non-0=true
+ * <li>Int64: integer number
+ * <li>String: string text as is
+ * </ul>
+ */
+public class LosslessADMJSONPrinterFactoryProvider implements IPrinterFactoryProvider {
+
+ public static final LosslessADMJSONPrinterFactoryProvider INSTANCE = new LosslessADMJSONPrinterFactoryProvider();
+
+ private LosslessADMJSONPrinterFactoryProvider() {
+ }
+
+ @Override
+ public IPrinterFactory getPrinterFactory(Object typeInfo) throws HyracksDataException {
+ IAType type = (IAType) typeInfo;
+
+ if (type != null) {
+ switch (type.getTypeTag()) {
+ case TINYINT:
+ return AInt8PrinterFactory.INSTANCE;
+ case SMALLINT:
+ return AInt16PrinterFactory.INSTANCE;
+ case INTEGER:
+ return AInt32PrinterFactory.INSTANCE;
+ case BIGINT:
+ return AInt64PrinterFactory.INSTANCE;
+ case MISSING:
+ return AMissingPrinterFactory.INSTANCE;
+ case NULL:
+ return ANullPrinterFactory.INSTANCE;
+ case BOOLEAN:
+ return ABooleanPrinterFactory.INSTANCE;
+ case FLOAT:
+ return AFloatPrinterFactory.INSTANCE;
+ case DOUBLE:
+ return ADoublePrinterFactory.INSTANCE;
+ case TIME:
+ return ATimePrinterFactory.INSTANCE;
+ case DATE:
+ return ADatePrinterFactory.INSTANCE;
+ case DATETIME:
+ return ADateTimePrinterFactory.INSTANCE;
+ case DURATION:
+ return ADurationPrinterFactory.INSTANCE;
+ case YEARMONTHDURATION:
+ return AYearMonthDurationPrinterFactory.INSTANCE;
+ case DAYTIMEDURATION:
+ return ADayTimeDurationPrinterFactory.INSTANCE;
+ case STRING:
+ return AStringPrinterFactory.INSTANCE;
+ case BINARY:
+ return ABinaryPrinterFactory.INSTANCE;
+ case UUID:
+ return AUUIDPrinterFactory.INSTANCE;
+ case POINT:
+ return APointPrinterFactory.INSTANCE;
+ case POINT3D:
+ return APoint3DPrinterFactory.INSTANCE;
+ case CIRCLE:
+ return ACirclePrinterFactory.INSTANCE;
+ case LINE:
+ return ALinePrinterFactory.INSTANCE;
+ case RECTANGLE:
+ return ARectanglePrinterFactory.INSTANCE;
+ case POLYGON:
+ return APolygonPrinterFactory.INSTANCE;
+ case OBJECT:
+ return new ARecordPrinterFactory((ARecordType) type);
+ case ARRAY:
+ return new AOrderedlistPrinterFactory((AOrderedListType) type);
+ case MULTISET:
+ return new AUnorderedlistPrinterFactory((AUnorderedListType) type);
+ case UNION:
+ if (((AUnionType) type).isUnknownableType()) {
+ return new AOptionalFieldPrinterFactory((AUnionType) type);
+ } else {
+ return new AUnionPrinterFactory((AUnionType) type);
+ }
+ case GEOMETRY:
+ case INTERVAL:
+ // NYI
+ case SHORTWITHOUTTYPEINFO:
+ case ANY:
+ case BITARRAY:
+ case ENUM:
+ case SPARSOBJECT:
+ case SYSTEM_NULL:
+ case TYPE:
+ case UINT16:
+ case UINT32:
+ case UINT64:
+ case UINT8:
+ // These types are not intended to be printed to the user.
+ break;
+ }
+ }
+ return AObjectPrinterFactory.INSTANCE;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/NullIntrospector.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/NullIntrospector.java
new file mode 100644
index 0000000..f496efe
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/NullIntrospector.java
@@ -0,0 +1,55 @@
+/*
+ * 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.formats.nontagged;
+
+import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_MISSING_TYPE_TAG;
+import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_NULL_TYPE_TAG;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
+
+import com.fasterxml.jackson.databind.JsonNode;
+
+public class NullIntrospector implements INullIntrospector {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final INullIntrospector INSTANCE = new NullIntrospector();
+
+ private NullIntrospector() {
+ }
+
+ @Override
+ public boolean isNull(byte[] bytes, int offset, int length) {
+ return (bytes[offset] == SERIALIZED_MISSING_TYPE_TAG || bytes[offset] == SERIALIZED_NULL_TYPE_TAG)
+ && length == 1;
+ }
+
+ @Override
+ public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+ return registry.getClassIdentifier(getClass(), serialVersionUID);
+ }
+
+ @SuppressWarnings("squid:S1172") // unused parameter
+ public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+ return INSTANCE;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java
index ab9f4e0..286d1cc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/PredicateEvaluatorFactoryProvider.java
@@ -26,31 +26,24 @@
import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
-/*
-Provides PredicateEvaluator for equi-join cases to properly take care of NULL fields, being compared with each other.
-If any of the join keys, from either side, is NULL, record should not pass equi-join condition.
-*/
+/**
+ * Provides PredicateEvaluator for equi-join cases to disqualify tuples having NULL/MISSING fields
+ * If any of the join keys, from either side, is NULL/MISSING, the tuple will not pass equi-join condition.
+ */
public class PredicateEvaluatorFactoryProvider implements IPredicateEvaluatorFactoryProvider {
private static final long serialVersionUID = 1L;
public static final PredicateEvaluatorFactoryProvider INSTANCE = new PredicateEvaluatorFactoryProvider();
@Override
- public IPredicateEvaluatorFactory getPredicateEvaluatorFactory(final int[] keys0, final int[] keys1) {
+ public IPredicateEvaluatorFactory getPredicateEvaluatorFactory(final int[] keys) {
return new IPredicateEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
public IPredicateEvaluator createPredicateEvaluator() {
- return new IPredicateEvaluator() {
-
- @Override
- public boolean evaluate(IFrameTupleAccessor fta0, int tupId0, IFrameTupleAccessor fta1,
- int tupId1) {
- return noNullOrMissingInKeys(fta0, tupId0, keys0) && noNullOrMissingInKeys(fta1, tupId1, keys1);
- }
- };
+ return (fta, tupId) -> noNullOrMissingInKeys(fta, tupId, keys);
}
};
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java
index 1788445..97fe9c1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/TypeTraitProvider.java
@@ -28,6 +28,7 @@
// WARNING: the byte sizes depend on the serializer!
// currently assuming a serializer that adds a 1-byte type indicator before the data
+ private static final ITypeTraits ZERO_BYTE_TYPE_TRAIT = new FixedLengthTypeTrait(1);
private static final ITypeTraits ONE_BYTE_TYPE_TRAIT = new FixedLengthTypeTrait(1 + 1);
private static final ITypeTraits TWO_BYTE_TYPE_TRAIT = new FixedLengthTypeTrait(2 + 1);
private static final ITypeTraits FOUR_BYTE_TYPE_TRAIT = new FixedLengthTypeTrait(4 + 1);
@@ -48,6 +49,9 @@
return null;
}
switch (type.getTypeTag()) {
+ case NULL:
+ case MISSING:
+ return ZERO_BYTE_TYPE_TRAIT;
case BOOLEAN:
case TINYINT:
return ONE_BYTE_TYPE_TRAIT;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADate.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADate.java
index 2a80f28..63c8f31 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADate.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADate.java
@@ -73,10 +73,10 @@
@Override
public String toString() {
StringBuilder sbder = new StringBuilder();
- sbder.append("\"date\": { ");
+ sbder.append("date: { ");
try {
- GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTimeInDay * CHRONON_OF_DAY, 0,
- sbder, GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY, false);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTimeInDay * CHRONON_OF_DAY, sbder,
+ GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY);
} catch (IOException e) {
throw new RuntimeException(e);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADateTime.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADateTime.java
index df38d1c..b366c37 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADateTime.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ADateTime.java
@@ -115,8 +115,8 @@
private static StringBuilder appendDatetime(StringBuilder sbder, long chrononTime) {
try {
- GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTime, 0, sbder,
- GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTime, sbder,
+ GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND);
return sbder;
} catch (IOException e) {
throw new IllegalStateException(e);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java
index 4fb0b67..93d40fe 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AInterval.java
@@ -100,28 +100,28 @@
sbder.append("date: { ");
GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalStart * ADate.CHRONON_OF_DAY,
- 0, sbder, GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY, false);
+ sbder, GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY);
sbder.append(" }, date: {");
GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd * ADate.CHRONON_OF_DAY,
- 0, sbder, GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY, false);
+ sbder, GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY);
sbder.append(" }");
} else if (typetag == ATypeTag.TIME.serialize()) {
sbder.append("time: { ");
- GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalStart, 0, sbder,
- GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalStart, sbder,
+ GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND);
sbder.append(" }, time: { ");
- GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd, 0, sbder,
- GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd, sbder,
+ GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND);
sbder.append(" }");
} else if (typetag == ATypeTag.DATETIME.serialize()) {
sbder.append("datetime: { ");
- GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalStart, 0, sbder,
- GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalStart, sbder,
+ GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND);
sbder.append(" }, datetime: { ");
- GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd, 0, sbder,
- GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(intervalEnd, sbder,
+ GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND);
sbder.append(" }");
}
} catch (IOException e) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
index 785c62d..9ec73ee 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AOrderedList.java
@@ -79,7 +79,7 @@
return false;
} else {
AOrderedList y = (AOrderedList) o;
- return InMemUtils.cursorEquals(this.getCursor(), y.getCursor());
+ return InMemUtils.deepEqualCursors(this.getCursor(), y.getCursor());
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ATime.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ATime.java
index 5375f7e..226957e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ATime.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ATime.java
@@ -90,8 +90,8 @@
StringBuilder sbder = new StringBuilder();
sbder.append("time: { ");
try {
- GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTime, 0, sbder,
- GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND, true);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(chrononTime, sbder,
+ GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND);
} catch (IOException e) {
throw new RuntimeException(e);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUUID.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUUID.java
index 10c2157..fb44db2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUUID.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUUID.java
@@ -84,29 +84,38 @@
@Override
public String toString() {
- StringBuilder buf = new StringBuilder(UUID_CHARS + 9);
- buf.append("uuid: { ");
- return appendLiteralOnly(buf).append(" }").toString();
+ try {
+ StringBuilder buf = new StringBuilder(UUID_CHARS + 9);
+ buf.append("uuid: { ");
+ appendLiteralOnly(buf);
+ buf.append(" }");
+ return buf.toString();
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
}
- public StringBuilder appendLiteralOnly(StringBuilder buf) {
- return appendLiteralOnly(uuidBytes, 0, buf);
+ public void appendLiteralOnly(Appendable buf) throws IOException {
+ appendLiteralOnly(uuidBytes, 0, buf);
}
- private static StringBuilder digits(byte b[], int offset, int count, StringBuilder result) {
+ private static void digits(byte b[], int offset, int count, Appendable result) throws IOException {
for (int i = 0; i < count; i++) {
result.append(CHARS[(b[offset + i] >> 4) & 0xf]);
result.append(CHARS[b[offset + i] & 0xf]);
}
- return result;
}
- public static StringBuilder appendLiteralOnly(byte[] bytes, int offset, StringBuilder result) {
- digits(bytes, offset, 4, result).append('-');
- digits(bytes, offset + 4, 2, result).append('-');
- digits(bytes, offset + 6, 2, result).append('-');
- digits(bytes, offset + 8, 2, result).append('-');
- return digits(bytes, offset + 10, 6, result);
+ public static void appendLiteralOnly(byte[] bytes, int offset, Appendable result) throws IOException {
+ digits(bytes, offset, 4, result);
+ result.append('-');
+ digits(bytes, offset + 4, 2, result);
+ result.append('-');
+ digits(bytes, offset + 6, 2, result);
+ result.append('-');
+ digits(bytes, offset + 8, 2, result);
+ result.append('-');
+ digits(bytes, offset + 10, 6, result);
}
public void writeTo(DataOutput out) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
index 137171d..bd4586a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AUnorderedList.java
@@ -56,7 +56,7 @@
return false;
} else {
AUnorderedList y = (AUnorderedList) o;
- return InMemUtils.cursorEquals(this.getCursor(), y.getCursor());
+ return InMemUtils.deepEqualCursors(this.getCursor(), y.getCursor());
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/InMemUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/InMemUtils.java
index 65c7e61..af0cb0d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/InMemUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/InMemUtils.java
@@ -20,14 +20,14 @@
public class InMemUtils {
- public final static boolean cursorEquals(IACursor c1, IACursor c2) {
+ public final static boolean deepEqualCursors(IACursor c1, IACursor c2) {
while (c1.next()) {
if (!(c2.next())) {
return false;
}
IAObject thisO = c1.get();
IAObject otherO = c2.get();
- if (!(thisO.equals(otherO))) {
+ if (!(thisO.deepEqual(otherO))) {
return false;
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java
index a4abbd6..c30662a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java
@@ -18,8 +18,6 @@
*/
package org.apache.asterix.om.base.temporal;
-import static org.apache.asterix.om.base.temporal.GregorianCalendarSystem.CHRONON_OF_DAY;
-
import java.io.DataOutput;
import java.io.IOException;
@@ -148,7 +146,7 @@
throw new HyracksDataException(dateErrorMessage);
}
- return GregorianCalendarSystem.getInstance().getChronon(year, month, day, 0, 0, 0, 0, 0);
+ return GregorianCalendarSystem.getInstance().getChronon(year, month, day, 0, 0, 0, 0);
}
/**
@@ -244,7 +242,7 @@
throw new HyracksDataException(dateErrorMessage);
}
- return GregorianCalendarSystem.getInstance().getChronon(year, month, day, 0, 0, 0, 0, 0);
+ return GregorianCalendarSystem.getInstance().getChronon(year, month, day, 0, 0, 0, 0);
}
/**
@@ -340,7 +338,7 @@
throw new HyracksDataException(dateErrorMessage);
}
- return GregorianCalendarSystem.getInstance().getChronon(year, month, day, 0, 0, 0, 0, 0);
+ return GregorianCalendarSystem.getInstance().getChronon(year, month, day, 0, 0, 0, 0);
}
/**
@@ -354,7 +352,7 @@
*/
public static int parseDatePartInDays(String dateString, int start, int length) throws HyracksDataException {
long chronon = parseDatePart(dateString, start, length);
- return convertParsedMillisecondsToDays(chronon);
+ return GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
}
/**
@@ -368,18 +366,6 @@
*/
public static int parseDatePartInDays(char[] dateString, int start, int length) throws HyracksDataException {
long chronon = parseDatePart(dateString, start, length);
- return convertParsedMillisecondsToDays(chronon);
- }
-
- public static int convertParsedMillisecondsToDays(long chronon) throws HyracksDataException {
- if (chronon >= 0) {
- return (int) (chronon / CHRONON_OF_DAY);
- } else {
- if (chronon % CHRONON_OF_DAY != 0) {
- return (int) (chronon / CHRONON_OF_DAY - 1);
- } else {
- return (int) (chronon / CHRONON_OF_DAY);
- }
- }
+ return GregorianCalendarSystem.getInstance().getChrononInDays(chronon);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
index ec387e2..657a66e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParser;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import org.apache.hyracks.util.StringUtil;
public class ADurationParserFactory implements IValueParserFactory {
@@ -83,64 +84,48 @@
All
}
- interface IStringAccessor {
- char getCharAt(int index);
+ public static void parseDuration(CharSequence durationString, final int start, int length, IAObject mutableObject,
+ ADurationParseOption parseOption) throws HyracksDataException {
+ parseDuration(durationString, start, length, mutableObject, parseOption, StringUtil.getCharSequenceAccessor());
}
- public static void parseDuration(final Object durationString, final int start, int length, IAObject mutableObject,
+ public static void parseDuration(char[] durationString, final int start, int length, IAObject mutableObject,
ADurationParseOption parseOption) throws HyracksDataException {
+ parseDuration(durationString, start, length, mutableObject, parseOption, StringUtil.getCharArrayAccessor());
+ }
+
+ public static void parseDuration(byte[] durationString, final int start, int length, IAObject mutableObject,
+ ADurationParseOption parseOption) throws HyracksDataException {
+ parseDuration(durationString, start, length, mutableObject, parseOption,
+ StringUtil.getByteArrayAsCharAccessor());
+ }
+
+ public static <T> void parseDuration(T durationString, final int start, int length, IAObject mutableObject,
+ ADurationParseOption parseOption, StringUtil.ICharAccessor<T> charAccessor) throws HyracksDataException {
int offset = 0;
int value = 0, hour = 0, minute = 0, second = 0, millisecond = 0, year = 0, month = 0, day = 0;
State state = State.NOTHING_READ;
- IStringAccessor charAccessor;
-
- if (durationString instanceof char[]) {
- charAccessor = new IStringAccessor() {
- @Override
- public char getCharAt(int index) {
- return ((char[]) durationString)[start + index];
- }
- };
- } else if (durationString instanceof byte[]) {
- charAccessor = new IStringAccessor() {
-
- @Override
- public char getCharAt(int index) {
- return (char) (((byte[]) durationString)[start + index]);
- }
- };
- } else if (durationString instanceof String) {
- charAccessor = new IStringAccessor() {
-
- @Override
- public char getCharAt(int index) {
- return ((String) durationString).charAt(start + index);
- }
- };
- } else {
- throw new HyracksDataException(durationErrorMessage);
- }
-
short sign = 1;
- if (charAccessor.getCharAt(offset) == '-') {
+ if (charAccessor.charAt(durationString, start + offset) == '-') {
offset++;
sign = -1;
}
- if (charAccessor.getCharAt(offset) != 'P') {
+ if (charAccessor.charAt(durationString, start + offset) != 'P') {
throw new HyracksDataException(durationErrorMessage + ": Missing leading 'P'.");
}
offset++;
for (; offset < length; offset++) {
- if (charAccessor.getCharAt(offset) >= '0' && charAccessor.getCharAt(offset) <= '9') {
+ if (charAccessor.charAt(durationString, start + offset) >= '0'
+ && charAccessor.charAt(durationString, start + offset) <= '9') {
// accumulate the digit fields
- value = value * DECIMAL_UNIT + charAccessor.getCharAt(offset) - '0';
+ value = value * DECIMAL_UNIT + charAccessor.charAt(durationString, start + offset) - '0';
} else {
- switch (charAccessor.getCharAt(offset)) {
+ switch (charAccessor.charAt(durationString, start + offset)) {
case 'Y':
if (state.compareTo(State.YEAR) < 0) {
if (parseOption == ADurationParseOption.DAY_TIME) {
@@ -213,11 +198,11 @@
}
int i = 1;
for (; offset + i < length; i++) {
- if (charAccessor.getCharAt(offset + i) >= '0'
- && charAccessor.getCharAt(offset + i) <= '9') {
+ if (charAccessor.charAt(durationString, start + offset + i) >= '0'
+ && charAccessor.charAt(durationString, start + offset + i) <= '9') {
if (i < 4) {
- millisecond =
- millisecond * DECIMAL_UNIT + (charAccessor.getCharAt(offset + i) - '0');
+ millisecond = millisecond * DECIMAL_UNIT
+ + (charAccessor.charAt(durationString, start + offset + i) - '0');
} else {
throw new HyracksDataException(
durationErrorMessage + ": wrong MILLISECOND field.");
@@ -276,6 +261,8 @@
((AMutableYearMonthDuration) mutableObject).setMonths(totalMonths);
} else if (mutableObject instanceof AMutableDayTimeDuration) {
((AMutableDayTimeDuration) mutableObject).setMilliseconds(totalMilliseconds);
+ } else {
+ throw new IllegalArgumentException();
}
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
index 039e026..ca3552b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
@@ -153,10 +153,10 @@
}
if (length > offset) {
- timezone = parseTimezonePart(timeString, start + offset);
+ parseTimezonePart(timeString, start + offset); // parsed, then ignored
}
- return GregorianCalendarSystem.getInstance().getChronon(hour, min, sec, millis, timezone);
+ return GregorianCalendarSystem.getInstance().getChronon(hour, min, sec, millis);
}
/**
@@ -223,7 +223,6 @@
int offset = 0;
int hour = 0, min = 0, sec = 0, millis = 0;
- int timezone = 0;
boolean isExtendedForm = false;
if (timeString[start + offset + 2] == ':') {
@@ -307,10 +306,10 @@
}
if (length > offset) {
- timezone = parseTimezonePart(timeString, start + offset);
+ parseTimezonePart(timeString, start + offset); // parsed, then ignored
}
- return GregorianCalendarSystem.getInstance().getChronon(hour, min, sec, millis, timezone);
+ return GregorianCalendarSystem.getInstance().getChronon(hour, min, sec, millis);
}
/**
@@ -462,10 +461,10 @@
}
if (length > offset) {
- timezone = parseTimezonePart(timeString, start + offset);
+ parseTimezonePart(timeString, start + offset); // parsed and ignored
}
- return GregorianCalendarSystem.getInstance().getChronon(hour, min, sec, millis, timezone);
+ return GregorianCalendarSystem.getInstance().getChronon(hour, min, sec, millis);
}
/**
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
index 70f9da4..db771f9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
@@ -37,13 +37,14 @@
* <p/>
* - <b>Y</b>: a digit for the year field. At most 4 year format characters are allowed for a valid format string.<br/>
* - <b>M</b>: a digit or character for the month field. At most 3 month format characters are allowed for a valid format string. When three month format characters are used, the shorten month names (like JAN, FEB etc.) are expected in the string to be parsed. Otherwise digits are expected.<br/>
+ * - <b>Q</b>: a digit for the quarter field (1-4). At most 2 format characters are allowed.<br/>
* - <b>D</b>: a digit for the day field. At most 2 day format characters are allowed.<br/>
* - <b>h</b>: a digit for the hour field. At most 2 hour format characters are allowed.<br/>
* - <b>m</b>: a digit for the minute field. At most 2 minute format characters are allowed.<br/>
* - <b>s</b>: a digit for the second field. At most 2 second format characters are allowed.<br/>
* - <b>n</b>: a digit for the millisecond field. At most 3 millisecond format characters are allowed.<br/>
* - <b>a</b>: the AM/PM field. At most 1 am/pm format character is allowed, and it matches with AM and PM case-insensitively. <br/>
- * - <b>z</b>: the timezone field. At most 1 timezone format characters are allowed. The valid timezone string matching with this format character include:<br/>
+ * - <b>z</b>: (parse only) the timezone field. At most 1 timezone format characters are allowed. The valid timezone string matching with this format character include:<br/>
* -- <b>Z</b>: a single upper-case character representing the UTC timezone;<br/>
* -- <b>[UTC|GMT]+xx[:]xx</b>: representing a timezone by providing the actual offset time from the UTC time;<br/>
* -- A string representation of a timezone like PST, Asia/Shanghai. The names of the timezones are following the Zoneinfo database provided by the JDK library. See {@link TimeZone} for more details on this.<br/>
@@ -64,6 +65,7 @@
private static final char MINUTE_CHAR = 'm';
private static final char SECOND_CHAR = 's';
private static final char MILLISECOND_CHAR = 'n';
+ private static final char MILLISECOND_CHAR_ALT = 'S';
private static final char AMPM_CHAR = 'a';
private static final char TIMEZONE_CHAR = 'z';
@@ -76,6 +78,7 @@
private enum DateTimeProcessState {
YEAR,
+ QUARTER,
MONTH,
DAY,
WEEKDAY,
@@ -91,23 +94,37 @@
// For date
private static final char YEAR_CHAR = 'Y';
+ private static final char QUARTER_CHAR = 'Q';
private static final char MONTH_CHAR = 'M';
private static final char DAY_CHAR = 'D';
- private static final char WEEKDAY_CHAR = 'W';
+ private static final char WEEKDAY_CHAR = 'E';
private static final int MAX_YEAR_CHARS = 4;
- private static final int MAX_MONTH_CHARS = 3;
- private static final int MAX_DAY_CHARS = 2;
- private static final int MAX_WEEKDAY_CHAR = 1;
+ private static final int MAX_QUARTER_CHARS = 2;
+ private static final int MAX_MONTH_CHARS = 4;
+ private static final int MAX_DAY_CHARS_PARSE = 2;
+ private static final int MAX_DAY_CHARS_PRINT = 3; // + DDD = Day of Year
+ private static final int MIN_WEEKDAY_CHAR = 3;
+ private static final int MAX_WEEKDAY_CHAR = 4;
private static final byte[][] MONTH_NAMES = new byte[][] { "jan".getBytes(ENCODING), "feb".getBytes(ENCODING),
"mar".getBytes(ENCODING), "apr".getBytes(ENCODING), "may".getBytes(ENCODING), "jun".getBytes(ENCODING),
"jul".getBytes(ENCODING), "aug".getBytes(ENCODING), "sep".getBytes(ENCODING), "oct".getBytes(ENCODING),
"nov".getBytes(ENCODING), "dec".getBytes(ENCODING) };
- private static final byte[][] WEEKDAY_FULL_NAMES = new byte[][] { "monday".getBytes(ENCODING),
- "tuesday".getBytes(ENCODING), "wednesday".getBytes(ENCODING), "thursday".getBytes(ENCODING),
- "friday".getBytes(ENCODING), "saturday".getBytes(ENCODING), "sunday".getBytes(ENCODING) };
+ private static final byte[][] MONTH_FULL_NAMES =
+ new byte[][] { "january".getBytes(ENCODING), "february".getBytes(ENCODING), "march".getBytes(ENCODING),
+ "april".getBytes(ENCODING), "may".getBytes(ENCODING), "june".getBytes(ENCODING),
+ "july".getBytes(ENCODING), "august".getBytes(ENCODING), "september".getBytes(ENCODING),
+ "october".getBytes(ENCODING), "november".getBytes(ENCODING), "december".getBytes(ENCODING) };
+
+ private static final byte[][] WEEKDAY_NAMES = new byte[][] { "sun".getBytes(ENCODING), "mon".getBytes(ENCODING),
+ "tue".getBytes(ENCODING), "wed".getBytes(ENCODING), "thu".getBytes(ENCODING), "fri".getBytes(ENCODING),
+ "sat".getBytes(ENCODING) };
+
+ private static final byte[][] WEEKDAY_FULL_NAMES = new byte[][] { "sunday".getBytes(ENCODING),
+ "monday".getBytes(ENCODING), "tuesday".getBytes(ENCODING), "wednesday".getBytes(ENCODING),
+ "thursday".getBytes(ENCODING), "friday".getBytes(ENCODING), "saturday".getBytes(ENCODING) };
private static final byte[] UTC_BYTEARRAY = "utc".getBytes(ENCODING);
private static final byte[] GMT_BYTEARRAY = "gmt".getBytes(ENCODING);
@@ -175,7 +192,7 @@
}
private int parseFormatField(byte[] format, int formatStart, int formatLength, int formatPointer, char formatChar,
- int maxAllowedFormatCharCopied) {
+ int maxAllowedFormatCharCopied) throws AsterixTemporalTypeParseException {
int formatCharCopies = 0;
@@ -186,7 +203,7 @@
formatCharCopies++;
}
if (formatCharCopies > maxAllowedFormatCharCopied) {
- throw new IllegalStateException(
+ throw new AsterixTemporalTypeParseException(
"The format string for " + formatChar + " is too long: expected no more than "
+ maxAllowedFormatCharCopied + " but got " + formatCharCopies);
}
@@ -200,7 +217,7 @@
DATETIME
}
- private boolean byteArrayEqualToString(byte[] barray, int start, int length, byte[] str) {
+ public static boolean byteArrayEqualToString(byte[] barray, int start, int length, byte[] str) {
if (length != str.length) {
return false;
} else {
@@ -208,7 +225,7 @@
}
}
- private boolean byteArrayBeingWithString(byte[] barray, int start, int length, byte[] str) {
+ public static boolean byteArrayBeingWithString(byte[] barray, int start, int length, byte[] str) {
boolean beginWith = true;
if (length <= str.length) {
for (int i = 0; i < length; i++) {
@@ -223,18 +240,31 @@
return beginWith;
}
- private int monthIDSearch(byte[] barray, int start, int length) {
- for (int i = 0; i < MONTH_NAMES.length; i++) {
- if (byteArrayEqualToString(barray, start, length, MONTH_NAMES[i])) {
+ private int monthIDSearch(byte[] barray, int start, int length, boolean useShortNames) {
+ byte[][] monthNames = useShortNames ? MONTH_NAMES : MONTH_FULL_NAMES;
+ for (int i = 0; i < monthNames.length; i++) {
+ if (byteArrayEqualToString(barray, start, length, monthNames[i])) {
return i;
}
}
return -1;
}
- private int weekdayIDSearch(byte[] barray, int start, int length) {
- for (int i = 0; i < WEEKDAY_FULL_NAMES.length; i++) {
- if (byteArrayBeingWithString(barray, start, length, WEEKDAY_FULL_NAMES[i])) {
+ @Deprecated
+ public static int weekdayIDSearchLax(byte[] barray, int start, int length, boolean useShortNames) {
+ byte[][] weekdayNames = useShortNames ? WEEKDAY_NAMES : WEEKDAY_FULL_NAMES;
+ for (int i = 0; i < weekdayNames.length; i++) {
+ if (byteArrayBeingWithString(barray, start, length, weekdayNames[i])) {
+ return i;
+ }
+ }
+ return -1;
+ }
+
+ public static int weekdayIDSearch(byte[] barray, int start, int length, boolean useShortNames) {
+ byte[][] weekdayNames = useShortNames ? WEEKDAY_NAMES : WEEKDAY_FULL_NAMES;
+ for (int i = 0; i < weekdayNames.length; i++) {
+ if (byteArrayEqualToString(barray, start, length, weekdayNames[i])) {
return i;
}
}
@@ -257,14 +287,14 @@
return -1;
}
- private byte toLower(byte b) {
+ private static byte toLower(byte b) {
if (b >= 'A' && b <= 'Z') {
return (byte) (b - TO_LOWER_OFFSET);
}
return b;
}
- private byte toUpper(byte b) {
+ private static byte toUpper(byte b) {
if (b >= 'a' && b <= 'z') {
return (byte) (b + TO_LOWER_OFFSET);
}
@@ -275,13 +305,14 @@
int formatStart, int formatLength, DateTimeParseMode parseMode, boolean raiseParseDataError)
throws AsterixTemporalTypeParseException {
return parseDateTime(outChronon, null, null, null, data, dataStart, dataLength, format, formatStart,
- formatLength, parseMode, raiseParseDataError, '\0');
+ formatLength, parseMode, raiseParseDataError, '\0', false);
}
public boolean parseDateTime(AMutableInt64 outChronon, Mutable<Boolean> outTimeZoneExists,
AMutableInt32 outTimeZone, Mutable<Character> dateTimeSeparatorOut, byte[] data, int dataStart,
int dataLength, byte[] format, int formatStart, int formatLength, DateTimeParseMode parseMode,
- boolean raiseParseDataError, char altSeparatorChar) throws AsterixTemporalTypeParseException {
+ boolean raiseParseDataError, char altSeparatorChar, boolean adjustChrononByTimezone)
+ throws AsterixTemporalTypeParseException {
int year = 0, month = 0, day = 0, hour = 0, min = 0, sec = 0, ms = 0, timezone = 0;
boolean timezoneExists = false;
@@ -310,6 +341,13 @@
formatPointer += pointerMove;
formatCharCopies += pointerMove;
break;
+ case QUARTER_CHAR:
+ processState = DateTimeProcessState.QUARTER;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, QUARTER_CHAR,
+ MAX_QUARTER_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
case MONTH_CHAR:
processState = DateTimeProcessState.MONTH;
pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MONTH_CHAR,
@@ -319,8 +357,8 @@
break;
case DAY_CHAR:
processState = DateTimeProcessState.DAY;
- pointerMove =
- parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR, MAX_DAY_CHARS);
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR,
+ MAX_DAY_CHARS_PARSE);
formatPointer += pointerMove;
formatCharCopies += pointerMove;
break;
@@ -328,6 +366,11 @@
processState = DateTimeProcessState.WEEKDAY;
pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, WEEKDAY_CHAR,
MAX_WEEKDAY_CHAR);
+ if (pointerMove < MIN_WEEKDAY_CHAR) {
+ throw new AsterixTemporalTypeParseException(
+ String.format("Expected at least %d '%s' characters but got %d", MIN_WEEKDAY_CHAR,
+ WEEKDAY_CHAR, pointerMove));
+ }
formatPointer += pointerMove;
formatCharCopies += pointerMove;
break;
@@ -359,7 +402,13 @@
formatPointer += pointerMove;
formatCharCopies += pointerMove;
break;
-
+ case MILLISECOND_CHAR_ALT:
+ processState = DateTimeProcessState.MILLISECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer,
+ MILLISECOND_CHAR_ALT, MAX_MILLISECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
case AMPM_CHAR:
processState = DateTimeProcessState.AMPM;
pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, AMPM_CHAR,
@@ -403,13 +452,14 @@
default:
throw new AsterixTemporalTypeParseException("Unexpected date format string at "
- + (formatStart + formatPointer) + ": " + format[formatStart + formatPointer]);
+ + (formatStart + formatPointer) + ": " + (char) format[formatStart + formatPointer]);
}
// check whether the process state is valid for the parse mode
switch (processState) {
case YEAR:
+ case QUARTER:
case MONTH:
case DAY:
if (parseMode == DateTimeParseMode.TIME_ONLY) {
@@ -441,7 +491,6 @@
case DAY:
int maxAllowedFormatCharCopies = (processState == DateTimeProcessState.YEAR) ? 4 : 2;
int parsedValue = 0;
-
int processedFieldsCount = 0;
for (int i = 0; i < formatCharCopies; i++) {
if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
@@ -469,17 +518,88 @@
if (negativeYear) {
year *= -1;
}
+ // Allow month and day to be missing if we parsed year
+ if (month == 0) {
+ month = GregorianCalendarSystem.FIELD_MINS[GregorianCalendarSystem.Fields.MONTH.ordinal()];
+ }
+ if (day == 0) {
+ day = GregorianCalendarSystem.FIELD_MINS[GregorianCalendarSystem.Fields.DAY.ordinal()];
+ }
} else {
+ if (parsedValue == 0) {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException(
+ "Incorrect day value at " + (dataStart + dataStringPointer));
+ } else {
+ return false;
+ }
+ }
day = parsedValue;
}
break;
+ case QUARTER:
+ // the month is in the number format
+ parsedValue = 0;
+ int processedQuarterFieldsCount = 0;
+ for (int i = 0; i < formatCharCopies; i++) {
+ if (data[dataStart + dataStringPointer] < '0' || data[dataStart + dataStringPointer] > '9') {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException("Unexpected char for quarter field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ } else {
+ return false;
+ }
+ }
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ if (processedQuarterFieldsCount++ > 2) {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException("Unexpected char for quarter field at "
+ + (dataStart + dataStringPointer) + ": " + data[dataStart + dataStringPointer]);
+ } else {
+ return false;
+ }
+ }
+ }
+ // if there are more than 2 digits for the quarter string
+ while (processedQuarterFieldsCount < 2 && dataStringPointer < dataLength
+ && data[dataStart + dataStringPointer] >= '0'
+ && data[dataStart + dataStringPointer] <= '9') {
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
+ dataStringPointer++;
+ processedQuarterFieldsCount++;
+ }
+ if (parsedValue == 0) {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException(
+ "Incorrect quarter value at " + (dataStart + dataStringPointer));
+ } else {
+ return false;
+ }
+ }
+ month = (parsedValue - 1) * 3 + 1;
+ // Allow day to be missing if we parsed quarter
+ if (day == 0) {
+ day = GregorianCalendarSystem.FIELD_MINS[GregorianCalendarSystem.Fields.DAY.ordinal()];
+ }
+ break;
case MONTH:
- if (formatCharCopies == 3) {
+ if (formatCharCopies >= 3) {
// the month is in the text format
- int monthNameMatch = monthIDSearch(data, dataStart + dataStringPointer, 3);
+ int processedMonthFieldsCount = 0;
+ while (((dataStringPointer + processedMonthFieldsCount < dataLength)) && ((data[dataStart
+ + dataStringPointer + processedMonthFieldsCount] >= 'a'
+ && data[dataStart + dataStringPointer + processedMonthFieldsCount] <= 'z')
+ || (data[dataStart + dataStringPointer + processedMonthFieldsCount] >= 'A'
+ && data[dataStart + dataStringPointer + processedMonthFieldsCount] <= 'Z'))) {
+ processedMonthFieldsCount++;
+ }
+ boolean useShortNames = formatCharCopies == 3;
+ int monthNameMatch = monthIDSearch(data, dataStart + dataStringPointer,
+ processedMonthFieldsCount, useShortNames);
if (monthNameMatch >= 0) {
month = monthNameMatch + 1;
- dataStringPointer += 3;
+ dataStringPointer += processedMonthFieldsCount;
} else {
if (raiseParseDataError) {
throw new AsterixTemporalTypeParseException(
@@ -491,6 +611,8 @@
}
}
} else {
+ // the month is in the number format
+ parsedValue = 0;
int processedMonthFieldsCount = 0;
for (int i = 0; i < formatCharCopies; i++) {
if (data[dataStart + dataStringPointer] < '0'
@@ -503,7 +625,7 @@
return false;
}
}
- month = month * 10 + (data[dataStart + dataStringPointer] - '0');
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
dataStringPointer++;
if (processedMonthFieldsCount++ > 2) {
if (raiseParseDataError) {
@@ -515,26 +637,43 @@
}
}
}
- // if there are more than 2 digits for the day string
+ // if there are more than 2 digits for the month string
while (processedMonthFieldsCount < 2 && dataStringPointer < dataLength
&& data[dataStart + dataStringPointer] >= '0'
&& data[dataStart + dataStringPointer] <= '9') {
- month = month * 10 + (data[dataStart + dataStringPointer] - '0');
+ parsedValue = parsedValue * 10 + (data[dataStart + dataStringPointer] - '0');
dataStringPointer++;
processedMonthFieldsCount++;
}
+ if (parsedValue == 0) {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException(
+ "Incorrect month value at " + (dataStart + dataStringPointer));
+ } else {
+ return false;
+ }
+ }
+ month = parsedValue;
+ }
+ // Allow day to be missing if we parsed month
+ if (day == 0) {
+ day = GregorianCalendarSystem.FIELD_MINS[GregorianCalendarSystem.Fields.DAY.ordinal()];
}
break;
case WEEKDAY:
int processedWeekdayFieldsCount = 0;
- while ((data[dataStart + dataStringPointer + processedWeekdayFieldsCount] >= 'a'
+ while ((dataStringPointer + processedWeekdayFieldsCount < dataLength) && ((data[dataStart
+ + dataStringPointer + processedWeekdayFieldsCount] >= 'a'
&& data[dataStart + dataStringPointer + processedWeekdayFieldsCount] <= 'z')
|| (data[dataStart + dataStringPointer + processedWeekdayFieldsCount] >= 'A'
- && data[dataStart + dataStringPointer + processedWeekdayFieldsCount] <= 'Z')) {
+ && data[dataStart + dataStringPointer + processedWeekdayFieldsCount] <= 'Z'))) {
processedWeekdayFieldsCount++;
}
// match the weekday name
- if (weekdayIDSearch(data, dataStart + dataStringPointer, processedWeekdayFieldsCount) < 0) {
+ boolean useShortNames = formatCharCopies == 3;
+ int weekdayNameMatch = weekdayIDSearch(data, dataStart + dataStringPointer,
+ processedWeekdayFieldsCount, useShortNames);
+ if (weekdayNameMatch < 0) {
if (raiseParseDataError) {
throw new AsterixTemporalTypeParseException("Unexpected string for day-of-week: "
+ new String(data, dataStart + dataStringPointer,
@@ -799,8 +938,40 @@
}
}
- long chronon = parseMode == DateTimeParseMode.TIME_ONLY ? CAL.getChronon(hour, min, sec, ms, timezone)
- : CAL.getChronon(year, month, day, hour, min, sec, ms, timezone);
+ long chronon;
+ if (parseMode == DateTimeParseMode.TIME_ONLY) {
+ int minYear = GregorianCalendarSystem.FIELD_MINS[GregorianCalendarSystem.Fields.YEAR.ordinal()];
+ int minMonth = GregorianCalendarSystem.FIELD_MINS[GregorianCalendarSystem.Fields.MONTH.ordinal()];
+ int minDay = GregorianCalendarSystem.FIELD_MINS[GregorianCalendarSystem.Fields.DAY.ordinal()];
+ if (!CAL.validate(minYear, minMonth, minDay, hour, min, sec, ms)) {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException("Invalid time value");
+ } else {
+ return false;
+ }
+ }
+ chronon = CAL.getChronon(hour, min, sec, ms);
+ } else {
+ if (!CAL.validate(year, month, day, hour, min, sec, ms)) {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException("Invalid date/time value");
+ } else {
+ return false;
+ }
+ }
+ chronon = CAL.getChronon(year, month, day, hour, min, sec, ms);
+ }
+
+ if (timezoneExists && adjustChrononByTimezone) {
+ if (!CAL.validateTimeZone(timezone)) {
+ if (raiseParseDataError) {
+ throw new AsterixTemporalTypeParseException("Invalid time zone");
+ } else {
+ return false;
+ }
+ }
+ chronon += timezone;
+ }
outChronon.setValue(chronon);
if (dateTimeSeparatorOut != null) {
@@ -815,11 +986,13 @@
return true;
}
- public void printDateTime(long chronon, int timezone, byte[] format, int formatStart, int formatLength,
- Appendable appender, DateTimeParseMode parseMode) throws HyracksDataException {
+ public void printDateTime(long chronon, byte[] format, int formatStart, int formatLength, Appendable appender,
+ DateTimeParseMode parseMode) throws HyracksDataException {
int year = CAL.getYear(chronon);
int month = CAL.getMonthOfYear(chronon, year);
int day = CAL.getDayOfMonthYear(chronon, year, month);
+ int dayOfYear = CAL.getDayOfYear(chronon, year);
+ int dayOfWeek = CAL.getDayOfWeek(chronon);
int hour = CAL.getHourOfDay(chronon);
int min = CAL.getMinOfHour(chronon);
int sec = CAL.getSecOfMin(chronon);
@@ -858,6 +1031,13 @@
formatPointer += pointerMove;
formatCharCopies += pointerMove;
break;
+ case QUARTER_CHAR:
+ processState = DateTimeProcessState.QUARTER;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, QUARTER_CHAR,
+ MAX_QUARTER_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
case MONTH_CHAR:
processState = DateTimeProcessState.MONTH;
pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, MONTH_CHAR,
@@ -867,8 +1047,20 @@
break;
case DAY_CHAR:
processState = DateTimeProcessState.DAY;
- pointerMove =
- parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR, MAX_DAY_CHARS);
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, DAY_CHAR,
+ MAX_DAY_CHARS_PRINT);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
+ case WEEKDAY_CHAR:
+ processState = DateTimeProcessState.WEEKDAY;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, WEEKDAY_CHAR,
+ MAX_WEEKDAY_CHAR);
+ if (pointerMove < MIN_WEEKDAY_CHAR) {
+ throw new AsterixTemporalTypeParseException(
+ String.format("Expected at least %d '%s' characters but got %d", MIN_WEEKDAY_CHAR,
+ WEEKDAY_CHAR, pointerMove));
+ }
formatPointer += pointerMove;
formatCharCopies += pointerMove;
break;
@@ -900,7 +1092,13 @@
formatPointer += pointerMove;
formatCharCopies += pointerMove;
break;
-
+ case MILLISECOND_CHAR_ALT:
+ processState = DateTimeProcessState.MILLISECOND;
+ pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer,
+ MILLISECOND_CHAR_ALT, MAX_MILLISECOND_CHARS);
+ formatPointer += pointerMove;
+ formatCharCopies += pointerMove;
+ break;
case AMPM_CHAR:
processState = DateTimeProcessState.AMPM;
pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, AMPM_CHAR,
@@ -909,13 +1107,6 @@
formatCharCopies += pointerMove;
break;
- case TIMEZONE_CHAR:
- processState = DateTimeProcessState.TIMEZONE;
- pointerMove = parseFormatField(format, formatStart, formatLength, formatPointer, TIMEZONE_CHAR,
- MAX_TIMEZONE_CHARS);
- formatPointer += pointerMove;
- formatCharCopies += pointerMove;
- break;
case ' ':
case HYPHEN_CHAR:
case COLON_CHAR:
@@ -937,15 +1128,17 @@
default:
throw new HyracksDataException("Unexpected format string at " + (formatStart + formatPointer) + ": "
- + format[formatStart + formatPointer]);
+ + (char) (format[formatStart + formatPointer]));
}
// check whether the process state is valid for the parse mode
switch (processState) {
case YEAR:
+ case QUARTER:
case MONTH:
case DAY:
+ case WEEKDAY:
if (parseMode == DateTimeParseMode.TIME_ONLY) {
throw new HyracksDataException("Unexpected date format string when parsing a time value");
}
@@ -955,7 +1148,6 @@
case SECOND:
case MILLISECOND:
case AMPM:
- case TIMEZONE:
if (parseMode == DateTimeParseMode.DATE_ONLY) {
throw new HyracksDataException("Unexpected time format string when parsing a date value");
}
@@ -971,21 +1163,25 @@
appender.append('-');
year *= -1;
}
+ case QUARTER:
case MONTH:
- if (processState == DateTimeProcessState.MONTH && formatCharCopies == 3) {
- for (byte b : MONTH_NAMES[month - 1]) {
+ if (processState == DateTimeProcessState.MONTH && formatCharCopies >= 3) {
+ byte[][] monthNames = formatCharCopies > 3 ? MONTH_FULL_NAMES : MONTH_NAMES;
+ for (byte b : monthNames[month - 1]) {
appender.append((char) toUpper(b));
}
break;
}
case DAY:
- int val = 0;
+ int val;
if (processState == DateTimeProcessState.YEAR) {
val = year;
+ } else if (processState == DateTimeProcessState.QUARTER) {
+ val = ((month - 1) / 3) + 1;
} else if (processState == DateTimeProcessState.MONTH) {
val = month;
} else {
- val = day;
+ val = formatCharCopies == 3 ? dayOfYear : day;
}
String strVal = String.valueOf(val);
int valFieldCount = strVal.length();
@@ -994,6 +1190,14 @@
}
appender.append(strVal);
break;
+ case WEEKDAY:
+ byte[][] weekdayNames = formatCharCopies == 3 ? WEEKDAY_NAMES : WEEKDAY_FULL_NAMES;
+ byte[] weekday = weekdayNames[dayOfWeek];
+ for (int i = 0; i < weekday.length; i++) {
+ byte b = weekday[i];
+ appender.append((char) (i == 0 ? toUpper(b) : b));
+ }
+ break;
case HOUR:
case MINUTE:
case SECOND:
@@ -1040,6 +1244,7 @@
appender.append(strMS);
}
break;
+ /* TODO: enable when we support "datetime with timezone" datatype
case TIMEZONE:
if (timezone == 0) {
appender.append('Z');
@@ -1061,6 +1266,7 @@
}
appender.append(String.valueOf(timezoneField));
break;
+ */
case AMPM:
if (usePM) {
appender.append("PM");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DurationArithmeticOperations.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DurationArithmeticOperations.java
index 26c3fb3..baa86b5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DurationArithmeticOperations.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DurationArithmeticOperations.java
@@ -94,7 +94,7 @@
}
}
- return GREG_CAL.getChronon(year, month, day, hour, min, sec, ms, 0) + dayTimeDuration;
+ return GREG_CAL.getChronon(year, month, day, hour, min, sec, ms) + dayTimeDuration;
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java
index deb15c8..2f838be 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/GregorianCalendarSystem.java
@@ -121,10 +121,8 @@
/**
* Check whether the given date time value is a valid date time following the gregorian calendar system.
- *
- * @param fields
- * @return
*/
+ @Override
public boolean validate(int year, int month, int day, int hour, int min, int sec, int millis) {
// Check whether each field is within the value domain
if (year < FIELD_MINS[0] || year > FIELD_MAXS[0]) {
@@ -187,7 +185,7 @@
}
/**
- * Validate the given chronon time and time zone.
+ * Get the UTC chronon time of the given date time.
*
* @param year
* @param month
@@ -196,30 +194,12 @@
* @param min
* @param sec
* @param millis
- * @param timezone
- * @return
*/
- public boolean validate(int year, int month, int day, int hour, int min, int sec, int millis, int timezone) {
- return validate(year, month, day, hour, min, sec, millis) && validateTimeZone(timezone);
- }
-
- /**
- * Get the UTC chronon time of the given date time and time zone.
- *
- * @param year
- * @param month
- * @param day
- * @param hour
- * @param min
- * @param sec
- * @param millis
- * @param timezone
- * @return
- */
- public long getChronon(int year, int month, int day, int hour, int min, int sec, int millis, int timezone) {
+ @Override
+ public long getChronon(int year, int month, int day, int hour, int min, int sec, int millis) {
// Added milliseconds for all fields but month and day
long chrononTime = chrononizeBeginningOfYear(year) + hour * CHRONON_OF_HOUR + min * CHRONON_OF_MINUTE
- + sec * CHRONON_OF_SECOND + millis + timezone;
+ + sec * CHRONON_OF_SECOND + millis;
// Added milliseconds for days of the month.
chrononTime += (day - 1 + DAYS_SINCE_MONTH_BEGIN_ORDI[month - 1]) * CHRONON_OF_DAY;
@@ -233,19 +213,17 @@
}
/**
- * Get the chronon time (number of milliseconds) of the given time and time zone.
+ * Get the UTC chronon time (number of milliseconds) of the given time and time zone.
*
* @param hour
* @param min
* @param sec
* @param millis
- * @param timezone
* @return
*/
- public int getChronon(int hour, int min, int sec, int millis, int timezone) {
+ public int getChronon(int hour, int min, int sec, int millis) {
// Added milliseconds for all fields but month and day
- long chrononTime =
- hour * CHRONON_OF_HOUR + min * CHRONON_OF_MINUTE + sec * CHRONON_OF_SECOND + millis + timezone;
+ long chrononTime = hour * CHRONON_OF_HOUR + min * CHRONON_OF_MINUTE + sec * CHRONON_OF_SECOND + millis;
return (int) chrononTime;
}
@@ -254,50 +232,50 @@
}
public int getChrononInDays(long chronon) {
- if (chronon >= 0) {
- return (int) (chronon / CHRONON_OF_DAY);
- } else {
- if (chronon % CHRONON_OF_DAY != 0) {
- return (int) (chronon / CHRONON_OF_DAY - 1);
- } else {
- return (int) (chronon / CHRONON_OF_DAY);
- }
+ int dateChrononInDays = (int) (chronon / GregorianCalendarSystem.CHRONON_OF_DAY);
+ if (dateChrononInDays < 0 && (chronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0)) {
+ dateChrononInDays -= 1;
}
+ return dateChrononInDays;
+ }
+
+ public int getTimeChronon(long chronon) {
+ int timeChronon = (int) (chronon % GregorianCalendarSystem.CHRONON_OF_DAY);
+ if (timeChronon < 0) {
+ timeChronon += GregorianCalendarSystem.CHRONON_OF_DAY;
+ }
+ return timeChronon;
}
/**
- * Get the extended string representation of the given UTC chronon time under the given time zone. Only fields
- * before
- * the given field index will be returned.
+ * Get the extended string representation of the given chronon time.
+ * Only fields before the given field index will be returned.
* <p/>
* The extended string representation is like:<br/>
- * [-]YYYY-MM-DDThh:mm:ss.xxx[Z|[+|-]hh:mm]
+ * [-]YYYY-MM-DDThh:mm:ss.xxx
*
* @param chrononTime
- * @param timezone
* @param sbder
* @param untilField
*/
- public void getExtendStringRepUntilField(long chrononTime, int timezone, Appendable sbder, Fields startField,
- Fields untilField, boolean withTimezone) throws IOException {
- getExtendStringRepUntilField(chrononTime, timezone, sbder, startField, untilField, withTimezone, 'T');
+ public void getExtendStringRepUntilField(long chrononTime, Appendable sbder, Fields startField, Fields untilField)
+ throws IOException {
+ getExtendStringRepUntilField(chrononTime, sbder, startField, untilField, 'T');
}
/**
- * Get the extended string representation of the given UTC chronon time under the given time zone. Only fields
- * before
- * the given field index will be returned.
+ * Get the extended string representation of the given chronon time.
+ * Only fields before the given field index will be returned.
* <p/>
* The extended string representation is like:<br/>
- * [-]YYYY-MM-DDThh:mm:ss.xxx[Z|[+|-]hh:mm]
+ * [-]YYYY-MM-DDThh:mm:ss.xxx
*
* @param chrononTime
- * @param timezone
* @param sbder
* @param untilField
*/
- public void getExtendStringRepUntilField(long chrononTime, int timezone, Appendable sbder, Fields startField,
- Fields untilField, boolean withTimezone, char dateTimeSeparator) throws IOException {
+ public void getExtendStringRepUntilField(long chrononTime, Appendable sbder, Fields startField, Fields untilField,
+ char dateTimeSeparator) throws IOException {
int year = getYear(chrononTime);
int month = getMonthOfYear(chrononTime, year);
@@ -357,28 +335,15 @@
sbder.append(String.format("%03d", getMillisOfSec(chrononTime)));
break;
}
-
- if (withTimezone) {
- if (timezone == 0) {
- sbder.append('Z');
- } else {
- int tzMin = (int) ((timezone % CHRONON_OF_HOUR) / CHRONON_OF_MINUTE);
- int tzHr = (int) (timezone / CHRONON_OF_HOUR);
- sbder.append(tzHr >= 0 ? '-' : '+').append(String.format("%02d", tzHr < 0 ? -tzHr : tzHr)).append(':')
- .append(String.format("%02d", tzMin < 0 ? -tzMin : tzMin));
- }
- }
}
/**
- * Get the basic string representation of a chronon time with the given time zone.
- *
+ * Get the basic string representation of a chronon time.
* @param chrononTime
- * @param timezone
* @param sbder
*/
- public void getBasicStringRepUntilField(long chrononTime, int timezone, Appendable sbder, Fields startField,
- Fields untilField, boolean withTimezone) throws IOException {
+ public void getBasicStringRepUntilField(long chrononTime, Appendable sbder, Fields startField, Fields untilField)
+ throws IOException {
int year = getYear(chrononTime);
int month = getMonthOfYear(chrononTime, year);
@@ -415,19 +380,22 @@
sbder.append(String.format("%03d", getMillisOfSec(chrononTime)));
break;
}
+ }
- if (withTimezone) {
- if (timezone == 0) {
- sbder.append('Z');
- } else {
- int tzMin = (int) (timezone % CHRONON_OF_HOUR / CHRONON_OF_MINUTE);
- if (tzMin < 0) {
- tzMin = (short) (-1 * tzMin);
- }
- int tzHr = (int) (timezone / CHRONON_OF_HOUR);
- sbder.append((tzHr >= 0 ? '-' : '+')).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr)))
- .append(String.format("%02d", tzMin));
+ /**
+ * Get the string representation of a timezone: Z|[+|-]hh:mm
+ */
+ public void getTimezoneStringRep(int timezone, Appendable sbder) throws IOException {
+ if (timezone == 0) {
+ sbder.append('Z');
+ } else {
+ int tzMin = (int) (timezone % CHRONON_OF_HOUR / CHRONON_OF_MINUTE);
+ if (tzMin < 0) {
+ tzMin = (short) (-1 * tzMin);
}
+ int tzHr = (int) (timezone / CHRONON_OF_HOUR);
+ sbder.append((tzHr >= 0 ? '-' : '+')).append(String.format("%02d", (tzHr < 0 ? -tzHr : tzHr))).append(':')
+ .append(String.format("%02d", tzMin));
}
}
@@ -441,7 +409,6 @@
* @param months
* @param sbder
*/
-
public void getDurationExtendStringRepWithTimezoneUntilField(long milliseconds, int months, StringBuilder sbder) {
boolean positive = true;
@@ -687,6 +654,14 @@
}
/**
+ * Get the quarter of the year for the given chronon time and the year.
+ */
+ public int getQuarterOfYear(long millis, int year) {
+ int month = getMonthOfYear(millis, year);
+ return (month - 1) / 3 + 1;
+ }
+
+ /**
* Get the hour of the day for the given chronon time.
*
* @param millis
@@ -763,7 +738,7 @@
}
/**
- * Get the day of week for the given chronon time. 0 (Sunday) to 7 (Saturday)
+ * Get the day of week for the given chronon time. 0 (Sunday) to 6 (Saturday)
*
* @param millis
* @return
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ICalendarSystem.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ICalendarSystem.java
index 5fd785d..7a70efd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ICalendarSystem.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ICalendarSystem.java
@@ -30,10 +30,9 @@
* @param min
* @param sec
* @param millis
- * @param timezone
* @return
*/
- public boolean validate(int year, int month, int day, int hour, int min, int sec, int millis, int timezone);
+ public boolean validate(int year, int month, int day, int hour, int min, int sec, int millis);
/**
* get the chronon time for the given time stamp in the calendar system.
@@ -45,10 +44,9 @@
* @param min
* @param sec
* @param millis
- * @param timezone
* @return
*/
- public long getChronon(int year, int month, int day, int hour, int min, int sec, int millis, int timezone);
+ public long getChronon(int year, int month, int day, int hour, int min, int sec, int millis);
/**
* get the chronon time for the given time in the calendar system
@@ -57,9 +55,8 @@
* @param min
* @param sec
* @param millis
- * @param timezone
* @return
*/
- public int getChronon(int hour, int min, int sec, int millis, int timezone);
+ public int getChronon(int hour, int min, int sec, int millis);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
index 337e966..8b5da27 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
@@ -22,6 +22,7 @@
import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.IAObject;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
public class AsterixConstantValue implements IAlgebricksConstantValue {
@@ -63,15 +64,27 @@
@Override
public boolean equals(Object o) {
- if (!(o instanceof AsterixConstantValue)) {
+ if (o instanceof AsterixConstantValue) {
+ AsterixConstantValue v2 = (AsterixConstantValue) o;
+ return object.deepEqual(v2.getObject());
+ } else if (o instanceof IAlgebricksConstantValue) {
+ return o.equals(this);
+ } else {
return false;
}
- AsterixConstantValue v2 = (AsterixConstantValue) o;
- return object.deepEqual(v2.getObject());
}
@Override
public int hashCode() {
- return object.hash();
+ switch (object.getType().getTypeTag()) {
+ case MISSING:
+ return ConstantExpression.MISSING.hashCode();
+ case NULL:
+ return ConstantExpression.NULL.hashCode();
+ case BOOLEAN:
+ return (isTrue() ? ConstantExpression.TRUE : ConstantExpression.FALSE).hashCode();
+ default:
+ return object.hash();
+ }
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index a2e235e..0c9f0dd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -102,6 +102,7 @@
import org.apache.asterix.om.typecomputer.impl.NotUnknownTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NullIfTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NullableDoubleTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NullableTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericAddSubMulDivTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericBinaryToDoubleTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericDivideTypeComputer;
@@ -134,6 +135,7 @@
import org.apache.asterix.om.typecomputer.impl.TreatAsTypeComputer;
import org.apache.asterix.om.typecomputer.impl.UnaryBinaryInt64TypeComputer;
import org.apache.asterix.om.typecomputer.impl.UniformInputTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.UnionMbrAggTypeComputer;
import org.apache.asterix.om.typecomputer.impl.UnorderedListConstructorTypeComputer;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -360,6 +362,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "round-half-to-even", 1);
public static final FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN2 =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "round-half-to-even", 2);
+ public static final FunctionIdentifier NUMERIC_ROUND_HALF_UP2 =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "round-half-up", 2);
public static final FunctionIdentifier NUMERIC_TRUNC =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "trunc", 2);
@@ -496,6 +500,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "repeat", 2);
public static final FunctionIdentifier STRING_SPLIT =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "split", 2);
+ public static final FunctionIdentifier STRING_PARSE_JSON =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "parse-json", 1);
public static final FunctionIdentifier DATASET =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", FunctionIdentifier.VARARGS); // 1 or 2
@@ -605,6 +611,14 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-kurtosis", 1);
public static final FunctionIdentifier NULL_WRITER =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-null-writer", 1);
+ public static final FunctionIdentifier UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-union_mbr", 1);
+ public static final FunctionIdentifier LOCAL_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-union_mbr", 1);
+ public static final FunctionIdentifier INTERMEDIATE_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-union_mbr", 1);
+ public static final FunctionIdentifier GLOBAL_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-union_mbr", 1);
public static final FunctionIdentifier SCALAR_ARRAYAGG =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "arrayagg", 1);
@@ -632,6 +646,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "skewness", 1);
public static final FunctionIdentifier SCALAR_KURTOSIS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "kurtosis", 1);
+ public static final FunctionIdentifier SCALAR_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "union_mbr", 1);
// serializable aggregate functions
public static final FunctionIdentifier SERIAL_AVG =
@@ -834,6 +850,14 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sql-kurtosis", 1);
public static final FunctionIdentifier LOCAL_SQL_KURTOSIS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sql-kurtosis", 1);
+ public static final FunctionIdentifier SQL_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sql-union_mbr", 1);
+ public static final FunctionIdentifier LOCAL_SQL_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sql-union_mbr", 1);
+ public static final FunctionIdentifier INTERMEDIATE_SQL_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-sql-union_mbr", 1);
+ public static final FunctionIdentifier GLOBAL_SQL_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sql-union_mbr", 1);
public static final FunctionIdentifier SCALAR_SQL_AVG =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-avg", 1);
@@ -857,6 +881,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-skewness", 1);
public static final FunctionIdentifier SCALAR_SQL_KURTOSIS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-kurtosis", 1);
+ public static final FunctionIdentifier SCALAR_SQL_UNION_MBR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-union_mbr", 1);
// serializable sql aggregate functions
public static final FunctionIdentifier SERIAL_SQL_AVG =
@@ -1021,6 +1047,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent-rank-impl", FunctionIdentifier.VARARGS);
public static final FunctionIdentifier WIN_MARK_FIRST_MISSING_IMPL = new FunctionIdentifier(
FunctionConstants.ASTERIX_NS, "win-mark-first-missing-impl", FunctionIdentifier.VARARGS);
+ public static final FunctionIdentifier WIN_MARK_FIRST_NULL_IMPL = new FunctionIdentifier(
+ FunctionConstants.ASTERIX_NS, "win-mark-first-null-impl", FunctionIdentifier.VARARGS);
public static final FunctionIdentifier WIN_PARTITION_LENGTH_IMPL =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "win-partition-length-impl", 0);
@@ -1031,6 +1059,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "subset-collection", 3);
public static final FunctionIdentifier RANGE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2);
+ public static final FunctionIdentifier SPATIAL_TILE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "spatial-tile", 4);
// fuzzy functions
public static final FunctionIdentifier FUZZY_EQ =
@@ -1090,24 +1120,42 @@
// constructors:
public static final FunctionIdentifier BOOLEAN_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "boolean", 1);
+ public static final FunctionIdentifier BOOLEAN_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "boolean-default-null", 1);
public static final FunctionIdentifier STRING_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string", 1);
+ public static final FunctionIdentifier STRING_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-default-null", 1);
public static final FunctionIdentifier BINARY_HEX_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "hex", 1);
public static final FunctionIdentifier BINARY_BASE64_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "base64", 1);
+ public static final FunctionIdentifier BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "base64-default-null", 1);
public static final FunctionIdentifier INT8_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int8", 1);
+ public static final FunctionIdentifier INT8_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int8-default-null", 1);
public static final FunctionIdentifier INT16_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int16", 1);
+ public static final FunctionIdentifier INT16_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int16-default-null", 1);
public static final FunctionIdentifier INT32_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int32", 1);
+ public static final FunctionIdentifier INT32_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int32-default-null", 1);
public static final FunctionIdentifier INT64_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int64", 1);
+ public static final FunctionIdentifier INT64_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "int64-default-null", 1);
public static final FunctionIdentifier FLOAT_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "float", 1);
+ public static final FunctionIdentifier FLOAT_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "float-default-null", 1);
public static final FunctionIdentifier DOUBLE_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "double", 1);
+ public static final FunctionIdentifier DOUBLE_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "double-default-null", 1);
public static final FunctionIdentifier POINT_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "point", 1);
public static final FunctionIdentifier POINT3D_CONSTRUCTOR =
@@ -1122,19 +1170,45 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "polygon", 1);
public static final FunctionIdentifier TIME_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 1);
+ public static final FunctionIdentifier TIME_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time-default-null", 1);
+ public static final FunctionIdentifier TIME_CONSTRUCTOR_WITH_FORMAT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time", 2);
+ public static final FunctionIdentifier TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "time-default-null", 2);
public static final FunctionIdentifier DATE_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 1);
+ public static final FunctionIdentifier DATE_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date-default-null", 1);
+ public static final FunctionIdentifier DATE_CONSTRUCTOR_WITH_FORMAT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date", 2);
+ public static final FunctionIdentifier DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date-default-null", 2);
public static final FunctionIdentifier DATETIME_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 1);
+ public static final FunctionIdentifier DATETIME_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime-default-null", 1);
+ public static final FunctionIdentifier DATETIME_CONSTRUCTOR_WITH_FORMAT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime", 2);
+ public static final FunctionIdentifier DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime-default-null", 2);
public static final FunctionIdentifier DURATION_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration", 1);
+ public static final FunctionIdentifier DURATION_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration-default-null", 1);
public static final FunctionIdentifier UUID_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "uuid", 1);
+ public static final FunctionIdentifier UUID_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "uuid-default-null", 1);
public static final FunctionIdentifier YEAR_MONTH_DURATION_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "year-month-duration", 1);
+ public static final FunctionIdentifier YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "year-month-duration-default-null", 1);
public static final FunctionIdentifier DAY_TIME_DURATION_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "day-time-duration", 1);
+ public static final FunctionIdentifier DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "day-time-duration-default-null", 1);
public static final FunctionIdentifier INTERVAL_CONSTRUCTOR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "interval", 2);
@@ -1172,10 +1246,16 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "interval-ended-by", 2);
public static final FunctionIdentifier CURRENT_TIME =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "current-time", 0);
+ public static final FunctionIdentifier CURRENT_TIME_IMMEDIATE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "current-time-immediate", 0);
public static final FunctionIdentifier CURRENT_DATE =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "current-date", 0);
+ public static final FunctionIdentifier CURRENT_DATE_IMMEDIATE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "current-date-immediate", 0);
public static final FunctionIdentifier CURRENT_DATETIME =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "current-datetime", 0);
+ public static final FunctionIdentifier CURRENT_DATETIME_IMMEDIATE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "current-datetime-immediate", 0);
public static final FunctionIdentifier DURATION_EQUAL =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "duration-equal", 2);
public static final FunctionIdentifier YEAR_MONTH_DURATION_GREATER_THAN =
@@ -1233,6 +1313,10 @@
public static final FunctionIdentifier CAST_TYPE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cast", 1);
public static final FunctionIdentifier CAST_TYPE_LAX =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cast-lax", 1);
+ public static final FunctionIdentifier REFERENCE_TILE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reference-tile", 6);
+ public static final FunctionIdentifier GET_INTERSECTION =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-intersection", 2);
public static final FunctionIdentifier CREATE_UUID =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-uuid", 0);
@@ -1376,6 +1460,10 @@
public static final FunctionIdentifier ST_POLYGONIZE =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-polygonize", 1);
+ public static final FunctionIdentifier ST_MBR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-mbr", 1);
+ public static final FunctionIdentifier ST_MBR_ENLARGE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-mbr-enlarge", 2);
+
// Spatial and temporal type accessors
public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-year", 1);
@@ -1421,8 +1509,12 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "unix-time-from-time-in-ms", 1);
public final static FunctionIdentifier UNIX_TIME_FROM_DATETIME_IN_MS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "unix-time-from-datetime-in-ms", 1);
+ public final static FunctionIdentifier UNIX_TIME_FROM_DATETIME_IN_MS_WITH_TZ =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "unix-time-from-datetime-in-ms", 2);
public final static FunctionIdentifier UNIX_TIME_FROM_DATETIME_IN_SECS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "unix-time-from-datetime-in-secs", 1);
+ public final static FunctionIdentifier UNIX_TIME_FROM_DATETIME_IN_SECS_WITH_TZ =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "unix-time-from-datetime-in-secs", 2);
public static final FunctionIdentifier DATE_FROM_UNIX_TIME_IN_DAYS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "date-from-unix-time-in-days", 1);
public static final FunctionIdentifier DATE_FROM_DATETIME =
@@ -1433,8 +1525,12 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-time-from-datetime", 1);
public static final FunctionIdentifier DATETIME_FROM_UNIX_TIME_IN_MS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime-from-unix-time-in-ms", 1);
+ public static final FunctionIdentifier DATETIME_FROM_UNIX_TIME_IN_MS_WITH_TZ =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime-from-unix-time-in-ms", 2);
public static final FunctionIdentifier DATETIME_FROM_UNIX_TIME_IN_SECS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime-from-unix-time-in-secs", 1);
+ public static final FunctionIdentifier DATETIME_FROM_UNIX_TIME_IN_SECS_WITH_TZ =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime-from-unix-time-in-secs", 2);
public static final FunctionIdentifier DATETIME_FROM_DATE_TIME =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "datetime-from-date-time", 2);
public static final FunctionIdentifier CALENDAR_DURATION_FROM_DATETIME =
@@ -1447,6 +1543,16 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "adjust-datetime-for-timezone", 2);
public static final FunctionIdentifier DAY_OF_WEEK =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "day-of-week", 1);
+ public static final FunctionIdentifier DAY_OF_WEEK2 =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "day-of-week", 2);
+ public static final FunctionIdentifier DAY_OF_YEAR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "day-of-year", 1);
+ public static final FunctionIdentifier QUARTER_OF_YEAR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "quarter-of-year", 1);
+ public static final FunctionIdentifier WEEK_OF_YEAR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "week-of-year", 1);
+ public static final FunctionIdentifier WEEK_OF_YEAR2 =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "week-of-year", 2);
public static final FunctionIdentifier PARSE_DATE =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "parse-date", 2);
public static final FunctionIdentifier PARSE_TIME =
@@ -1640,7 +1746,7 @@
// and then, Asterix builtin functions
addPrivateFunction(CHECK_UNKNOWN, NotUnknownTypeComputer.INSTANCE, true);
addPrivateFunction(ANY_COLLECTION_MEMBER, CollectionMemberResultType.INSTANCE_MISSABLE, true);
- addFunction(BOOLEAN_CONSTRUCTOR, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(BOOLEAN_CONSTRUCTOR, ABooleanTypeComputer.INSTANCE_NULLABLE, true);
addFunction(CIRCLE_CONSTRUCTOR, ACircleTypeComputer.INSTANCE, true);
addPrivateFunction(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE, true);
addFunction(GROUPING, AInt64TypeComputer.INSTANCE, true);
@@ -1656,23 +1762,25 @@
addFunction(CREATE_UUID, AUUIDTypeComputer.INSTANCE, false);
addFunction(UUID, AUUIDTypeComputer.INSTANCE, false);
addPrivateFunction(CREATE_QUERY_UID, ABinaryTypeComputer.INSTANCE, false);
- addFunction(UUID_CONSTRUCTOR, AUUIDTypeComputer.INSTANCE, true);
+ addFunction(UUID_CONSTRUCTOR, AUUIDTypeComputer.INSTANCE_NULLABLE, true);
addFunction(RANDOM, ADoubleTypeComputer.INSTANCE, false);
addFunction(RANDOM_WITH_SEED, NumericUnaryTypeComputer.INSTANCE_DOUBLE, false);
- addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE, true);
- addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE, true);
- addFunction(DOUBLE_CONSTRUCTOR, ADoubleTypeComputer.INSTANCE, true);
- addFunction(DURATION_CONSTRUCTOR, ADurationTypeComputer.INSTANCE, true);
- addFunction(YEAR_MONTH_DURATION_CONSTRUCTOR, AYearMonthDurationTypeComputer.INSTANCE, true);
- addFunction(DAY_TIME_DURATION_CONSTRUCTOR, ADayTimeDurationTypeComputer.INSTANCE, true);
+ addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(DATE_CONSTRUCTOR_WITH_FORMAT, ADateTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(DATETIME_CONSTRUCTOR_WITH_FORMAT, ADateTimeTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(DOUBLE_CONSTRUCTOR, ADoubleTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(DURATION_CONSTRUCTOR, ADurationTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(YEAR_MONTH_DURATION_CONSTRUCTOR, AYearMonthDurationTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(DAY_TIME_DURATION_CONSTRUCTOR, ADayTimeDurationTypeComputer.INSTANCE_NULLABLE, true);
addFunction(EDIT_DISTANCE, AInt64TypeComputer.INSTANCE, true);
addFunction(EDIT_DISTANCE_CHECK, OrderedListOfAnyTypeComputer.INSTANCE, true);
addPrivateFunction(EDIT_DISTANCE_STRING_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE, true);
addPrivateFunction(EDIT_DISTANCE_LIST_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE, true);
addPrivateFunction(EMPTY_STREAM, ABooleanTypeComputer.INSTANCE, true);
- addFunction(FLOAT_CONSTRUCTOR, AFloatTypeComputer.INSTANCE, true);
+ addFunction(FLOAT_CONSTRUCTOR, AFloatTypeComputer.INSTANCE_NULLABLE, true);
addPrivateFunction(FUZZY_EQ, BooleanFunctionTypeComputer.INSTANCE, true);
addPrivateFunction(GET_HANDLE, AnyTypeComputer.INSTANCE, true);
addPrivateFunction(GET_ITEM, NonTaggedGetItemResultType.INSTANCE, true);
@@ -1681,15 +1789,39 @@
addPrivateFunction(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
addPrivateFunction(HASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
addPrivateFunction(INDEX_SEARCH, AnyTypeComputer.INSTANCE, true);
- addFunction(INT8_CONSTRUCTOR, AInt8TypeComputer.INSTANCE, true);
- addFunction(INT16_CONSTRUCTOR, AInt16TypeComputer.INSTANCE, true);
- addFunction(INT32_CONSTRUCTOR, AInt32TypeComputer.INSTANCE, true);
- addFunction(INT64_CONSTRUCTOR, AInt64TypeComputer.INSTANCE, true);
+ addFunction(INT8_CONSTRUCTOR, AInt8TypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(INT16_CONSTRUCTOR, AInt16TypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(INT32_CONSTRUCTOR, AInt32TypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(INT64_CONSTRUCTOR, AInt64TypeComputer.INSTANCE_NULLABLE, true);
addFunction(LEN, AInt64TypeComputer.INSTANCE, true);
addFunction(LINE_CONSTRUCTOR, ALineTypeComputer.INSTANCE, true);
addPrivateFunction(MAKE_FIELD_INDEX_HANDLE, AnyTypeComputer.INSTANCE, true);
addPrivateFunction(MAKE_FIELD_NAME_HANDLE, AnyTypeComputer.INSTANCE, true);
+ // cast null type constructors
+ addPrivateFunction(BOOLEAN_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BOOLEAN, true);
+ addPrivateFunction(INT8_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT8, true);
+ addPrivateFunction(INT16_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT16, true);
+ addPrivateFunction(INT32_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT32, true);
+ addPrivateFunction(INT64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_INT64, true);
+ addPrivateFunction(FLOAT_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_FLOAT, true);
+ addPrivateFunction(DOUBLE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DOUBLE, true);
+ addPrivateFunction(STRING_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_STRING, true);
+ addPrivateFunction(DATE_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE, true);
+ addPrivateFunction(DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE, true);
+ addPrivateFunction(TIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_TIME, true);
+ addPrivateFunction(TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_TIME, true);
+ addPrivateFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DATE_TIME, true);
+ addPrivateFunction(DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT, NullableTypeComputer.INSTANCE_DATE_TIME,
+ true);
+ addPrivateFunction(DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DURATION, true);
+ addPrivateFunction(DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_DAY_TIME_DURATION,
+ true);
+ addPrivateFunction(YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
+ NullableTypeComputer.INSTANCE_YEAR_MONTH_DURATION, true);
+ addPrivateFunction(UUID_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_UUID, true);
+ addPrivateFunction(BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR, NullableTypeComputer.INSTANCE_BINARY, true);
+
addPrivateFunction(NUMERIC_UNARY_MINUS, NumericUnaryTypeComputer.INSTANCE, true);
addPrivateFunction(NUMERIC_SUBTRACT, NumericAddSubMulDivTypeComputer.INSTANCE_SUB, true);
addPrivateFunction(NUMERIC_MULTIPLY, NumericAddSubMulDivTypeComputer.INSTANCE_MUL_POW, true);
@@ -1723,6 +1855,7 @@
addFunction(NUMERIC_ROUND_WITH_ROUND_DIGIT, NumericRoundTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ROUND_HALF_TO_EVEN, NumericUnaryTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ROUND_HALF_TO_EVEN2, NumericRoundTypeComputer.INSTANCE_ROUND_HF_TRUNC, true);
+ addFunction(NUMERIC_ROUND_HALF_UP2, NumericRoundTypeComputer.INSTANCE_ROUND_HF_TRUNC, true);
addFunction(NUMERIC_TRUNC, NumericRoundTypeComputer.INSTANCE_ROUND_HF_TRUNC, true);
addFunction(BINARY_LENGTH, UnaryBinaryInt64TypeComputer.INSTANCE, true);
@@ -1749,7 +1882,7 @@
addFunction(IS_BIT_SET_WITH_ALL_FLAG, BitValuePositionFlagTypeComputer.INSTANCE_TEST_WITH_FLAG, true);
// string functions
- addFunction(STRING_CONSTRUCTOR, AStringTypeComputer.INSTANCE, true); // TODO(ali)
+ addFunction(STRING_CONSTRUCTOR, AStringTypeComputer.INSTANCE_NULLABLE, true);
addFunction(STRING_LIKE, BooleanFunctionTypeComputer.INSTANCE, true);
addFunction(STRING_CONTAINS, UniformInputTypeComputer.STRING_BOOLEAN_INSTANCE, true);
addFunction(STRING_TO_CODEPOINT, UniformInputTypeComputer.STRING_INT64_LIST_INSTANCE, true);
@@ -1794,6 +1927,7 @@
addFunction(STRING_JOIN, StringJoinTypeComputer.INSTANCE, true);
addFunction(STRING_REPEAT, AStringTypeComputer.INSTANCE_NULLABLE, true);
addFunction(STRING_SPLIT, UniformInputTypeComputer.STRING_STRING_LIST_INSTANCE, true);
+ addFunction(STRING_PARSE_JSON, AnyTypeComputer.INSTANCE, true);
addPrivateFunction(ORDERED_LIST_CONSTRUCTOR, OrderedListConstructorTypeComputer.INSTANCE, true);
addFunction(POINT_CONSTRUCTOR, APointTypeComputer.INSTANCE, true);
@@ -1810,7 +1944,7 @@
addFunction(TO_DOUBLE, ToDoubleTypeComputer.INSTANCE, true);
addFunction(TO_NUMBER, ToNumberTypeComputer.INSTANCE, true);
addFunction(TO_OBJECT, ToObjectTypeComputer.INSTANCE, true);
- addFunction(TO_STRING, AStringTypeComputer.INSTANCE, true);
+ addFunction(TO_STRING, AStringTypeComputer.INSTANCE_NULLABLE, true);
addPrivateFunction(TREAT_AS_INTEGER, TreatAsTypeComputer.INSTANCE_INTEGER, true);
addPrivateFunction(IS_NUMERIC_ADD_COMPATIBLE, BooleanOnlyTypeComputer.INSTANCE, true);
@@ -1834,6 +1968,8 @@
new ScalarVersionOfAggregateResultType(NumericSumAggTypeComputer.INSTANCE);
ScalarVersionOfAggregateResultType scalarMinMaxTypeComputer =
new ScalarVersionOfAggregateResultType(MinMaxAggTypeComputer.INSTANCE);
+ ScalarVersionOfAggregateResultType scalarUnionMbrTypeComputer =
+ new ScalarVersionOfAggregateResultType(UnionMbrAggTypeComputer.INSTANCE);
addPrivateFunction(LISTIFY, OrderedListConstructorTypeComputer.INSTANCE, true);
addFunction(SCALAR_ARRAYAGG, ScalarArrayAggTypeComputer.INSTANCE, true);
@@ -1877,6 +2013,10 @@
addFunction(KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
addPrivateFunction(GLOBAL_KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
addPrivateFunction(NULL_WRITER, PropagateTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addPrivateFunction(LOCAL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addPrivateFunction(INTERMEDIATE_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addPrivateFunction(GLOBAL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
// SUM
addFunction(SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1935,6 +2075,7 @@
addPrivateFunction(SERIAL_GLOBAL_SQL_KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
addPrivateFunction(SERIAL_LOCAL_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
addPrivateFunction(SERIAL_INTERMEDIATE_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
+ addFunction(SCALAR_UNION_MBR, scalarUnionMbrTypeComputer, true);
// SQL SUM
addFunction(SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1995,6 +2136,11 @@
addPrivateFunction(LOCAL_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
addPrivateFunction(INTERMEDIATE_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
addFunction(SCALAR_SQL_KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
+ addFunction(SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addPrivateFunction(LOCAL_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addPrivateFunction(INTERMEDIATE_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addPrivateFunction(GLOBAL_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addFunction(SCALAR_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
addPrivateFunction(SERIAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
addPrivateFunction(SERIAL_COUNT, AInt64TypeComputer.INSTANCE, true);
@@ -2114,6 +2260,7 @@
addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, false);
addFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, false);
addPrivateFunction(WIN_MARK_FIRST_MISSING_IMPL, ABooleanTypeComputer.INSTANCE, false);
+ addPrivateFunction(WIN_MARK_FIRST_NULL_IMPL, ABooleanTypeComputer.INSTANCE, false);
addPrivateFunction(WIN_PARTITION_LENGTH_IMPL, AInt64TypeComputer.INSTANCE, false);
// Similarity functions
@@ -2139,6 +2286,9 @@
addFunction(GET_CIRCLE_RADIUS_ACCESSOR, ADoubleTypeComputer.INSTANCE, true);
addFunction(GET_CIRCLE_CENTER_ACCESSOR, APointTypeComputer.INSTANCE, true);
addFunction(GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR, OrderedListOfAPointTypeComputer.INSTANCE, true);
+ addPrivateFunction(SPATIAL_TILE, AInt32TypeComputer.INSTANCE, true);
+ addPrivateFunction(REFERENCE_TILE, AInt32TypeComputer.INSTANCE, true);
+ addPrivateFunction(GET_INTERSECTION, ARectangleTypeComputer.INSTANCE, true);
//geo functions
addFunction(ST_AREA, ADoubleTypeComputer.INSTANCE, true);
@@ -2209,9 +2359,12 @@
addPrivateFunction(ST_UNION_SQL_AGG, AGeometryTypeComputer.INSTANCE, true);
addFunction(ST_POLYGONIZE, AGeometryTypeComputer.INSTANCE, true);
+ addPrivateFunction(ST_MBR, ARectangleTypeComputer.INSTANCE, true);
+ addPrivateFunction(ST_MBR_ENLARGE, ARectangleTypeComputer.INSTANCE, true);
+
// Binary functions
- addFunction(BINARY_HEX_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
- addFunction(BINARY_BASE64_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
+ addFunction(BINARY_HEX_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(BINARY_BASE64_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE_NULLABLE, true);
addPrivateFunction(SUBSET_COLLECTION, SubsetCollectionTypeComputer.INSTANCE, true);
addFunction(SWITCH_CASE, SwitchCaseComputer.INSTANCE, true);
@@ -2221,7 +2374,8 @@
addPrivateFunction(CAST_TYPE_LAX, CastTypeLaxComputer.INSTANCE, true);
addFunction(TID, AInt64TypeComputer.INSTANCE, true);
- addFunction(TIME_CONSTRUCTOR, ATimeTypeComputer.INSTANCE, true);
+ addFunction(TIME_CONSTRUCTOR, ATimeTypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(TIME_CONSTRUCTOR_WITH_FORMAT, ATimeTypeComputer.INSTANCE_NULLABLE, true);
addPrivateFunction(TYPE_OF, AnyTypeComputer.INSTANCE, true);
addPrivateFunction(UNORDERED_LIST_CONSTRUCTOR, UnorderedListConstructorTypeComputer.INSTANCE, true);
addFunction(WORD_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
@@ -2302,14 +2456,18 @@
addFunction(UNIX_TIME_FROM_DATE_IN_DAYS, AInt64TypeComputer.INSTANCE, true);
addFunction(UNIX_TIME_FROM_TIME_IN_MS, AInt64TypeComputer.INSTANCE, true);
addFunction(UNIX_TIME_FROM_DATETIME_IN_MS, AInt64TypeComputer.INSTANCE, true);
+ addFunction(UNIX_TIME_FROM_DATETIME_IN_MS_WITH_TZ, AInt64TypeComputer.INSTANCE, false);
addFunction(UNIX_TIME_FROM_DATETIME_IN_SECS, AInt64TypeComputer.INSTANCE, true);
+ addFunction(UNIX_TIME_FROM_DATETIME_IN_SECS_WITH_TZ, AInt64TypeComputer.INSTANCE, false);
addFunction(DATE_FROM_UNIX_TIME_IN_DAYS, ADateTypeComputer.INSTANCE, true);
addFunction(DATE_FROM_DATETIME, ADateTypeComputer.INSTANCE, true);
addFunction(TIME_FROM_UNIX_TIME_IN_MS, ATimeTypeComputer.INSTANCE, true);
addFunction(TIME_FROM_DATETIME, ATimeTypeComputer.INSTANCE, true);
addFunction(DATETIME_FROM_DATE_TIME, ADateTimeTypeComputer.INSTANCE, true);
addFunction(DATETIME_FROM_UNIX_TIME_IN_MS, ADateTimeTypeComputer.INSTANCE, true);
+ addFunction(DATETIME_FROM_UNIX_TIME_IN_MS_WITH_TZ, ADateTimeTypeComputer.INSTANCE, false);
addFunction(DATETIME_FROM_UNIX_TIME_IN_SECS, ADateTimeTypeComputer.INSTANCE, true);
+ addFunction(DATETIME_FROM_UNIX_TIME_IN_SECS_WITH_TZ, ADateTimeTypeComputer.INSTANCE, false);
addFunction(CALENDAR_DURATION_FROM_DATETIME, ADurationTypeComputer.INSTANCE, true);
addFunction(CALENDAR_DURATION_FROM_DATE, ADurationTypeComputer.INSTANCE, true);
addFunction(ADJUST_DATETIME_FOR_TIMEZONE, AStringTypeComputer.INSTANCE, true);
@@ -2328,8 +2486,11 @@
addFunction(INTERVAL_ENDS, ABooleanTypeComputer.INSTANCE, true);
addFunction(INTERVAL_ENDED_BY, ABooleanTypeComputer.INSTANCE, true);
addFunction(CURRENT_DATE, ADateTypeComputer.INSTANCE, false);
+ addFunction(CURRENT_DATE_IMMEDIATE, ADateTypeComputer.INSTANCE, false);
addFunction(CURRENT_TIME, ATimeTypeComputer.INSTANCE, false);
+ addFunction(CURRENT_TIME_IMMEDIATE, ATimeTypeComputer.INSTANCE, false);
addFunction(CURRENT_DATETIME, ADateTimeTypeComputer.INSTANCE, false);
+ addFunction(CURRENT_DATETIME_IMMEDIATE, ADateTimeTypeComputer.INSTANCE, false);
addPrivateFunction(DAY_TIME_DURATION_GREATER_THAN, ABooleanTypeComputer.INSTANCE, true);
addPrivateFunction(DAY_TIME_DURATION_LESS_THAN, ABooleanTypeComputer.INSTANCE, true);
addPrivateFunction(YEAR_MONTH_DURATION_GREATER_THAN, ABooleanTypeComputer.INSTANCE, true);
@@ -2343,6 +2504,11 @@
addFunction(GET_YEAR_MONTH_DURATION, AYearMonthDurationTypeComputer.INSTANCE, true);
addFunction(INTERVAL_BIN, AIntervalTypeComputer.INSTANCE, true);
addFunction(DAY_OF_WEEK, AInt64TypeComputer.INSTANCE, true);
+ addFunction(DAY_OF_WEEK2, AInt64TypeComputer.INSTANCE_NULLABLE, true);
+ addFunction(DAY_OF_YEAR, AInt64TypeComputer.INSTANCE, true);
+ addFunction(QUARTER_OF_YEAR, AInt64TypeComputer.INSTANCE, true);
+ addFunction(WEEK_OF_YEAR, AInt64TypeComputer.INSTANCE, true);
+ addFunction(WEEK_OF_YEAR2, AInt64TypeComputer.INSTANCE_NULLABLE, true);
addFunction(PARSE_DATE, ADateTypeComputer.INSTANCE, true);
addFunction(PARSE_TIME, ATimeTypeComputer.INSTANCE, true);
addFunction(PARSE_DATETIME, ADateTimeTypeComputer.INSTANCE, true);
@@ -3018,6 +3184,18 @@
addScalarAgg(ST_UNION_SQL_AGG, SCALAR_ST_UNION_SQL_AGG);
addDistinctAgg(ST_UNION_SQL_AGG_DISTINCT, ST_UNION_SQL_AGG);
addScalarAgg(ST_UNION_SQL_AGG_DISTINCT, SCALAR_ST_UNION_SQL_AGG_DISTINCT);
+
+ // SQL UNION MBR
+ addAgg(SQL_UNION_MBR);
+ addAgg(LOCAL_SQL_UNION_MBR);
+ addAgg(GLOBAL_SQL_UNION_MBR);
+ addLocalAgg(SQL_UNION_MBR, LOCAL_SQL_UNION_MBR);
+ addIntermediateAgg(LOCAL_SQL_UNION_MBR, INTERMEDIATE_SQL_UNION_MBR);
+ addIntermediateAgg(GLOBAL_SQL_UNION_MBR, GLOBAL_SQL_UNION_MBR);
+ addIntermediateAgg(SQL_UNION_MBR, GLOBAL_SQL_UNION_MBR);
+ addGlobalAgg(SQL_UNION_MBR, GLOBAL_SQL_UNION_MBR);
+
+ addScalarAgg(SQL_UNION_MBR, SCALAR_SQL_UNION_MBR);
}
interface BuiltinFunctionProperty {
@@ -3055,6 +3233,7 @@
addWindowFunction(RATIO_TO_REPORT, RATIO_TO_REPORT_IMPL, HAS_LIST_ARG);
addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, NO_FRAME_CLAUSE);
addWindowFunction(null, WIN_MARK_FIRST_MISSING_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
+ addWindowFunction(null, WIN_MARK_FIRST_NULL_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
addWindowFunction(null, WIN_PARTITION_LENGTH_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
}
@@ -3062,6 +3241,7 @@
addUnnestFun(RANGE, true);
addUnnestFun(SCAN_COLLECTION, false);
addUnnestFun(SUBSET_COLLECTION, false);
+ addUnnestFun(SPATIAL_TILE, false);
}
public enum DataSourceFunctionProperty implements BuiltinFunctionProperty {
@@ -3258,11 +3438,18 @@
}
public enum SpatialFilterKind {
- SI
+ SI,
+ STFR
}
static {
spatialFilterFunctions.put(BuiltinFunctions.SPATIAL_INTERSECT, SpatialFilterKind.SI);
+ spatialFilterFunctions.put(BuiltinFunctions.ST_INTERSECTS, SpatialFilterKind.STFR);
+ spatialFilterFunctions.put(BuiltinFunctions.ST_OVERLAPS, SpatialFilterKind.STFR);
+ spatialFilterFunctions.put(BuiltinFunctions.ST_TOUCHES, SpatialFilterKind.STFR);
+ spatialFilterFunctions.put(BuiltinFunctions.ST_CONTAINS, SpatialFilterKind.STFR);
+ spatialFilterFunctions.put(BuiltinFunctions.ST_CROSSES, SpatialFilterKind.STFR);
+ spatialFilterFunctions.put(BuiltinFunctions.ST_WITHIN, SpatialFilterKind.STFR);
}
public static boolean isGlobalAggregateFunction(FunctionIdentifier fi) {
@@ -3270,7 +3457,11 @@
}
public static boolean isSpatialFilterFunction(FunctionIdentifier fi) {
- return spatialFilterFunctions.get(fi) != null;
+ return spatialFilterFunctions.get(fi) == SpatialFilterKind.SI;
+ }
+
+ public static boolean isSTFilterRefineFunction(FunctionIdentifier fi) {
+ return spatialFilterFunctions.get(fi) == SpatialFilterKind.STFR;
}
static {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java
index d1f6ab7..3fd23f7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/AListPrinter.java
@@ -74,7 +74,11 @@
IVisitablePointable item = items.get(i);
ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
.deserialize(itemTypeTag.getByteArray()[itemTypeTag.getStartOffset()]);
- itemVisitorArg.second = item.getLength() <= 1 ? ATypeTag.NULL : typeTag;
+ itemVisitorArg.second = getItemTypeTag(item, typeTag);
item.accept(visitor, itemVisitorArg);
}
+
+ protected ATypeTag getItemTypeTag(IVisitablePointable item, ATypeTag typeTag) {
+ return item.getLength() <= 1 ? ATypeTag.NULL : typeTag;
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
index 3858bd6..7823637 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/ARecordPrinter.java
@@ -86,11 +86,18 @@
IVisitablePointable fieldValue, ATypeTag fieldTypeTag) throws HyracksDataException {
itemVisitorArg.second = fieldTypeTag;
if (fieldNameSeparator != null) {
- // print field name
- fieldName.accept(visitor, nameVisitorArg);
+ printFieldName(ps, visitor, fieldName);
ps.print(fieldNameSeparator);
}
- // print field value
+ printFieldValue(visitor, fieldValue);
+ }
+
+ protected void printFieldName(PrintStream ps, IPrintVisitor visitor, IVisitablePointable fieldName)
+ throws HyracksDataException {
+ fieldName.accept(visitor, nameVisitorArg);
+ }
+
+ protected void printFieldValue(IPrintVisitor visitor, IVisitablePointable fieldValue) throws HyracksDataException {
fieldValue.accept(visitor, itemVisitorArg);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/losslessadm/APrintVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/losslessadm/APrintVisitor.java
new file mode 100644
index 0000000..025590b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/printer/json/losslessadm/APrintVisitor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.om.pointables.printer.json.losslessadm;
+
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.json.losslessadm.AObjectPrinterFactory;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.printer.AListPrinter;
+import org.apache.asterix.om.pointables.printer.ARecordPrinter;
+import org.apache.asterix.om.pointables.printer.AbstractPrintVisitor;
+import org.apache.asterix.om.pointables.printer.IPrintVisitor;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class APrintVisitor extends AbstractPrintVisitor {
+
+ private final org.apache.asterix.om.pointables.printer.json.clean.APrintVisitor cleanPrintVisitor =
+ new org.apache.asterix.om.pointables.printer.json.clean.APrintVisitor();
+
+ @Override
+ protected AListPrinter createListPrinter(AListVisitablePointable accessor) {
+ return new AListPrinter("[ ", " ]", ", ") {
+ @Override
+ protected ATypeTag getItemTypeTag(IVisitablePointable item, ATypeTag typeTag) {
+ // avoid MISSING to NULL conversion, because we print MISSING as is in this format
+ return typeTag;
+ }
+ };
+ }
+
+ @Override
+ protected ARecordPrinter createRecordPrinter(ARecordVisitablePointable accessor) {
+ return new ARecordPrinter("{ ", " }", ", ", ": ") {
+ @Override
+ protected void printFieldName(PrintStream ps, IPrintVisitor visitor, IVisitablePointable fieldName)
+ throws HyracksDataException {
+ super.printFieldName(ps, cleanPrintVisitor, fieldName);
+ }
+ };
+ }
+
+ @Override
+ protected boolean printFlatValue(ATypeTag typeTag, byte[] b, int s, int l, PrintStream ps)
+ throws HyracksDataException {
+ return AObjectPrinterFactory.printFlatValue(typeTag, b, s, l, ps);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java
index 9332633..f60268a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABinaryTypeComputer.java
@@ -19,21 +19,15 @@
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class ABinaryTypeComputer extends AbstractResultTypeComputer {
- public static final ABinaryTypeComputer INSTANCE = new ABinaryTypeComputer();
+public class ABinaryTypeComputer extends AbstractConstructorTypeComputer {
- private ABinaryTypeComputer() {
+ public static final ABinaryTypeComputer INSTANCE = new ABinaryTypeComputer(false);
- }
+ public static final ABinaryTypeComputer INSTANCE_NULLABLE = new ABinaryTypeComputer(true);
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.ABINARY;
+ private ABinaryTypeComputer(boolean nullable) {
+ super(BuiltinType.ABINARY, nullable);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java
index 2a054bf..16349d0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ABooleanTypeComputer.java
@@ -18,22 +18,23 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-public class ABooleanTypeComputer extends AbstractResultTypeComputer {
+public class ABooleanTypeComputer extends AbstractConstructorTypeComputer {
- public static final ABooleanTypeComputer INSTANCE = new ABooleanTypeComputer();
+ public static final ABooleanTypeComputer INSTANCE = new ABooleanTypeComputer(false);
- private ABooleanTypeComputer() {
+ public static final ABooleanTypeComputer INSTANCE_NULLABLE = new ABooleanTypeComputer(true);
+
+ private ABooleanTypeComputer(boolean nullable) {
+ super(BuiltinType.ABOOLEAN, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.ABOOLEAN;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType)
+ || ATypeHierarchy.getTypeDomain(inputType.getTypeTag()) == ATypeHierarchy.Domain.NUMERIC;
}
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java
index 2e2667a..19915c6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTimeTypeComputer.java
@@ -18,22 +18,22 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class ADateTimeTypeComputer extends AbstractResultTypeComputer {
+public class ADateTimeTypeComputer extends AbstractConstructorTypeComputer {
- public static final ADateTimeTypeComputer INSTANCE = new ADateTimeTypeComputer();
+ public static final ADateTimeTypeComputer INSTANCE = new ADateTimeTypeComputer(false);
- private ADateTimeTypeComputer() {
+ public static final ADateTimeTypeComputer INSTANCE_NULLABLE = new ADateTimeTypeComputer(true);
+
+ private ADateTimeTypeComputer(boolean nullable) {
+ super(BuiltinType.ADATETIME, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.ADATETIME;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType) || inputType.getTypeTag() == ATypeTag.DATE;
}
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java
index 5ac7613..dc64137 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADateTypeComputer.java
@@ -18,22 +18,22 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class ADateTypeComputer extends AbstractResultTypeComputer {
+public class ADateTypeComputer extends AbstractConstructorTypeComputer {
- public static final ADateTypeComputer INSTANCE = new ADateTypeComputer();
+ public static final ADateTypeComputer INSTANCE = new ADateTypeComputer(false);
- private ADateTypeComputer() {
+ public static final ADateTypeComputer INSTANCE_NULLABLE = new ADateTypeComputer(true);
+
+ private ADateTypeComputer(boolean nullable) {
+ super(BuiltinType.ADATE, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.ADATE;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType) || inputType.getTypeTag() == ATypeTag.DATETIME;
}
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java
index b8cc9b4..d8d3065 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADayTimeDurationTypeComputer.java
@@ -18,23 +18,22 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class ADayTimeDurationTypeComputer extends AbstractResultTypeComputer {
+public class ADayTimeDurationTypeComputer extends AbstractConstructorTypeComputer {
- public static final ADayTimeDurationTypeComputer INSTANCE = new ADayTimeDurationTypeComputer();
+ public static final ADayTimeDurationTypeComputer INSTANCE = new ADayTimeDurationTypeComputer(false);
- private ADayTimeDurationTypeComputer() {
+ public static final ADayTimeDurationTypeComputer INSTANCE_NULLABLE = new ADayTimeDurationTypeComputer(true);
+ private ADayTimeDurationTypeComputer(boolean nullable) {
+ super(BuiltinType.ADAYTIMEDURATION, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.ADAYTIMEDURATION;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType) || inputType.getTypeTag() == ATypeTag.DURATION;
}
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java
index 00301b6..e1b71a6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADoubleTypeComputer.java
@@ -18,28 +18,15 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
-import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class ADoubleTypeComputer extends AbstractResultTypeComputer {
+public class ADoubleTypeComputer extends AbstractNumericConstructorTypeComputer {
public static final ADoubleTypeComputer INSTANCE = new ADoubleTypeComputer(false);
public static final ADoubleTypeComputer INSTANCE_NULLABLE = new ADoubleTypeComputer(true);
- private final IAType type;
-
private ADoubleTypeComputer(boolean nullable) {
- IAType t = BuiltinType.ADOUBLE;
- type = nullable ? AUnionType.createNullableType(t) : t;
- }
-
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return type;
+ super(BuiltinType.ADOUBLE, nullable);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java
index 4f23ed0..ecc21a3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ADurationTypeComputer.java
@@ -18,22 +18,23 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class ADurationTypeComputer extends AbstractResultTypeComputer {
+public class ADurationTypeComputer extends AbstractConstructorTypeComputer {
- public static final ADurationTypeComputer INSTANCE = new ADurationTypeComputer();
+ public static final ADurationTypeComputer INSTANCE = new ADurationTypeComputer(false);
- private ADurationTypeComputer() {
+ public static final ADurationTypeComputer INSTANCE_NULLABLE = new ADurationTypeComputer(true);
+
+ private ADurationTypeComputer(boolean nullable) {
+ super(BuiltinType.ADURATION, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.ADURATION;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType) || inputType.getTypeTag() == ATypeTag.YEARMONTHDURATION
+ || inputType.getTypeTag() == ATypeTag.DAYTIMEDURATION;
}
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java
index 77c01e1..29d2078 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AFloatTypeComputer.java
@@ -18,22 +18,15 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AFloatTypeComputer extends AbstractResultTypeComputer {
+public class AFloatTypeComputer extends AbstractNumericConstructorTypeComputer {
- public static final AFloatTypeComputer INSTANCE = new AFloatTypeComputer();
+ public static final AFloatTypeComputer INSTANCE = new AFloatTypeComputer(false);
- private AFloatTypeComputer() {
+ public static final AFloatTypeComputer INSTANCE_NULLABLE = new AFloatTypeComputer(true);
+
+ private AFloatTypeComputer(boolean nullable) {
+ super(BuiltinType.AFLOAT, nullable);
}
-
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.AFLOAT;
- }
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java
index 1fbc603..c5be15e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt16TypeComputer.java
@@ -18,22 +18,13 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AInt16TypeComputer extends AbstractResultTypeComputer {
+public class AInt16TypeComputer extends AbstractNumericConstructorTypeComputer {
- public static final AInt16TypeComputer INSTANCE = new AInt16TypeComputer();
+ public static final AInt16TypeComputer INSTANCE_NULLABLE = new AInt16TypeComputer(true);
- private AInt16TypeComputer() {
+ private AInt16TypeComputer(boolean nullable) {
+ super(BuiltinType.AINT16, nullable);
}
-
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.AINT16;
- }
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java
index 8f676ae..0edf633 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt32TypeComputer.java
@@ -18,22 +18,15 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AInt32TypeComputer extends AbstractResultTypeComputer {
+public class AInt32TypeComputer extends AbstractNumericConstructorTypeComputer {
- public static final AInt32TypeComputer INSTANCE = new AInt32TypeComputer();
+ public static final AInt32TypeComputer INSTANCE = new AInt32TypeComputer(false);
- private AInt32TypeComputer() {
+ public static final AInt32TypeComputer INSTANCE_NULLABLE = new AInt32TypeComputer(true);
+
+ private AInt32TypeComputer(boolean nullable) {
+ super(BuiltinType.AINT32, nullable);
}
-
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.AINT32;
- }
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java
index d495acf..34699d2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt64TypeComputer.java
@@ -18,28 +18,15 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
-import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AInt64TypeComputer extends AbstractResultTypeComputer {
+public class AInt64TypeComputer extends AbstractNumericConstructorTypeComputer {
public static final AInt64TypeComputer INSTANCE = new AInt64TypeComputer(false);
public static final AInt64TypeComputer INSTANCE_NULLABLE = new AInt64TypeComputer(true);
- private final IAType type;
-
private AInt64TypeComputer(boolean nullable) {
- IAType t = BuiltinType.AINT64;
- type = nullable ? AUnionType.createNullableType(t) : t;
- }
-
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return type;
+ super(BuiltinType.AINT64, nullable);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java
index 4fe6dc3..29e9843 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AInt8TypeComputer.java
@@ -18,22 +18,13 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AInt8TypeComputer extends AbstractResultTypeComputer {
+public class AInt8TypeComputer extends AbstractNumericConstructorTypeComputer {
- public static final AInt8TypeComputer INSTANCE = new AInt8TypeComputer();
+ public static final AInt8TypeComputer INSTANCE_NULLABLE = new AInt8TypeComputer(true);
- private AInt8TypeComputer() {
+ private AInt8TypeComputer(boolean nullable) {
+ super(BuiltinType.AINT8, nullable);
}
-
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.AINT8;
- }
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java
index d392e8d..cd7bd54 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AStringTypeComputer.java
@@ -18,27 +18,43 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
-import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AStringTypeComputer extends AbstractResultTypeComputer {
+public class AStringTypeComputer extends AbstractConstructorTypeComputer {
- public static final AStringTypeComputer INSTANCE = new AStringTypeComputer(BuiltinType.ASTRING);
- public static final AStringTypeComputer INSTANCE_NULLABLE =
- new AStringTypeComputer(AUnionType.createNullableType(BuiltinType.ASTRING));
+ public static final AStringTypeComputer INSTANCE = new AStringTypeComputer(false);
- private final IAType outputType;
+ public static final AStringTypeComputer INSTANCE_NULLABLE = new AStringTypeComputer(true);
- private AStringTypeComputer(IAType outputType) {
- this.outputType = outputType;
+ private AStringTypeComputer(boolean nullable) {
+ super(BuiltinType.ASTRING, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return outputType;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ if (super.isAlwaysCastable(inputType)) {
+ return true;
+ }
+ switch (inputType.getTypeTag()) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ case BOOLEAN:
+ case DATETIME:
+ case DATE:
+ case TIME:
+ case DURATION:
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ case UUID:
+ case BINARY:
+ return true;
+ default:
+ return false;
+ }
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java
index 359a9e4..f395946 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ATimeTypeComputer.java
@@ -18,22 +18,22 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class ATimeTypeComputer extends AbstractResultTypeComputer {
+public class ATimeTypeComputer extends AbstractConstructorTypeComputer {
- public static final ATimeTypeComputer INSTANCE = new ATimeTypeComputer();
+ public static final ATimeTypeComputer INSTANCE = new ATimeTypeComputer(false);
- private ATimeTypeComputer() {
+ public static final ATimeTypeComputer INSTANCE_NULLABLE = new ATimeTypeComputer(true);
+
+ private ATimeTypeComputer(boolean nullable) {
+ super(BuiltinType.ATIME, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.ATIME;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType) || inputType.getTypeTag() == ATypeTag.DATETIME;
}
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java
index d241585..9063844 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AUUIDTypeComputer.java
@@ -19,22 +19,15 @@
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AUUIDTypeComputer extends AbstractResultTypeComputer {
+public class AUUIDTypeComputer extends AbstractConstructorTypeComputer {
- public static final AUUIDTypeComputer INSTANCE = new AUUIDTypeComputer();
+ public static final AUUIDTypeComputer INSTANCE = new AUUIDTypeComputer(false);
- private AUUIDTypeComputer() {
+ public static final AUUIDTypeComputer INSTANCE_NULLABLE = new AUUIDTypeComputer(true);
+
+ private AUUIDTypeComputer(boolean nullable) {
+ super(BuiltinType.AUUID, nullable);
}
-
- @Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.AUUID;
- }
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java
index 7885a75..023f4d4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AYearMonthDurationTypeComputer.java
@@ -18,23 +18,22 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class AYearMonthDurationTypeComputer extends AbstractResultTypeComputer {
+public class AYearMonthDurationTypeComputer extends AbstractConstructorTypeComputer {
- public static final AYearMonthDurationTypeComputer INSTANCE = new AYearMonthDurationTypeComputer();
+ public static final AYearMonthDurationTypeComputer INSTANCE = new AYearMonthDurationTypeComputer(false);
- private AYearMonthDurationTypeComputer() {
+ public static final AYearMonthDurationTypeComputer INSTANCE_NULLABLE = new AYearMonthDurationTypeComputer(true);
+ private AYearMonthDurationTypeComputer(boolean nullable) {
+ super(BuiltinType.AYEARMONTHDURATION, nullable);
}
@Override
- protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- return BuiltinType.AYEARMONTHDURATION;
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType) || inputType.getTypeTag() == ATypeTag.DURATION;
}
-
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractConstructorTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractConstructorTypeComputer.java
new file mode 100644
index 0000000..a1db49f
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractConstructorTypeComputer.java
@@ -0,0 +1,54 @@
+/*
+ * 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.om.typecomputer.impl;
+
+import java.util.Objects;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public abstract class AbstractConstructorTypeComputer extends AbstractResultTypeComputer {
+
+ protected final IAType primeType;
+
+ protected final boolean nullable;
+
+ protected AbstractConstructorTypeComputer(IAType primeType, boolean nullable) {
+ this.primeType = Objects.requireNonNull(primeType);
+ this.nullable = nullable;
+ }
+
+ @Override
+ protected final IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes)
+ throws AlgebricksException {
+ if (!nullable || (strippedInputTypes.length == 1 && isAlwaysCastable(strippedInputTypes[0]))) {
+ return primeType;
+ } else {
+ return AUnionType.createNullableType(primeType);
+ }
+ }
+
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return primeType.deepEqual(inputType);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractNumericConstructorTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractNumericConstructorTypeComputer.java
new file mode 100644
index 0000000..080dc7b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractNumericConstructorTypeComputer.java
@@ -0,0 +1,36 @@
+/*
+ * 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.om.typecomputer.impl;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+
+public abstract class AbstractNumericConstructorTypeComputer extends AbstractConstructorTypeComputer {
+
+ protected AbstractNumericConstructorTypeComputer(IAType primeType, boolean nullable) {
+ super(primeType, nullable);
+ }
+
+ @Override
+ protected boolean isAlwaysCastable(IAType inputType) {
+ return super.isAlwaysCastable(inputType)
+ || ATypeHierarchy.getTypeDomain(inputType.getTypeTag()) == ATypeHierarchy.Domain.NUMERIC;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
index 633feeb..3adbc28 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NonTaggedGetItemResultType.java
@@ -19,6 +19,7 @@
package org.apache.asterix.om.typecomputer.impl;
import org.apache.asterix.om.exceptions.TypeMismatchException;
+import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
@@ -44,7 +45,7 @@
ATypeTag actualTypeTag = type.getTypeTag();
if (argIndex == 0) {
if (type.getTypeTag() != ATypeTag.MULTISET && type.getTypeTag() != ATypeTag.ARRAY) {
- throw new TypeMismatchException(sourceLoc, funcId, argIndex, actualTypeTag, ATypeTag.STRING,
+ throw new TypeMismatchException(sourceLoc, funcId, argIndex, actualTypeTag, ATypeTag.MULTISET,
ATypeTag.ARRAY);
}
} else {
@@ -60,6 +61,10 @@
if (type.getTypeTag() == ATypeTag.ANY) {
return BuiltinType.ANY;
}
+ if (!type.getTypeTag().isListType()) {
+ throw new TypeMismatchException(expr.getSourceLocation(), BuiltinFunctions.GET_ITEM, 0, type.getTypeTag(),
+ ATypeTag.MULTISET, ATypeTag.ARRAY);
+ }
IAType itemType = ((AbstractCollectionType) type).getItemType();
if (itemType.getTypeTag() == ATypeTag.ANY) {
return itemType;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullableTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullableTypeComputer.java
new file mode 100644
index 0000000..eaa15c7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullableTypeComputer.java
@@ -0,0 +1,77 @@
+/*
+ * 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.om.typecomputer.impl;
+
+import static org.apache.asterix.om.types.BuiltinType.ABINARY;
+import static org.apache.asterix.om.types.BuiltinType.ABOOLEAN;
+import static org.apache.asterix.om.types.BuiltinType.ADATE;
+import static org.apache.asterix.om.types.BuiltinType.ADATETIME;
+import static org.apache.asterix.om.types.BuiltinType.ADAYTIMEDURATION;
+import static org.apache.asterix.om.types.BuiltinType.ADOUBLE;
+import static org.apache.asterix.om.types.BuiltinType.ADURATION;
+import static org.apache.asterix.om.types.BuiltinType.AFLOAT;
+import static org.apache.asterix.om.types.BuiltinType.AINT16;
+import static org.apache.asterix.om.types.BuiltinType.AINT32;
+import static org.apache.asterix.om.types.BuiltinType.AINT64;
+import static org.apache.asterix.om.types.BuiltinType.AINT8;
+import static org.apache.asterix.om.types.BuiltinType.ASTRING;
+import static org.apache.asterix.om.types.BuiltinType.ATIME;
+import static org.apache.asterix.om.types.BuiltinType.AUUID;
+import static org.apache.asterix.om.types.BuiltinType.AYEARMONTHDURATION;
+
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class NullableTypeComputer implements IResultTypeComputer {
+
+ public static final NullableTypeComputer INSTANCE_INT8 = new NullableTypeComputer(AINT8);
+ public static final NullableTypeComputer INSTANCE_INT16 = new NullableTypeComputer(AINT16);
+ public static final NullableTypeComputer INSTANCE_INT32 = new NullableTypeComputer(AINT32);
+ public static final NullableTypeComputer INSTANCE_INT64 = new NullableTypeComputer(AINT64);
+ public static final NullableTypeComputer INSTANCE_FLOAT = new NullableTypeComputer(AFLOAT);
+ public static final NullableTypeComputer INSTANCE_DOUBLE = new NullableTypeComputer(ADOUBLE);
+ public static final NullableTypeComputer INSTANCE_BOOLEAN = new NullableTypeComputer(ABOOLEAN);
+ public static final NullableTypeComputer INSTANCE_STRING = new NullableTypeComputer(ASTRING);
+ public static final NullableTypeComputer INSTANCE_DATE = new NullableTypeComputer(ADATE);
+ public static final NullableTypeComputer INSTANCE_TIME = new NullableTypeComputer(ATIME);
+ public static final NullableTypeComputer INSTANCE_DATE_TIME = new NullableTypeComputer(ADATETIME);
+ public static final NullableTypeComputer INSTANCE_DURATION = new NullableTypeComputer(ADURATION);
+ public static final NullableTypeComputer INSTANCE_DAY_TIME_DURATION = new NullableTypeComputer(ADAYTIMEDURATION);
+ public static final NullableTypeComputer INSTANCE_YEAR_MONTH_DURATION =
+ new NullableTypeComputer(AYEARMONTHDURATION);
+ public static final NullableTypeComputer INSTANCE_UUID = new NullableTypeComputer(AUUID);
+ public static final NullableTypeComputer INSTANCE_BINARY = new NullableTypeComputer(ABINARY);
+
+ private final IAType nullablePrimeType;
+
+ private NullableTypeComputer(IAType primeType) {
+ this.nullablePrimeType = AUnionType.createNullableType(primeType);
+ }
+
+ @Override
+ public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+ IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+ return nullablePrimeType;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
index 6deb17c..6b08230 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
@@ -22,9 +22,11 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
+import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
@@ -51,7 +53,7 @@
IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
- /**
+ /*
* if type has been top-down propagated, use the enforced type
*/
ARecordType type = (ARecordType) TypeCastUtils.getRequiredType(f);
@@ -66,6 +68,7 @@
// but are additional possible field names. For example, a field "foo" with type
// ANY cannot be in the closed part, but "foo" is a possible field name.
Set<String> allPossibleAdditionalFieldNames = new HashSet<>();
+ LinkedHashSet<String> allPossibleFieldNamesOrdered = new LinkedHashSet<>();
boolean canProvideAdditionFieldInfo = true;
boolean isOpen = false;
while (argIter.hasNext()) {
@@ -91,11 +94,17 @@
}
isOpen = true;
}
+ allPossibleFieldNamesOrdered.add(fieldName);
}
String[] fieldNames = namesList.toArray(new String[0]);
IAType[] fieldTypes = typesList.toArray(new IAType[0]);
- return canProvideAdditionFieldInfo
- ? new ARecordType(null, fieldNames, fieldTypes, isOpen, allPossibleAdditionalFieldNames)
- : new ARecordType(null, fieldNames, fieldTypes, isOpen);
+ ARecordType resultType;
+ if (isOpen && canProvideAdditionFieldInfo) {
+ resultType = new ARecordType(null, fieldNames, fieldTypes, isOpen, allPossibleAdditionalFieldNames);
+ resultType.getAnnotations().add(new RecordFieldOrderAnnotation(allPossibleFieldNamesOrdered));
+ } else {
+ resultType = new ARecordType(null, fieldNames, fieldTypes, isOpen);
+ }
+ return resultType;
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnionMbrAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnionMbrAggTypeComputer.java
new file mode 100644
index 0000000..ed03b21
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnionMbrAggTypeComputer.java
@@ -0,0 +1,48 @@
+/*
+ * 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.om.typecomputer.impl;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class UnionMbrAggTypeComputer extends AggregateResultTypeComputer {
+
+ public static final UnionMbrAggTypeComputer INSTANCE = new UnionMbrAggTypeComputer();
+
+ private UnionMbrAggTypeComputer() {
+ }
+
+ @Override
+ protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+ ATypeTag tag = strippedInputTypes[0].getTypeTag();
+ switch (tag) {
+ case RECTANGLE:
+ return AUnionType.createNullableType(BuiltinType.ARECTANGLE);
+ case ANY:
+ return BuiltinType.ANY;
+ default:
+ // All other possible cases.
+ return BuiltinType.ANULL;
+ }
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index b91b4ae..e256e1b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -22,8 +22,10 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Set;
import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
@@ -145,6 +147,17 @@
return annotations;
}
+ public IRecordTypeAnnotation findAnnotation(IRecordTypeAnnotation.Kind kind) {
+ if (annotations != null) {
+ for (IRecordTypeAnnotation ant : annotations) {
+ if (ant.getKind().equals(kind)) {
+ return ant;
+ }
+ }
+ }
+ return null;
+ }
+
@Override
public String toString() {
return append(new StringBuilder()).toString();
@@ -312,7 +325,10 @@
newTypes[i] = type.fieldTypes[i];
}
}
- return new ARecordType(type.typeName, type.fieldNames, newTypes, type.isOpen);
+ Set<String> newAllPossibleAdditionalFieldNames =
+ allPossibleAdditionalFieldNames != null ? new HashSet<>(allPossibleAdditionalFieldNames) : null;
+ return new ARecordType(type.typeName, type.fieldNames, newTypes, type.isOpen,
+ newAllPossibleAdditionalFieldNames);
}
@Override
@@ -344,7 +360,8 @@
}
ARecordType rt = (ARecordType) obj;
return (isOpen == rt.isOpen) && Arrays.deepEquals(fieldNames, rt.fieldNames)
- && Arrays.deepEquals(fieldTypes, rt.fieldTypes);
+ && Arrays.deepEquals(fieldTypes, rt.fieldTypes)
+ && Objects.equals(allPossibleAdditionalFieldNames, rt.allPossibleAdditionalFieldNames);
}
@Override
@@ -428,4 +445,9 @@
}
return false;
}
+
+ @Override
+ public <R, T> R accept(IATypeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
index 7ef18d4..caa926d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AUnionType.java
@@ -70,6 +70,16 @@
return false;
}
+ public IAType getType(ATypeTag typeTag) {
+ for (int i = 0; i < unionList.size(); i++) {
+ IAType type = unionList.get(i);
+ if (typeTag == type.getTypeTag()) {
+ return type;
+ }
+ }
+ return null;
+ }
+
public IAType getActualType() {
return unionList.get(AUnionType.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
}
@@ -241,6 +251,11 @@
return jsonObject;
}
+ @Override
+ public <R, T> R accept(IATypeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
+
public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
throws HyracksDataException {
String typeName = json.get(TYPE_NAME_FIELD).asText();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
index ef15224..9b75566 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/AbstractCollectionType.java
@@ -70,6 +70,11 @@
return isTyped() && itemType.getTypeName().equals(type.getTypeName());
}
+ @Override
+ public <R, T> R accept(IATypeVisitor<R, T> visitor, T arg) {
+ return visitor.visit(this, arg);
+ }
+
JsonNode convertToJson(IPersistedResourceRegistry registry, Class<? extends IJsonSerializable> clazz, long version)
throws HyracksDataException {
final ObjectNode jsonObject = registry.getClassIdentifier(clazz, version);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
index bdac9e9..2404ac2d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
@@ -1081,6 +1081,14 @@
return getType().getTypeTag().serialize();
}
+ /**
+ * Visit built-in type as a flat type
+ */
+ @Override
+ public <R, T> R accept(IATypeVisitor<R, T> visitor, T arg) {
+ return visitor.visitFlat(this, arg);
+ }
+
private static JsonNode convertToJson(IPersistedResourceRegistry registry, short tag, long version) {
ObjectNode jsonNode = registry.getClassIdentifier(BuiltinType.class, version);
jsonNode.put(TAG_FIELD, tag);
@@ -1090,7 +1098,10 @@
@SuppressWarnings("squid:S1172") // unused parameter
public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
byte tag = (byte) json.get(TAG_FIELD).shortValue();
- ATypeTag typeTag = VALUE_TYPE_MAPPING[tag];
+ return getBuiltinType(VALUE_TYPE_MAPPING[tag]);
+ }
+
+ public static IAType getBuiltinType(ATypeTag typeTag) {
switch (typeTag) {
case TYPE:
return ALL_TYPE;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/IAType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/IAType.java
index 0e6cc4f..ed4c2bb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/IAType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/IAType.java
@@ -31,4 +31,11 @@
public String getTypeName();
+ /**
+ * Allow for additional traversal and processing for {@link IAType}
+ *
+ * @param visitor visitor
+ * @param arg visitor's argument
+ */
+ <R, T> R accept(IATypeVisitor<R, T> visitor, T arg);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/IATypeVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/IATypeVisitor.java
new file mode 100644
index 0000000..0951763
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/IATypeVisitor.java
@@ -0,0 +1,35 @@
+/*
+ * 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.om.types;
+
+/**
+ * Allows for a specialized processing for the {@link IAType}
+ *
+ * @param <R> return type
+ * @param <T> argument type
+ */
+public interface IATypeVisitor<R, T> {
+ R visit(ARecordType recordType, T arg);
+
+ R visit(AbstractCollectionType collectionType, T arg);
+
+ R visit(AUnionType unionType, T arg);
+
+ R visitFlat(IAType flatType, T arg);
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
index b444053..645c12a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
@@ -29,6 +29,7 @@
public static boolean canBeMissing(IAType t) {
switch (t.getTypeTag()) {
case MISSING:
+ case ANY:
return true;
case UNION:
return ((AUnionType) t).isMissableType();
@@ -37,6 +38,18 @@
}
}
+ public static boolean canBeNull(IAType t) {
+ switch (t.getTypeTag()) {
+ case NULL:
+ case ANY:
+ return true;
+ case UNION:
+ return ((AUnionType) t).isNullableType();
+ default:
+ return false;
+ }
+ }
+
public static boolean canBeUnknown(IAType t) {
switch (t.getTypeTag()) {
case MISSING:
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java
index 85d71a8..a8e2633 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt16TypeConvertComputer.java
@@ -85,8 +85,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (short) Math.floor(sourceValue);
+ return (short) sourceValue;
}
private void raiseBoundaryCheckException(double sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java
index 5653731..4181d41 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt32TypeConvertComputer.java
@@ -88,8 +88,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (int) Math.floor(sourceValue);
+ return (int) sourceValue;
}
private int raiseBoundaryCheckException(double sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java
index cb9a6bc..a2bb3a6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt64TypeConvertComputer.java
@@ -93,8 +93,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (long) Math.floor(sourceValue);
+ return (long) sourceValue;
}
private void raiseBoundaryCheckException(double sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java
index 69934b5..afb7ff6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/DoubleToInt8TypeConvertComputer.java
@@ -85,8 +85,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (byte) Math.floor(sourceValue);
+ return (byte) sourceValue;
}
private void raiseBoundaryCheckException(double sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java
index 4906c56..e15dd87 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt16TypeConvertComputer.java
@@ -85,8 +85,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (short) Math.floor(sourceValue);
+ return (short) sourceValue;
}
private void raiseBoundaryException(float sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java
index 366278d..bbbc7a8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt32TypeConvertComputer.java
@@ -88,8 +88,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (int) Math.floor(sourceValue);
+ return (int) sourceValue;
}
private void raiseBoundaryCheckException(float sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java
index 291cab6..2416300 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt64TypeConvertComputer.java
@@ -88,8 +88,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (long) Math.floor(sourceValue);
+ return (long) sourceValue;
}
private void raiseBoundaryCheckException(float sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java
index 30f19c3..2aff3239 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/hierachy/FloatToInt8TypeConvertComputer.java
@@ -84,8 +84,7 @@
}
}
- // Math.floor to truncate decimal portion
- return (byte) Math.floor(sourceValue);
+ return (byte) sourceValue;
}
private void raiseBoundaryException(float sourceValue) throws HyracksDataException {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/visitor/SimpleStringBuilderForIATypeVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/visitor/SimpleStringBuilderForIATypeVisitor.java
new file mode 100644
index 0000000..9f0c82b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/visitor/SimpleStringBuilderForIATypeVisitor.java
@@ -0,0 +1,99 @@
+/*
+ * 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.om.types.visitor;
+
+import java.util.List;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.IATypeVisitor;
+
+/**
+ * This visitor produces a oneliner JSON-like representation of {@link IAType} to be interpreted by the user.
+ */
+public class SimpleStringBuilderForIATypeVisitor implements IATypeVisitor<Void, StringBuilder> {
+ /**
+ * Example: {"field1":string,"field2":[bigint]}
+ */
+ @Override
+ public Void visit(ARecordType recordType, StringBuilder arg) {
+ String[] fieldNames = recordType.getFieldNames();
+ IAType[] fieldTypes = recordType.getFieldTypes();
+
+ arg.append("{");
+ for (int i = 0; i < fieldNames.length; i++) {
+ if (i > 0) {
+ arg.append(',');
+ }
+ arg.append(fieldNames[i]);
+ arg.append(':');
+ fieldTypes[i].accept(this, arg);
+ }
+ arg.append("}");
+ return null;
+ }
+
+ /**
+ * Example:
+ * - Array: [{"field1":bigint}]
+ * - Multiset: {{bigint}}
+ */
+ @Override
+ public Void visit(AbstractCollectionType collectionType, StringBuilder arg) {
+ IAType itemType = collectionType.getItemType();
+
+ arg.append(collectionType.getTypeTag() == ATypeTag.ARRAY ? "[" : "{{");
+ itemType.accept(this, arg);
+ arg.append(collectionType.getTypeTag() == ATypeTag.ARRAY ? "]" : "}}");
+ return null;
+ }
+
+ /**
+ * Example: A union type of array, object, and bigint
+ * - <[{"field1":...}],{"field1:...},bigint>
+ */
+ @Override
+ public Void visit(AUnionType unionType, StringBuilder arg) {
+ List<IAType> unionList = unionType.getUnionList();
+
+ arg.append("<");
+ for (int i = 0; i < unionList.size(); i++) {
+ if (i > 0) {
+ arg.append(',');
+ }
+ unionList.get(i).accept(this, arg);
+ }
+ arg.append(">");
+ return null;
+ }
+
+ /**
+ * Example:
+ * - bigint
+ * - string
+ */
+ @Override
+ public Void visitFlat(IAType flatType, StringBuilder arg) {
+ arg.append(flatType.getTypeTag());
+ return null;
+ }
+}
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/common/TypeResolverUtilTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/common/TypeResolverUtilTest.java
index 5303870..dad6ee9 100644
--- a/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/common/TypeResolverUtilTest.java
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/common/TypeResolverUtilTest.java
@@ -20,11 +20,15 @@
package org.apache.asterix.dataflow.data.common;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
+import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
+import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
+import org.apache.asterix.common.annotations.RecordFieldOrderAnnotation;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.AUnionType;
@@ -124,6 +128,42 @@
}
@Test
+ public void testRecordTypeFieldOrderHint() {
+ // Constructs input types.
+ ARecordType leftRecordType = new ARecordType(null, new String[] { "a", "b", "c" },
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, }, true,
+ new HashSet<>(Arrays.asList("d", "e")));
+ leftRecordType.getAnnotations()
+ .add(new RecordFieldOrderAnnotation(new LinkedHashSet<>(Arrays.asList("a", "b", "c", "d", "e"))));
+
+ ARecordType rightRecordType = new ARecordType(null, new String[] { "a", "c", "d" },
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, }, true,
+ new HashSet<>(Arrays.asList("e", "f")));
+ rightRecordType.getAnnotations()
+ .add(new RecordFieldOrderAnnotation(new LinkedHashSet<>(Arrays.asList("a", "c", "d", "e", "f"))));
+
+ // Resolves input types to a generalized type.
+ List<IAType> inputTypes = new ArrayList<>();
+ inputTypes.add(leftRecordType);
+ inputTypes.add(rightRecordType);
+ ARecordType resolvedType = (ARecordType) TypeResolverUtil.resolve(inputTypes);
+
+ // Constructs the expected type.
+ Set<String> possibleAdditionalFields = new HashSet<>(Arrays.asList("b", "d", "e", "f"));
+ ARecordType expectedType = new ARecordType(null, new String[] { "a", "c" },
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, true, possibleAdditionalFields);
+ expectedType.getAnnotations()
+ .add(new RecordFieldOrderAnnotation(new LinkedHashSet<>(Arrays.asList("a", "b", "c", "d", "e", "f"))));
+
+ // Compares the resolved type with the expected type.
+ Assert.assertEquals(expectedType, resolvedType);
+
+ IRecordTypeAnnotation expecedAnn = expectedType.findAnnotation(IRecordTypeAnnotation.Kind.RECORD_FIELD_ORDER);
+ IRecordTypeAnnotation resolvedAnn = resolvedType.findAnnotation(IRecordTypeAnnotation.Kind.RECORD_FIELD_ORDER);
+ Assert.assertEquals(expecedAnn, resolvedAnn);
+ }
+
+ @Test
public void testOrderedListType() {
// Constructs input types.
ARecordType leftRecordType = new ARecordType("null", new String[] { "a", "b" },
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/test/om/types/visitor/TypeSimpleStringBuilderTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/test/om/types/visitor/TypeSimpleStringBuilderTest.java
new file mode 100644
index 0000000..8186810
--- /dev/null
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/test/om/types/visitor/TypeSimpleStringBuilderTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.test.om.types.visitor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.visitor.SimpleStringBuilderForIATypeVisitor;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TypeSimpleStringBuilderTest {
+ private static final ARecordType ROOT_TYPE;
+ private static final String EXPECTED_STRING;
+
+ static {
+ StringBuilder expectedStringBuilder = new StringBuilder();
+ //Record with two fields
+ ARecordType recordType = createNestedRecord(expectedStringBuilder);
+ String recordTypeString = getStringAndReset(expectedStringBuilder);
+
+ //Array of records
+ AOrderedListType arrayOfRecords = new AOrderedListType(recordType, "arrayOfRecords");
+ surroundString(expectedStringBuilder, "[", "]", recordTypeString);
+ String arrayOfRecordsString = getStringAndReset(expectedStringBuilder);
+
+ //Multiset of records
+ AUnorderedListType multiSetOfRecords = new AUnorderedListType(recordType, "multiSetOfRecords");
+ surroundString(expectedStringBuilder, "{{", "}}", recordTypeString);
+ String multiSetOfRecordsString = getStringAndReset(expectedStringBuilder);
+
+ //Union
+ List<IAType> unionList = new ArrayList<>();
+ unionList.add(recordType);
+ unionList.add(arrayOfRecords);
+ unionList.add(multiSetOfRecords);
+ unionList.add(BuiltinType.AINT64);
+ AUnionType unionType = new AUnionType(unionList, "unionType");
+ surroundString(expectedStringBuilder, "<", ">", recordTypeString, arrayOfRecordsString, multiSetOfRecordsString,
+ BuiltinType.AINT64.getTypeTag().toString());
+ String unionTypeString = getStringAndReset(expectedStringBuilder);
+
+ //Root type
+ String[] rootFieldNames = { BuiltinType.ANY.getTypeName(), arrayOfRecords.getTypeName(),
+ multiSetOfRecords.getTypeName(), unionType.getTypeName() };
+ IAType[] rootFieldTypes = { BuiltinType.ANY, arrayOfRecords, multiSetOfRecords, unionType };
+ ROOT_TYPE = new ARecordType("rootType", rootFieldNames, rootFieldTypes, false);
+ buildRecordString(expectedStringBuilder, rootFieldNames, rootFieldTypes[0].getTypeTag().toString(),
+ arrayOfRecordsString, multiSetOfRecordsString, unionTypeString);
+ EXPECTED_STRING = getStringAndReset(expectedStringBuilder);
+ }
+
+ private static ARecordType createNestedRecord(StringBuilder builder) {
+ String[] fieldNames = { "field1", "field2" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.AINT64 };
+ buildRecordString(builder, fieldNames, fieldTypes[0].getTypeTag().toString(),
+ fieldTypes[1].getTypeTag().toString());
+ return new ARecordType("nestedRecord", fieldNames, fieldTypes, true);
+ }
+
+ private static void surroundString(StringBuilder builder, String open, String close, String... strings) {
+ builder.append(open);
+ for (int i = 0; i < strings.length; i++) {
+ if (i > 0) {
+ builder.append(',');
+ }
+ builder.append(strings[i]);
+ }
+ builder.append(close);
+ }
+
+ private static void buildRecordString(StringBuilder builder, String[] fieldNames, String... fieldTypesStrings) {
+ builder.append('{');
+ for (int i = 0; i < fieldNames.length; i++) {
+ if (i > 0) {
+ builder.append(',');
+ }
+ builder.append(fieldNames[i]);
+ builder.append(':');
+ builder.append(fieldTypesStrings[i]);
+ }
+ builder.append('}');
+ }
+
+ private static String getStringAndReset(StringBuilder expectedStringBuilder) {
+ String value = expectedStringBuilder.toString();
+ expectedStringBuilder.setLength(0);
+ return value;
+ }
+
+ @Test
+ public void testSimpleStringBuilderForIAType() {
+ StringBuilder builder = new StringBuilder();
+ SimpleStringBuilderForIATypeVisitor visitor = new SimpleStringBuilderForIATypeVisitor();
+ ROOT_TYPE.accept(visitor, builder);
+ Assert.assertEquals(EXPECTED_STRING, builder.toString());
+ }
+
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
index 6b97306..e1f99f4 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
@@ -24,6 +24,8 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.exceptions.ReplicationException;
@@ -52,9 +54,12 @@
private static final int INITIAL_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(4, StorageUtil.StorageUnit.KILOBYTE);
private final INcApplicationContext appCtx;
private final ReplicaIdentifier id;
+ private double syncProgress = -1;
+ private long lastProgressTime = -1;
private ByteBuffer reusbaleBuf;
private PartitionReplicaStatus status = DISCONNECTED;
private ISocketChannel sc;
+ private Future<?> syncFuture;
public PartitionReplica(ReplicaIdentifier id, INcApplicationContext appCtx) {
this.id = id;
@@ -77,13 +82,18 @@
}
public synchronized void sync() {
+ sync(true, true);
+ }
+
+ public synchronized void sync(boolean register, boolean deltaRecovery) {
if (status == IN_SYNC || status == CATCHING_UP) {
return;
}
setStatus(CATCHING_UP);
- appCtx.getThreadExecutor().execute(() -> {
+ ExecutorService threadExecutor = (ExecutorService) appCtx.getThreadExecutor();
+ syncFuture = threadExecutor.submit(() -> {
try {
- new ReplicaSynchronizer(appCtx, this).sync();
+ new ReplicaSynchronizer(appCtx, this).sync(register, deltaRecovery);
setStatus(IN_SYNC);
} catch (Exception e) {
LOGGER.error(() -> "Failed to sync replica " + this, e);
@@ -94,6 +104,13 @@
});
}
+ public synchronized void abort() {
+ if (syncFuture != null) {
+ syncFuture.cancel(true);
+ }
+ syncFuture = null;
+ }
+
public synchronized ISocketChannel getChannel() {
try {
if (!NetworkingUtil.isHealthy(sc)) {
@@ -129,10 +146,21 @@
return reusbaleBuf;
}
+ public synchronized void setSyncProgress(double syncProgress) {
+ this.syncProgress = syncProgress;
+ lastProgressTime = System.nanoTime();
+ }
+
+ @Override
+ public synchronized double getSyncProgress() {
+ return syncProgress;
+ }
+
private JsonNode asJson() {
ObjectNode json = OBJECT_MAPPER.createObjectNode();
json.put("id", id.toString());
json.put("status", status.name());
+ json.put("syncProgress", syncProgress);
return json;
}
@@ -149,6 +177,19 @@
}
@Override
+ public synchronized long getLastProgressTime() {
+ switch (status) {
+ case IN_SYNC:
+ return System.nanoTime();
+ case CATCHING_UP:
+ return lastProgressTime;
+ case DISCONNECTED:
+ return -1;
+ }
+ return -1;
+ }
+
+ @Override
public int hashCode() {
return id.hashCode();
}
@@ -168,6 +209,17 @@
}
LOGGER.info(() -> "Replica " + this + " status changing: " + this.status + " -> " + status);
this.status = status;
+ switch (status) {
+ case IN_SYNC:
+ syncProgress = 1;
+ break;
+ case CATCHING_UP:
+ lastProgressTime = System.nanoTime();
+ break;
+ case DISCONNECTED:
+ syncProgress = -1;
+ break;
+ }
}
private void sendGoodBye() {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
index 440f8ef..004b640 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
@@ -19,6 +19,7 @@
package org.apache.asterix.replication.logging;
import java.io.IOException;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -70,7 +71,7 @@
+ System.lineSeparator()).getBytes());
break;
case LogType.FLUSH:
- checkpointReplicaIndexes(logRecord, logRecord.getDatasetId());
+ checkpointReplicaIndexes(logRecord, logRecord.getDatasetId(), logRecord.getResourcePartition());
break;
default:
throw new IllegalStateException("Unexpected log type: " + logRecord.getLogType());
@@ -83,13 +84,16 @@
}
}
- private void checkpointReplicaIndexes(RemoteLogRecord remoteLogMapping, int datasetId) throws HyracksDataException {
+ private void checkpointReplicaIndexes(RemoteLogRecord remoteLogMapping, int datasetId, int resourcePartition)
+ throws HyracksDataException {
final Set<Integer> masterPartitions = appCtx.getReplicaManager().getPartitions();
final Predicate<LocalResource> replicaIndexesPredicate = lr -> {
DatasetLocalResource dls = (DatasetLocalResource) lr.getResource();
- return dls.getDatasetId() == datasetId && !masterPartitions.contains(dls.getPartition());
+ return dls.getDatasetId() == datasetId && dls.getPartition() == resourcePartition
+ && !masterPartitions.contains(dls.getPartition());
};
- final Map<Long, LocalResource> resources = localResourceRep.getResources(replicaIndexesPredicate);
+ final Map<Long, LocalResource> resources =
+ localResourceRep.getResources(replicaIndexesPredicate, Collections.singleton(resourcePartition));
final List<DatasetResourceReference> replicaIndexesRef =
resources.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
for (DatasetResourceReference replicaIndexRef : replicaIndexesRef) {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
index dd953c4..48eb8e3 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
@@ -83,8 +83,9 @@
if (failedDest.contains(dest)) {
return;
}
- LOGGER.error("Replica failed", e);
+ LOGGER.debug("Replica failed", e);
if (destinations.contains(dest)) {
+ LOGGER.error("replica at {} failed", dest);
failedDest.add(dest);
}
replicationManager.notifyFailure(dest, e);
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
index 6a23ae6..f1d8d4d 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
@@ -215,11 +215,15 @@
if (failedSockets.contains(replicaSocket)) {
return;
}
- LOGGER.error("Replica failed", e);
+ LOGGER.debug("Replica failed", e);
failedSockets.add(replicaSocket);
Optional<ReplicationDestination> socketDest = destinations.entrySet().stream()
.filter(entry -> entry.getValue().equals(replicaSocket)).map(Map.Entry::getKey).findFirst();
- socketDest.ifPresent(dest -> replicationManager.notifyFailure(dest, e));
+ if (socketDest.isPresent()) {
+ ReplicationDestination dest = socketDest.get();
+ LOGGER.error("replica at {} failed", dest);
+ replicationManager.notifyFailure(dest, e);
+ }
}
private class TxnAckListener implements Runnable {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
index 7ed674e..5cd26de 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
@@ -30,7 +30,7 @@
import org.apache.asterix.common.replication.IReplicationDestination;
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.ReplicationStrategyFactory;
+import org.apache.asterix.common.replication.IReplicationStrategyFactory;
import org.apache.asterix.common.transactions.ILogRecord;
import org.apache.asterix.replication.api.ReplicationDestination;
import org.apache.hyracks.api.replication.IReplicationJob;
@@ -50,10 +50,11 @@
private final LogReplicationManager logReplicationManager;
private final IndexReplicationManager lsnIndexReplicationManager;
- public ReplicationManager(INcApplicationContext appCtx, ReplicationProperties replicationProperties) {
+ public ReplicationManager(INcApplicationContext appCtx, IReplicationStrategyFactory replicationStrategyFactory,
+ ReplicationProperties replicationProperties) {
this.replicationProperties = replicationProperties;
this.appCtx = appCtx;
- strategy = ReplicationStrategyFactory.create(replicationProperties.getReplicationStrategy());
+ strategy = replicationStrategyFactory.create(replicationProperties.getReplicationStrategy());
logReplicationManager = new LogReplicationManager(appCtx, this);
lsnIndexReplicationManager = new IndexReplicationManager(appCtx, this);
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
index 767eb76..97b6556 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.replication.messaging;
+import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.UNINITIALIZED_COMPONENT_SEQ;
+
import java.io.DataInput;
import java.io.DataOutputStream;
import java.io.IOException;
@@ -46,10 +48,12 @@
private final int partition;
private final long maxComponentId;
+ private final String masterNodeId;
- public CheckpointPartitionIndexesTask(int partition, long maxComponentId) {
+ public CheckpointPartitionIndexesTask(int partition, long maxComponentId, String masterNodeId) {
this.partition = partition;
this.maxComponentId = maxComponentId;
+ this.masterNodeId = masterNodeId;
}
@Override
@@ -64,7 +68,6 @@
for (LocalResource ls : partitionResources) {
DatasetResourceReference ref = DatasetResourceReference.of(ls);
final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(ref);
- indexCheckpointManager.delete();
// Get most recent sequence of existing files to avoid deletion
Path indexPath = StoragePathUtil.getIndexPath(ioManager, ref);
String[] files = indexPath.toFile().list(AbstractLSMIndexFileManager.COMPONENT_FILES_FILTER);
@@ -72,12 +75,16 @@
throw HyracksDataException
.create(new IOException(indexPath + " is not a directory or an IO Error occurred"));
}
- long maxComponentSequence = Long.MIN_VALUE;
+ long maxComponentSequence = UNINITIALIZED_COMPONENT_SEQ;
for (String file : files) {
maxComponentSequence =
Math.max(maxComponentSequence, IndexComponentFileReference.of(file).getSequenceEnd());
}
- indexCheckpointManager.init(maxComponentSequence, currentLSN, maxComponentId);
+ if (indexCheckpointManager.getCheckpointCount() > 0) {
+ indexCheckpointManager.flushed(maxComponentSequence, currentLSN, maxComponentId, masterNodeId);
+ } else {
+ indexCheckpointManager.init(maxComponentSequence, currentLSN, maxComponentId, masterNodeId);
+ }
}
ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
}
@@ -93,6 +100,11 @@
DataOutputStream dos = new DataOutputStream(out);
dos.writeInt(partition);
dos.writeLong(maxComponentId);
+ boolean hasMaster = masterNodeId != null;
+ dos.writeBoolean(hasMaster);
+ if (hasMaster) {
+ dos.writeUTF(masterNodeId);
+ }
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -102,7 +114,9 @@
try {
int partition = input.readInt();
long maxComponentId = input.readLong();
- return new CheckpointPartitionIndexesTask(partition, maxComponentId);
+ final boolean hasMaster = input.readBoolean();
+ final String masterNodeId = hasMaster ? input.readUTF() : null;
+ return new CheckpointPartitionIndexesTask(partition, maxComponentId, masterNodeId);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeletePartitionTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeletePartitionTask.java
new file mode 100644
index 0000000..90139df
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeletePartitionTask.java
@@ -0,0 +1,75 @@
+/*
+ * 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.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class DeletePartitionTask implements IReplicaTask {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final int partitionId;
+
+ public DeletePartitionTask(int partitionId) {
+ this.partitionId = partitionId;
+ }
+
+ @Override
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+ try {
+ PersistentLocalResourceRepository localResourceRepository =
+ (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+ LOGGER.warn("deleting storage partition {}", partitionId);
+ localResourceRepository.deletePartition(partitionId);
+ ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
+ } catch (Exception e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ @Override
+ public ReplicationProtocol.ReplicationRequestType getMessageType() {
+ return ReplicationProtocol.ReplicationRequestType.DELETE_PARTITION;
+ }
+
+ @Override
+ public void serialize(OutputStream out) throws HyracksDataException {
+ try {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeInt(partitionId);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ public static DeletePartitionTask create(DataInput input) throws IOException {
+ return new DeletePartitionTask(input.readInt());
+ }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
index 637efa9..04f9766 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
@@ -23,7 +23,6 @@
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
-import java.util.logging.Logger;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.exceptions.ReplicationException;
@@ -32,13 +31,15 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.api.util.IoUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
/**
* A task to drop an index that was dropped on master
*/
public class DropIndexTask implements IReplicaTask {
- private static final Logger LOGGER = Logger.getLogger(DeleteFileTask.class.getName());
+ private static final Logger LOGGER = LogManager.getLogger();
private final String file;
public DropIndexTask(String file) {
@@ -55,7 +56,7 @@
IoUtil.delete(indexDir);
LOGGER.info(() -> "Deleted index: " + indexFile.getAbsolutePath());
} else {
- LOGGER.warning(() -> "Requested to delete a non-existing index: " + indexFile.getAbsolutePath());
+ LOGGER.warn(() -> "Requested to delete a non-existing index: " + indexFile.getAbsolutePath());
}
ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
} catch (IOException e) {
@@ -65,7 +66,7 @@
@Override
public ReplicationProtocol.ReplicationRequestType getMessageType() {
- return ReplicationProtocol.ReplicationRequestType.REPLICATE_RESOURCE_FILE;
+ return ReplicationProtocol.ReplicationRequestType.DROP_INDEX;
}
@Override
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
index b8f61d0..1ea076d 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
@@ -46,11 +46,13 @@
private final long masterLsn;
private final long lastComponentId;
private final String file;
+ private final String masterNodeId;
- public MarkComponentValidTask(String file, long masterLsn, long lastComponentId) {
+ public MarkComponentValidTask(String file, long masterLsn, long lastComponentId, String masterNodeId) {
this.file = file;
this.lastComponentId = lastComponentId;
this.masterLsn = masterLsn;
+ this.masterNodeId = masterNodeId;
}
@Override
@@ -75,7 +77,7 @@
final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
final long componentSequence = IndexComponentFileReference.of(indexRef.getName()).getSequenceEnd();
- indexCheckpointManager.advanceValidComponentSequence(componentSequence);
+ indexCheckpointManager.advanceValidComponent(componentSequence, lastComponentId);
}
private void ensureComponentLsnFlushed(INcApplicationContext appCtx)
@@ -95,7 +97,7 @@
replicationTimeOut -= TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
}
final long componentSequence = IndexComponentFileReference.of(indexRef.getName()).getSequenceEnd();
- indexCheckpointManager.replicated(componentSequence, masterLsn, lastComponentId);
+ indexCheckpointManager.replicated(componentSequence, masterLsn, lastComponentId, masterNodeId);
}
}
@@ -111,6 +113,11 @@
dos.writeUTF(file);
dos.writeLong(masterLsn);
dos.writeLong(lastComponentId);
+ boolean hasMaster = masterNodeId != null;
+ dos.writeBoolean(hasMaster);
+ if (hasMaster) {
+ dos.writeUTF(masterNodeId);
+ }
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -120,6 +127,8 @@
final String indexFile = input.readUTF();
final long lsn = input.readLong();
final long lastComponentId = input.readLong();
- return new MarkComponentValidTask(indexFile, lsn, lastComponentId);
+ final boolean hasMaster = input.readBoolean();
+ final String masterNodeId = hasMaster ? input.readUTF() : null;
+ return new MarkComponentValidTask(indexFile, lsn, lastComponentId, masterNodeId);
}
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
index 9c3902e..1a5ba88 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
@@ -23,7 +23,9 @@
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.asterix.replication.api.IReplicationMessage;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,11 +33,16 @@
public class PartitionResourcesListResponse implements IReplicationMessage {
private final int partition;
- private final List<String> resources;
+ private final Map<String, Long> partitionReplicatedResources;
+ private final List<String> files;
+ private final boolean owner;
- public PartitionResourcesListResponse(int partition, List<String> resources) {
+ public PartitionResourcesListResponse(int partition, Map<String, Long> partitionReplicatedResources,
+ List<String> files, boolean owner) {
this.partition = partition;
- this.resources = resources;
+ this.partitionReplicatedResources = partitionReplicatedResources;
+ this.files = files;
+ this.owner = owner;
}
@Override
@@ -48,17 +55,23 @@
try {
DataOutputStream dos = new DataOutputStream(out);
dos.writeInt(partition);
- dos.writeInt(resources.size());
- for (String file : resources) {
+ dos.writeInt(files.size());
+ for (String file : files) {
dos.writeUTF(file);
}
+ dos.writeBoolean(owner);
+ dos.writeInt(partitionReplicatedResources.size());
+ for (Map.Entry<String, Long> stringLongEntry : partitionReplicatedResources.entrySet()) {
+ dos.writeUTF(stringLongEntry.getKey());
+ dos.writeLong(stringLongEntry.getValue());
+ }
} catch (IOException e) {
throw HyracksDataException.create(e);
}
}
- public List<String> getResources() {
- return resources;
+ public List<String> getFiles() {
+ return files;
}
public static PartitionResourcesListResponse create(DataInput input) throws IOException {
@@ -68,6 +81,20 @@
for (int i = 0; i < size; i++) {
resources.add(input.readUTF());
}
- return new PartitionResourcesListResponse(partition, resources);
+ boolean owner = input.readBoolean();
+ int resourceSize = input.readInt();
+ Map<String, Long> partitionReplicatedResources = new HashMap<>();
+ for (int i = 0; i < resourceSize; i++) {
+ partitionReplicatedResources.put(input.readUTF(), input.readLong());
+ }
+ return new PartitionResourcesListResponse(partition, partitionReplicatedResources, resources, owner);
+ }
+
+ public boolean isOwner() {
+ return owner;
+ }
+
+ public Map<String, Long> getPartitionReplicatedResources() {
+ return partitionReplicatedResources;
}
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
index cff12de..3ea252f 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
@@ -23,6 +23,7 @@
import java.io.IOException;
import java.io.OutputStream;
import java.util.List;
+import java.util.Map;
import java.util.stream.Collectors;
import org.apache.asterix.common.api.INcApplicationContext;
@@ -50,11 +51,15 @@
(PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
localResourceRepository.cleanup(partition);
final IReplicationStrategy replicationStrategy = appCtx.getReplicationManager().getReplicationStrategy();
- final List<String> partitionResources =
+ // .metadata file -> resource id
+ Map<String, Long> partitionReplicatedResources =
+ localResourceRepository.getPartitionReplicatedResources(partition, replicationStrategy);
+ // all data files in partitions + .metadata files
+ final List<String> partitionFiles =
localResourceRepository.getPartitionReplicatedFiles(partition, replicationStrategy).stream()
.map(StoragePathUtil::getFileRelativePath).collect(Collectors.toList());
- final PartitionResourcesListResponse response =
- new PartitionResourcesListResponse(partition, partitionResources);
+ final PartitionResourcesListResponse response = new PartitionResourcesListResponse(partition,
+ partitionReplicatedResources, partitionFiles, appCtx.getReplicaManager().isPartitionOwner(partition));
ReplicationProtocol.sendTo(worker.getChannel(), response, worker.getReusableBuffer());
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
index 5f7bee6..bf1613c 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.replication.messaging;
+import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.UNINITIALIZED_COMPONENT_SEQ;
+
import java.io.DataInput;
import java.io.DataOutputStream;
import java.io.IOException;
@@ -37,6 +39,7 @@
import org.apache.asterix.replication.api.IReplicaTask;
import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.replication.management.NetworkingUtil;
+import org.apache.commons.io.FileUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IIOManager;
@@ -53,26 +56,31 @@
private final String file;
private final long size;
private final boolean indexMetadata;
+ private final String masterNodeId;
- public ReplicateFileTask(String file, long size, boolean indexMetadata) {
+ public ReplicateFileTask(String file, long size, boolean indexMetadata, String masterNodeId) {
this.file = file;
this.size = size;
this.indexMetadata = indexMetadata;
+ this.masterNodeId = masterNodeId;
}
@Override
public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
try {
- LOGGER.info("attempting to replicate {}", this);
+ LOGGER.debug("attempting to receive file {} from master", this);
final IIOManager ioManager = appCtx.getIoManager();
// resolve path
final FileReference localPath = ioManager.resolve(file);
final Path resourceDir = Files.createDirectories(localPath.getFile().getParentFile().toPath());
+ if (indexMetadata) {
+ // ensure clean index directory
+ FileUtils.cleanDirectory(resourceDir.toFile());
+ }
// create mask
final Path maskPath = Paths.get(resourceDir.toString(),
StorageConstants.MASK_FILE_PREFIX + localPath.getFile().getName());
Files.createFile(maskPath);
-
// receive actual file
final Path filePath = Paths.get(resourceDir.toString(), localPath.getFile().getName());
Files.createFile(filePath);
@@ -87,7 +95,7 @@
}
//delete mask
Files.delete(maskPath);
- LOGGER.info(() -> "Replicated file: " + localPath);
+ LOGGER.info("received file {} from master", localPath);
ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
} catch (IOException e) {
throw new ReplicationException(e);
@@ -100,8 +108,8 @@
final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
indexCheckpointManager.delete();
- indexCheckpointManager.init(Long.MIN_VALUE, currentLSN,
- LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId());
+ indexCheckpointManager.init(UNINITIALIZED_COMPONENT_SEQ, currentLSN,
+ LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId(), masterNodeId);
LOGGER.info(() -> "Checkpoint index: " + indexRef);
}
@@ -117,6 +125,11 @@
dos.writeUTF(file);
dos.writeLong(size);
dos.writeBoolean(indexMetadata);
+ boolean hasMaster = masterNodeId != null;
+ dos.writeBoolean(hasMaster);
+ if (hasMaster) {
+ dos.writeUTF(masterNodeId);
+ }
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -126,7 +139,9 @@
final String s = input.readUTF();
final long i = input.readLong();
final boolean isMetadata = input.readBoolean();
- return new ReplicateFileTask(s, i, isMetadata);
+ final boolean hasMaster = input.readBoolean();
+ final String masterNodeId = hasMaster ? input.readUTF() : null;
+ return new ReplicateFileTask(s, i, isMetadata, masterNodeId);
}
@Override
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java
index ed2c93f..5b0c64e 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java
@@ -61,7 +61,8 @@
LSM_COMPONENT_MASK,
MARK_COMPONENT_VALID,
DROP_INDEX,
- REPLICATE_LOGS
+ REPLICATE_LOGS,
+ DELETE_PARTITION
}
private static final Map<Integer, ReplicationRequestType> TYPES = new HashMap<>();
@@ -177,6 +178,8 @@
return MarkComponentValidTask.create(dis);
case REPLICATE_LOGS:
return ReplicateLogsTask.create(dis);
+ case DELETE_PARTITION:
+ return DeletePartitionTask.create(dis);
default:
throw new IllegalStateException("Unrecognized replication message");
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
index 7bb2858..813b293 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
@@ -55,7 +55,9 @@
final IIOManager ioManager = appCtx.getIoManager();
final ISocketChannel channel = replica.getChannel();
final FileReference filePath = ioManager.resolve(file);
- ReplicateFileTask task = new ReplicateFileTask(file, filePath.getFile().length(), metadata);
+ String masterNode = appCtx.getReplicaManager().isPartitionOwner(replica.getIdentifier().getPartition())
+ ? appCtx.getServiceContext().getNodeId() : null;
+ ReplicateFileTask task = new ReplicateFileTask(file, filePath.getFile().length(), metadata, masterNode);
LOGGER.info("attempting to replicate {} to replica {}", task, replica);
ReplicationProtocol.sendTo(replica, task);
// send the file itself
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
index 47f872c..00e63ec 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
@@ -94,8 +94,10 @@
final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::replicate);
// send mark component valid
- MarkComponentValidTask markValidTask =
- new MarkComponentValidTask(indexFile, getReplicatedComponentLsn(), getReplicatedComponentId());
+ String masterNode = appCtx.getReplicaManager().isPartitionOwner(replica.getIdentifier().getPartition())
+ ? appCtx.getServiceContext().getNodeId() : null;
+ MarkComponentValidTask markValidTask = new MarkComponentValidTask(indexFile, getReplicatedComponentLsn(),
+ getReplicatedComponentId(), masterNode);
ReplicationProtocol.sendTo(replica, markValidTask);
ReplicationProtocol.waitForAck(replica);
LOGGER.debug("Replicated component ({}) to replica {}", indexFile, replica);
@@ -139,11 +141,12 @@
private long getReplicatedComponentId() throws HyracksDataException {
final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
- if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
+ if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH
+ && indexReplJob.getLSMOpType() != LSMOperationType.LOAD) {
return -1L;
}
final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
LSMComponentId id = (LSMComponentId) ctx.getComponentsToBeReplicated().get(0).getId();
- return id.getMinId();
+ return id.getMaxId();
}
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
index b47fd39..477559c 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
@@ -20,72 +20,187 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.replication.IReplicationStrategy;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IndexCheckpoint;
+import org.apache.asterix.common.storage.ResourceReference;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.messaging.DeletePartitionTask;
import org.apache.asterix.replication.messaging.PartitionResourcesListResponse;
import org.apache.asterix.replication.messaging.PartitionResourcesListTask;
import org.apache.asterix.replication.messaging.ReplicationProtocol;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.network.ISocketChannel;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexComponentFileReference;
+import org.apache.hyracks.storage.common.LocalResource;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
/**
* Ensures that the files between master and a replica are synchronized
*/
public class ReplicaFilesSynchronizer {
+ private static final Logger LOGGER = LogManager.getLogger();
private final PartitionReplica replica;
private final INcApplicationContext appCtx;
+ private final boolean deltaRecovery;
- public ReplicaFilesSynchronizer(INcApplicationContext appCtx, PartitionReplica replica) {
+ public ReplicaFilesSynchronizer(INcApplicationContext appCtx, PartitionReplica replica, boolean deltaRecovery) {
this.appCtx = appCtx;
this.replica = replica;
+ this.deltaRecovery = deltaRecovery;
}
public void sync() throws IOException {
final int partition = replica.getIdentifier().getPartition();
- final Set<String> replicaFiles = getReplicaFiles(partition);
+ if (!deltaRecovery) {
+ deletePartitionFromReplica(partition);
+ }
+ PartitionResourcesListResponse replicaResourceResponse = getReplicaFiles(partition);
+ Map<ResourceReference, Long> resourceReferenceLongMap = getValidReplicaResources(
+ replicaResourceResponse.getPartitionReplicatedResources(), replicaResourceResponse.isOwner());
+ // clean up files for invalid resources (deleted or recreated while the replica was down)
+ Set<String> deletedReplicaFiles =
+ cleanupReplicaInvalidResources(replicaResourceResponse, resourceReferenceLongMap);
final PersistentLocalResourceRepository localResourceRepository =
(PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
final IReplicationStrategy replicationStrategy = appCtx.getReplicationManager().getReplicationStrategy();
final Set<String> masterFiles =
localResourceRepository.getPartitionReplicatedFiles(partition, replicationStrategy).stream()
.map(StoragePathUtil::getFileRelativePath).collect(Collectors.toSet());
- // find files on master and not on replica
- final List<String> replicaMissingFiles =
- masterFiles.stream().filter(file -> !replicaFiles.contains(file)).collect(Collectors.toList());
- replicateMissingFiles(replicaMissingFiles);
- // find files on replica and not on master
- final List<String> replicaInvalidFiles =
- replicaFiles.stream().filter(file -> !masterFiles.contains(file)).collect(Collectors.toList());
- deleteInvalidFiles(replicaInvalidFiles);
+ // exclude from the replica files the list of invalid deleted files
+ final Set<String> replicaFiles = new HashSet<>(replicaResourceResponse.getFiles());
+ replicaFiles.removeAll(deletedReplicaFiles);
+ syncMissingFiles(replicaFiles, masterFiles);
+ deleteReplicaExtraFiles(replicaFiles, masterFiles);
}
- private Set<String> getReplicaFiles(int partition) throws IOException {
- final PartitionResourcesListTask replicaFilesRequest = new PartitionResourcesListTask(partition);
- final ISocketChannel channel = replica.getChannel();
- final ByteBuffer reusableBuffer = replica.getReusableBuffer();
- ReplicationProtocol.sendTo(replica, replicaFilesRequest);
- final PartitionResourcesListResponse response =
- (PartitionResourcesListResponse) ReplicationProtocol.read(channel, reusableBuffer);
- return new HashSet<>(response.getResources());
+ private void deletePartitionFromReplica(int partitionId) throws IOException {
+ DeletePartitionTask deletePartitionTask = new DeletePartitionTask(partitionId);
+ ReplicationProtocol.sendTo(replica, deletePartitionTask);
+ ReplicationProtocol.waitForAck(replica);
+ }
+
+ private void deleteReplicaExtraFiles(Set<String> replicaFiles, Set<String> masterFiles) {
+ final List<String> replicaInvalidFiles =
+ replicaFiles.stream().filter(file -> !masterFiles.contains(file)).collect(Collectors.toList());
+ if (!replicaInvalidFiles.isEmpty()) {
+ LOGGER.debug("deleting files not on current master {} on replica {}", replicaInvalidFiles,
+ replica.getIdentifier());
+ deleteInvalidFiles(replicaInvalidFiles);
+ }
+ }
+
+ private void syncMissingFiles(Set<String> replicaFiles, Set<String> masterFiles) {
+ final List<String> replicaMissingFiles =
+ masterFiles.stream().filter(file -> !replicaFiles.contains(file)).collect(Collectors.toList());
+ if (!replicaMissingFiles.isEmpty()) {
+ LOGGER.debug("replicating missing files {} on replica {}", replicaMissingFiles, replica.getIdentifier());
+ replicateMissingFiles(replicaMissingFiles);
+ }
}
private void replicateMissingFiles(List<String> files) {
final FileSynchronizer sync = new FileSynchronizer(appCtx, replica);
// sort files to ensure index metadata files starting with "." are replicated first
files.sort(String::compareTo);
- files.forEach(sync::replicate);
+ int missingFilesCount = files.size();
+ for (int i = 0; i < missingFilesCount; i++) {
+ String file = files.get(i);
+ sync.replicate(file);
+ replica.setSyncProgress((i + 1d) / missingFilesCount);
+ }
}
private void deleteInvalidFiles(List<String> files) {
final FileSynchronizer sync = new FileSynchronizer(appCtx, replica);
+ // sort files to ensure index metadata files starting with "." are deleted last
+ files.sort(String::compareTo);
+ Collections.reverse(files);
+ LOGGER.info("deleting {}", files);
files.forEach(sync::delete);
}
+
+ private long getResourceMasterValidSeq(ResourceReference rr) throws HyracksDataException {
+ IIndexCheckpointManager iIndexCheckpointManager = appCtx.getIndexCheckpointManagerProvider().get(rr);
+ int checkpointCount = iIndexCheckpointManager.getCheckpointCount();
+ if (checkpointCount > 0) {
+ IndexCheckpoint latest = iIndexCheckpointManager.getLatest();
+ long masterValidSeq = latest.getMasterValidSeq();
+ LOGGER.info("setting resource {} valid component seq to {}", rr, masterValidSeq);
+ return masterValidSeq;
+ }
+ return AbstractLSMIndexFileManager.UNINITIALIZED_COMPONENT_SEQ;
+ }
+
+ private Set<String> cleanupReplicaInvalidResources(PartitionResourcesListResponse replicaResourceResponse,
+ Map<ResourceReference, Long> validReplicaResources) {
+ Set<String> invalidFiles = new HashSet<>();
+ for (String replicaResPath : replicaResourceResponse.getFiles()) {
+ ResourceReference replicaRes = ResourceReference.of(replicaResPath);
+ if (!validReplicaResources.containsKey(replicaRes)) {
+ LOGGER.debug("replica invalid file {} to be deleted", replicaRes.getFileRelativePath());
+ invalidFiles.add(replicaResPath);
+ } else if (replicaResourceResponse.isOwner() && !replicaRes.isMetadataResource()) {
+ // find files where the owner generated and failed before replicating
+ Long masterValidSeq = validReplicaResources.get(replicaRes);
+ IndexComponentFileReference componentFileReference =
+ IndexComponentFileReference.of(replicaRes.getName());
+ if (componentFileReference.getSequenceStart() > masterValidSeq
+ || componentFileReference.getSequenceEnd() > masterValidSeq) {
+ LOGGER.debug("will ask replica {} to delete file {} based on valid master valid seq {}",
+ replica.getIdentifier(), replicaResPath, masterValidSeq);
+ invalidFiles.add(replicaResPath);
+ }
+ }
+ }
+ if (!invalidFiles.isEmpty()) {
+ LOGGER.info("will delete the following files from replica {}", invalidFiles);
+ deleteInvalidFiles(new ArrayList<>(invalidFiles));
+ }
+ return invalidFiles;
+ }
+
+ private PartitionResourcesListResponse getReplicaFiles(int partition) throws IOException {
+ final PartitionResourcesListTask replicaFilesRequest = new PartitionResourcesListTask(partition);
+ final ISocketChannel channel = replica.getChannel();
+ final ByteBuffer reusableBuffer = replica.getReusableBuffer();
+ ReplicationProtocol.sendTo(replica, replicaFilesRequest);
+ return (PartitionResourcesListResponse) ReplicationProtocol.read(channel, reusableBuffer);
+ }
+
+ private Map<ResourceReference, Long> getValidReplicaResources(Map<String, Long> partitionReplicatedResources,
+ boolean owner) throws HyracksDataException {
+ Map<ResourceReference, Long> resource2ValidSeqMap = new HashMap<>();
+ for (Map.Entry<String, Long> resourceEntry : partitionReplicatedResources.entrySet()) {
+ ResourceReference rr = ResourceReference.of(resourceEntry.getKey());
+ final PersistentLocalResourceRepository localResourceRepository =
+ (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+ LocalResource localResource = localResourceRepository.get(rr.getRelativePath().toString());
+ if (localResource != null) {
+ if (localResource.getId() != resourceEntry.getValue()) {
+ LOGGER.info("replica has resource {} but with different resource id; ours {}, theirs {}", rr,
+ localResource.getId(), resourceEntry.getValue());
+ } else {
+ long resourceMasterValidSeq = owner ? getResourceMasterValidSeq(rr) : Integer.MAX_VALUE;
+ resource2ValidSeqMap.put(rr, resourceMasterValidSeq);
+ }
+ }
+ }
+ return resource2ValidSeqMap;
+ }
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
index 261236c..2434686 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
@@ -43,23 +43,26 @@
this.replica = replica;
}
- public void sync() throws IOException {
- synchronized (appCtx.getReplicaManager().getReplicaSyncLock()) {
+ public void sync(boolean register, boolean deltaRecovery) throws IOException {
+ Object partitionLock = appCtx.getReplicaManager().getPartitionSyncLock(replica.getIdentifier().getPartition());
+ synchronized (partitionLock) {
final ICheckpointManager checkpointManager = appCtx.getTransactionSubsystem().getCheckpointManager();
try {
// suspend checkpointing datasets to prevent async IO operations while sync'ing replicas
checkpointManager.suspend();
- syncFiles();
+ syncFiles(deltaRecovery);
checkpointReplicaIndexes();
- appCtx.getReplicationManager().register(replica);
+ if (register) {
+ appCtx.getReplicationManager().register(replica);
+ }
} finally {
checkpointManager.resume();
}
}
}
- private void syncFiles() throws IOException {
- final ReplicaFilesSynchronizer fileSync = new ReplicaFilesSynchronizer(appCtx, replica);
+ private void syncFiles(boolean deltaRecovery) throws IOException {
+ final ReplicaFilesSynchronizer fileSync = new ReplicaFilesSynchronizer(appCtx, replica, deltaRecovery);
// flush replicated dataset to generate disk component for any remaining in-memory components
final IReplicationStrategy replStrategy = appCtx.getReplicationManager().getReplicationStrategy();
appCtx.getDatasetLifecycleManager().flushDataset(replStrategy);
@@ -69,8 +72,10 @@
private void checkpointReplicaIndexes() throws IOException {
final int partition = replica.getIdentifier().getPartition();
+ String masterNode =
+ appCtx.getReplicaManager().isPartitionOwner(partition) ? appCtx.getServiceContext().getNodeId() : null;
CheckpointPartitionIndexesTask task =
- new CheckpointPartitionIndexesTask(partition, getPartitionMaxComponentId(partition));
+ new CheckpointPartitionIndexesTask(partition, getPartitionMaxComponentId(partition), masterNode);
ReplicationProtocol.sendTo(replica, task);
ReplicationProtocol.waitForAck(replica);
}
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 27c05cd..22958ac 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -145,7 +145,6 @@
<dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
- <version>8.3.0</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarFirstElementAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarFirstElementAggregateDescriptor.java
new file mode 100644
index 0000000..22831d7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarFirstElementAggregateDescriptor.java
@@ -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.
+ */
+
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.collections.FirstElementAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarFirstElementAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final FunctionIdentifier FID = BuiltinFunctions.SCALAR_FIRST_ELEMENT;
+
+ public static final IFunctionDescriptorFactory FACTORY = ScalarFirstElementAggregateDescriptor::new;
+
+ private ScalarFirstElementAggregateDescriptor() {
+ super(FirstElementAggregateDescriptor.FACTORY);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..655bc8a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.aggregates.std.SqlUnionMbrAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSqlUnionMbrAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY =
+ createDescriptorFactory(ScalarSqlUnionMbrAggregateDescriptor::new);
+
+ private ScalarSqlUnionMbrAggregateDescriptor() {
+ super(SqlUnionMbrAggregateDescriptor.FACTORY);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SCALAR_SQL_UNION_MBR;
+ }
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ super.setImmutableStates(states);
+ aggFuncDesc.setImmutableStates(getItemType((IAType) states[0]));
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..188508c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarUnionMbrAggregateDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.aggregates.std.UnionMbrAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarUnionMbrAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY =
+ createDescriptorFactory(ScalarUnionMbrAggregateDescriptor::new);
+
+ private ScalarUnionMbrAggregateDescriptor() {
+ super(UnionMbrAggregateDescriptor.FACTORY);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SCALAR_UNION_MBR;
+ }
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ super.setImmutableStates(states);
+ aggFuncDesc.setImmutableStates(getItemType((IAType) states[0]));
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
index e621886..8c7a1ff 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableAvgAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -33,12 +32,7 @@
public class SerializableAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
index 99cd053..a00aa86 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableCountAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -35,12 +34,7 @@
public class SerializableCountAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableCountAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableCountAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
index e7f8845..fbbeecf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableGlobalAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalKurtosisAggregateDescriptor.java
index 5574e54..be3cff9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSkewnessAggregateDescriptor.java
index 5af0f39..430b01d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
index bb47742..9812b10 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlKurtosisAggregateDescriptor.java
index d24f083..b1ed637 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSkewnessAggregateDescriptor.java
index 24eb6d7..bf1eac6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevAggregateDescriptor.java
index 9ec4436..07165d2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevPopAggregateDescriptor.java
index 5ccf863..468d576 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarAggregateDescriptor.java
index 64c3aaf..dedf779 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarPopAggregateDescriptor.java
index 366893e..cad2eea 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalSqlVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalSqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalSqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevAggregateDescriptor.java
index e5fe639..482b12d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevPopAggregateDescriptor.java
index 86b3cb6..fc83077 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarAggregateDescriptor.java
index da5907d..e8ec6dc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableGlobalVarAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarPopAggregateDescriptor.java
index ea5c5c9..8847605 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableGlobalVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableGlobalVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableGlobalVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
index fb95732..3345c46 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateKurtosisAggregateDescriptor.java
index c02789f..beb7a6e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSkewnessAggregateDescriptor.java
index a760b20..747d115 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
index e4c668d..8c52df1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateSqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlKurtosisAggregateDescriptor.java
index 40aaaec..e0c6d03 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,8 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY =
+ SerializableIntermediateSqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSkewnessAggregateDescriptor.java
index e8d1709..8563e8c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,8 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY =
+ SerializableIntermediateSqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevAggregateDescriptor.java
index b2c396d..d5acc277 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateSqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevPopAggregateDescriptor.java
index cc0927e..89b6c8f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,8 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY =
+ SerializableIntermediateSqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarAggregateDescriptor.java
index c8e8bad..52569ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateSqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarPopAggregateDescriptor.java
index 26a466d..78b08f3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateSqlVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateSqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateSqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevAggregateDescriptor.java
index f963275..e53bde9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevPopAggregateDescriptor.java
index 2bcb057..fecf908 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarAggregateDescriptor.java
index e9222b2..04fa186 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarPopAggregateDescriptor.java
index cf2caae..22bd4ad 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableIntermediateVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableIntermediateVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableIntermediateVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableKurtosisAggregateDescriptor.java
index 39bdb79..94cc569 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableKurtosisAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -33,12 +32,7 @@
public class SerializableKurtosisAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
index c14aef7..0300631 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableLocalAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalKurtosisAggregateDescriptor.java
index 76ce6f4..d689d48 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSkewnessAggregateDescriptor.java
index 51d9c1a..0111e87 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
index 4db8418..7636c61 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableLocalSqlAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlKurtosisAggregateDescriptor.java
index 04e783d..08525cd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSkewnessAggregateDescriptor.java
index 2139e6f..f326517 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevAggregateDescriptor.java
index 1efbd50..c5ac171 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevPopAggregateDescriptor.java
index 62cb56a..46236d5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarAggregateDescriptor.java
index 1aa9798..460e762 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableLocalSqlVarAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarPopAggregateDescriptor.java
index 146c92a..b71d4d2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalSqlVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalSqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalSqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevAggregateDescriptor.java
index 1f13577..6c13550 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableLocalStddevAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevPopAggregateDescriptor.java
index a139883..235e646 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarAggregateDescriptor.java
index d328e55..5e0cdd7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableLocalVarAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarPopAggregateDescriptor.java
index e7df873..0acb09c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableLocalVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SerializableLocalVarPopAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableLocalVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableLocalVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSkewnessAggregateDescriptor.java
index d709757..4172571 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSkewnessAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -33,12 +32,7 @@
public class SerializableSkewnessAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
index 9d469b1..5939e49 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlAvgAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
public class SerializableSqlAvgAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
index 1818f07..b4a7620 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlCountAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -35,12 +34,7 @@
public class SerializableSqlCountAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlCountAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlCountAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlKurtosisAggregateDescriptor.java
index 323c26a..bc85ddd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlKurtosisAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
public class SerializableSqlKurtosisAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSkewnessAggregateDescriptor.java
index cf004be..0f72183 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlSkewnessAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
public class SerializableSqlSkewnessAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevAggregateDescriptor.java
index 8bef41a..af34674 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
public class SerializableSqlStddevAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevPopAggregateDescriptor.java
index 7479686..2c6882f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlStddevPopAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarAggregateDescriptor.java
index ae9e617..bf7c2b8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
public class SerializableSqlVarAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarPopAggregateDescriptor.java
index 4d2829e..34fd0da 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableSqlVarPopAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
public class SerializableSqlVarPopAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableSqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableSqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevAggregateDescriptor.java
index ff984f9..7cf0a26 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -33,12 +32,7 @@
public class SerializableStddevAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevPopAggregateDescriptor.java
index 1578acf..1ff7f99 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableStddevPopAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -33,12 +32,7 @@
public class SerializableStddevPopAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarAggregateDescriptor.java
index c014c4f..fb6b41c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -33,12 +32,7 @@
public class SerializableVarAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarPopAggregateDescriptor.java
index 5673539..d3eecbf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/SerializableVarPopAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.serializable.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -33,12 +32,7 @@
public class SerializableVarPopAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SerializableVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SerializableVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..4840f52
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateDescriptor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+
+public abstract class AbstractUnionMbrAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ IAType aggFieldType;
+
+ @Override
+ public void setImmutableStates(Object... types) {
+ aggFieldType = (IAType) types[0];
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateFunction.java
new file mode 100644
index 0000000..9414365
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateFunction.java
@@ -0,0 +1,139 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractUnionMbrAggregateFunction extends AbstractAggregateFunction {
+
+ private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private IPointable inputVal = new VoidPointable();
+ private final IScalarEvaluator eval;
+ protected final IEvaluatorContext context;
+ protected double currentMinX;
+ protected double currentMinY;
+ protected double currentMaxX;
+ protected double currentMaxY;
+
+ protected final AMutablePoint[] aPoint = { new AMutablePoint(0.0, 0.0), new AMutablePoint(0.0, 0.0) };
+ protected final AMutableRectangle aRect = new AMutableRectangle(aPoint[0], aPoint[1]);
+
+ private ISerializerDeserializer<ARectangle> rectangleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE);
+
+ public AbstractUnionMbrAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(sourceLoc);
+ this.eval = args[0].createScalarEvaluator(context);
+ this.context = context;
+ }
+
+ @Override
+ public void init() throws HyracksDataException {
+ // Initialize the resulting mbr coordinates
+ currentMinX = Double.POSITIVE_INFINITY;
+ currentMinY = Double.POSITIVE_INFINITY;
+ currentMaxX = Double.NEGATIVE_INFINITY;
+ currentMaxY = Double.NEGATIVE_INFINITY;
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
+ eval.evaluate(tuple, inputVal);
+ byte[] data = inputVal.getByteArray();
+ int offset = inputVal.getStartOffset();
+ int len = inputVal.getLength();
+ ATypeTag typeTag =
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
+ if (typeTag == ATypeTag.NULL || typeTag == ATypeTag.MISSING) {
+ processNull(typeTag);
+ } else if (typeTag == ATypeTag.RECTANGLE) {
+ double minX = ADoubleSerializerDeserializer.getDouble(data,
+ offset + 1 + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double minY = ADoubleSerializerDeserializer.getDouble(data,
+ offset + 1 + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+ double maxX = ADoubleSerializerDeserializer.getDouble(data,
+ offset + 1 + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double maxY = ADoubleSerializerDeserializer.getDouble(data,
+ offset + 1 + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+ currentMinX = Math.min(currentMinX, minX);
+ currentMinY = Math.min(currentMinY, minY);
+ currentMaxX = Math.max(currentMaxX, maxX);
+ currentMaxY = Math.max(currentMaxY, maxY);
+ }
+ }
+
+ @Override
+ public void finish(IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ try {
+ aPoint[0].setValue(currentMinX, currentMinY);
+ aPoint[1].setValue(currentMaxX, currentMaxY);
+ aRect.setValue(aPoint[0], aPoint[1]);
+ rectangleSerde.serialize(aRect, resultStorage.getDataOutput());
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ }
+
+ @Override
+ public void finishPartial(IPointable result) throws HyracksDataException {
+ if (!isValidCoordinates(currentMinX, currentMinY, currentMaxX, currentMaxY)) {
+ currentMinX = 0.0;
+ currentMinY = 0.0;
+ currentMaxX = 0.0;
+ currentMaxY = 0.0;
+ }
+
+ finish(result);
+ }
+
+ protected void processNull(ATypeTag typeTag) throws UnsupportedItemTypeException {
+ throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.UNION_MBR, typeTag.serialize());
+ }
+
+ private boolean isValidCoordinates(double minX, double minY, double maxX, double maxY) {
+ return (minX != Double.POSITIVE_INFINITY) && (minY != Double.POSITIVE_INFINITY)
+ && (maxX != Double.NEGATIVE_INFINITY) && (maxY != Double.NEGATIVE_INFINITY);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
index a0d06e7..6aa7d73 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class AvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
index 389c3af..8fc8a9f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/CountAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -35,12 +34,7 @@
public class CountAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CountAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CountAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
index bb15b2d..a1f216d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalKurtosisAggregateDescriptor.java
index 107f823..7136c19 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSkewnessAggregateDescriptor.java
index 0d3038a..fdb5673 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
index dcbc1a7..677eb9c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlKurtosisAggregateDescriptor.java
index d4e24bb..d4a3fb7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSkewnessAggregateDescriptor.java
index 2874c60..f63c256 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevAggregateDescriptor.java
index d81d435..28a090a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevPopAggregateDescriptor.java
index aeef3d0..1cbfb15 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..218ae39
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class GlobalSqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY =
+ AbstractAggregateFunctionDynamicDescriptor.createFactory(GlobalSqlUnionMbrAggregateDescriptor::new);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.GLOBAL_SQL_UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarAggregateDescriptor.java
index 5dcaa03..e96cbe8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarPopAggregateDescriptor.java
index afd6f66..da8c1b6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalSqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalSqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevAggregateDescriptor.java
index c4d1f38..8a6c90f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevPopAggregateDescriptor.java
index 52df8c2..7029ed8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..5b66bf4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class GlobalUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY =
+ AbstractAggregateFunctionDynamicDescriptor.createFactory(GlobalUnionMbrAggregateDescriptor::new);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.GLOBAL_UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarAggregateDescriptor.java
index 4ade667..2d6087b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarPopAggregateDescriptor.java
index 3a8a818..f1225b6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GlobalVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GlobalVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
index 8c34eec..2f76d7c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateKurtosisAggregateDescriptor.java
index 5a62762..67b631f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSkewnessAggregateDescriptor.java
index 9cd0b21..d0e2537 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
index 54f22ab..535b442 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class IntermediateSqlAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlKurtosisAggregateDescriptor.java
index 02b65f2..2ea358d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class IntermediateSqlKurtosisAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSkewnessAggregateDescriptor.java
index a8916f8..02d6f63 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class IntermediateSqlSkewnessAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevAggregateDescriptor.java
index 62663a4..0d3ea22 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class IntermediateSqlStddevAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevPopAggregateDescriptor.java
index d05e865..6196d5a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class IntermediateSqlStddevPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..52a517b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntermediateSqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY =
+ AbstractAggregateFunctionDynamicDescriptor.createFactory(IntermediateSqlUnionMbrAggregateDescriptor::new);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.INTERMEDIATE_SQL_UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarAggregateDescriptor.java
index c726870..00f222f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class IntermediateSqlVarAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarPopAggregateDescriptor.java
index 3ac9bff..4abf219 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class IntermediateSqlVarPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateSqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateSqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevAggregateDescriptor.java
index b0160b4..2fa255e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevPopAggregateDescriptor.java
index 6409dfc..c83b917 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..0d66149
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntermediateUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY =
+ AbstractAggregateFunctionDynamicDescriptor.createFactory(IntermediateUnionMbrAggregateDescriptor::new);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.INTERMEDIATE_UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarAggregateDescriptor.java
index 003c4e5..1a74d69 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarPopAggregateDescriptor.java
index 4dc8b2e..cbf2434 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntermediateVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntermediateVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/KurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/KurtosisAggregateDescriptor.java
index af35c1a..b28f88f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/KurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/KurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class KurtosisAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new KurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = KurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
index 2bb25f9..d9a132c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalKurtosisAggregateDescriptor.java
index 3018bcc..6366af8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalKurtosisAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSkewnessAggregateDescriptor.java
index b23402b..733db0e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSkewnessAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
index f3288c7..bff5a74 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSqlAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlKurtosisAggregateDescriptor.java
index c586eb3..081d241 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSqlKurtosisAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSkewnessAggregateDescriptor.java
index 366ff89d..cc68a43 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSqlSkewnessAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevAggregateDescriptor.java
index cb8dbed..5eb311d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSqlStddevAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevPopAggregateDescriptor.java
index ed60e90..c9bf62f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSqlStddevPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..3ee1e47
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LocalSqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY =
+ AbstractAggregateFunctionDynamicDescriptor.createFactory(LocalSqlUnionMbrAggregateDescriptor::new);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.LOCAL_SQL_UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarAggregateDescriptor.java
index fa0052e..98aadc7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSqlVarAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarPopAggregateDescriptor.java
index f8f27a3..305defa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalSqlVarPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalSqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalSqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevAggregateDescriptor.java
index 8d3509f..4e85e66 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalStddevAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevPopAggregateDescriptor.java
index cc17a33..29febfa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalStddevPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..0b5cbdd
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LocalUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY =
+ AbstractAggregateFunctionDynamicDescriptor.createFactory(LocalUnionMbrAggregateDescriptor::new);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.LOCAL_UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarAggregateDescriptor.java
index 288f8f9..bdb4753 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalVarAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarPopAggregateDescriptor.java
index a0fedbe..495b1ef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class LocalVarPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LocalVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LocalVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SkewnessAggregateDescriptor.java
index 98eef28..e762c42 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SkewnessAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
index da10cf6..213c708 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlAvgAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SqlAvgAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlAvgAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlAvgAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
index 1155bd5..f10dcc9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlCountAggregateDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -35,12 +34,7 @@
public class SqlCountAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlCountAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlCountAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlKurtosisAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlKurtosisAggregateDescriptor.java
index 7429542..53f5a1d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlKurtosisAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlKurtosisAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SqlKurtosisAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlKurtosisAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlKurtosisAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSkewnessAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSkewnessAggregateDescriptor.java
index 885608e..e17ec66 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSkewnessAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlSkewnessAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SqlSkewnessAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlSkewnessAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlSkewnessAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevAggregateDescriptor.java
index 360cdf3..6e29775 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SqlStddevAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlStddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlStddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevPopAggregateDescriptor.java
index 6f896b4..4424857 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlStddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SqlStddevPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlStddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlStddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..2dd125e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY =
+ AbstractAggregateFunctionDynamicDescriptor.createFactory(SqlUnionMbrAggregateDescriptor::new);
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SQL_UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateFunction.java
new file mode 100644
index 0000000..c101a69
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateFunction.java
@@ -0,0 +1,31 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class SqlUnionMbrAggregateFunction extends AbstractUnionMbrAggregateFunction {
+ public SqlUnionMbrAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(args, context, sourceLoc);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarAggregateDescriptor.java
index deefa55..1f9c0b7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SqlVarAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlVarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlVarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarPopAggregateDescriptor.java
index c24bf94..d7eb353 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlVarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class SqlVarPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SqlVarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SqlVarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevAggregateDescriptor.java
index 011efcc..ef3d9f9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class StddevAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StddevAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StddevAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevPopAggregateDescriptor.java
index 8fba9e5..ea6f209 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/StddevPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class StddevPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StddevPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StddevPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..33aa766
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class UnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = () -> new UnionMbrAggregateDescriptor();
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.UNION_MBR;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateFunction.java
new file mode 100644
index 0000000..e0fc3e7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateFunction.java
@@ -0,0 +1,32 @@
+/*
+ * 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.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class UnionMbrAggregateFunction extends AbstractUnionMbrAggregateFunction {
+
+ public UnionMbrAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(args, context, sourceLoc);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarAggregateDescriptor.java
index 37f0ee6..23dac92 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class VarAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new VarAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = VarAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarPopAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarPopAggregateDescriptor.java
index 0eeb7b3..45051d4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarPopAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/VarPopAggregateDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.aggregates.std;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class VarPopAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new VarPopAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = VarPopAggregateDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
index dead68b..40abd72 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
@@ -42,12 +41,7 @@
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.EMPTY_STREAM;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new EmptyStreamAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = EmptyStreamAggregateDescriptor::new;
@Override
public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
index d28c7c2..7bc366e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/stream/NonEmptyStreamAggregateDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
@@ -41,12 +40,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NonEmptyStreamAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NonEmptyStreamAggregateDescriptor::new;
@Override
public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
index 23795dc..1a3e81b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.APoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,13 +51,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.GET_CIRCLE_CENTER_ACCESSOR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CircleCenterAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CircleCenterAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
index 39484ae..e653813 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,13 +49,7 @@
public class CircleRadiusAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.GET_CIRCLE_RADIUS_ACCESSOR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CircleRadiusAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CircleRadiusAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
index a7f8500..03a63cc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
@@ -33,7 +33,6 @@
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.APoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
@@ -57,13 +56,7 @@
public class LineRectanglePolygonAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LineRectanglePolygonAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LineRectanglePolygonAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
index 6bcb936..2167f22 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -51,13 +50,7 @@
public class PointXCoordinateAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.GET_POINT_X_COORDINATE_ACCESSOR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PointXCoordinateAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = PointXCoordinateAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
index 3c32e51..e1b8e83 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -54,13 +53,7 @@
private static final FunctionIdentifier FID = BuiltinFunctions.GET_POINT_Y_COORDINATE_ACCESSOR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PointYCoordinateAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = PointYCoordinateAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
index 9ea0954..aa09e2d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalDayAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -54,13 +53,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_DAY;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalDayAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalDayAccessor::new;
/* (non-Javadoc)
* @see org.apache.asterix.om.function.IFunctionDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
index a53f35e..77485ae 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalHourAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -54,14 +53,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_HOUR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalHourAccessor();
- }
-
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalHourAccessor::new;
/* (non-Javadoc)
* @see org.apache.asterix.om.function.IFunctionDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
index 1dcb45b..6dd9efe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableTime;
import org.apache.asterix.om.base.ATime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -55,13 +54,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_END;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalEndAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalEndAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
index bc9b393..5afcd9c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDateAccessor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.ADate;
import org.apache.asterix.om.base.AMutableDate;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,13 +49,7 @@
public class TemporalIntervalEndDateAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_END_DATE;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalEndDateAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalEndDateAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
index 3faef26..cb060fc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndDatetimeAccessor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.ADateTime;
import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,13 +51,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_END_DATETIME;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalEndDatetimeAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalEndDatetimeAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
index ed1962c..05833b3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalEndTimeAccessor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableTime;
import org.apache.asterix.om.base.ATime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,13 +49,7 @@
public class TemporalIntervalEndTimeAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_END_TIME;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalEndTimeAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalEndTimeAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
index e2df591..74fdc76 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableTime;
import org.apache.asterix.om.base.ATime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -55,13 +54,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_START;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalStartAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalStartAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
index 57e2730..f0f07ef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDateAccessor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.ADate;
import org.apache.asterix.om.base.AMutableDate;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -53,13 +52,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_START_DATE;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalStartDateAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalStartDateAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
index be6787f..d5aa67d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartDatetimeAccessor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.ADateTime;
import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,13 +49,7 @@
public class TemporalIntervalStartDatetimeAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_START_DATETIME;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalStartDatetimeAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalStartDatetimeAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
index ee86f64..43b1757 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalIntervalStartTimeAccessor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableTime;
import org.apache.asterix.om.base.ATime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -51,13 +50,7 @@
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_START_TIME;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalIntervalStartTimeAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalIntervalStartTimeAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
index 08e1484..78fac67 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMillisecondAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -53,14 +52,7 @@
public class TemporalMillisecondAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_MILLISEC;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalMillisecondAccessor();
- }
-
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalMillisecondAccessor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
index 512b82e..987a0b9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMinuteAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -56,14 +55,7 @@
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_MIN;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalMinuteAccessor();
- }
-
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalMinuteAccessor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
index 0dde65c..de2f8ac 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalMonthAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -53,14 +52,7 @@
public class TemporalMonthAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_MONTH;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalMonthAccessor();
- }
-
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalMonthAccessor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
index fcd952d..2aae1fb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalSecondAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -53,14 +52,7 @@
public class TemporalSecondAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_SEC;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalSecondAccessor();
- }
-
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalSecondAccessor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
index be3615f..182b2f4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/accessors/TemporalYearAccessor.java
@@ -31,7 +31,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -55,13 +54,7 @@
public class TemporalYearAccessor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
private static final FunctionIdentifier FID = BuiltinFunctions.ACCESSOR_TEMPORAL_YEAR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TemporalYearAccessor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TemporalYearAccessor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
index e834522..67d4808 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/ListAccessor.java
@@ -98,6 +98,10 @@
}
}
+ public ATypeTag getItemTypeAt(int itemIndex) throws HyracksDataException {
+ return getItemType(getItemOffset(itemIndex));
+ }
+
public void writeItem(int itemIndex, DataOutput dos) throws IOException {
int itemOffset = getItemOffset(itemIndex);
int itemLength = getItemLength(itemOffset);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/NumberUtils.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/NumberUtils.java
index 5fd7892..d4cce79 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/NumberUtils.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/NumberUtils.java
@@ -20,8 +20,14 @@
package org.apache.asterix.runtime.evaluators.common;
import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.base.AMutableInt16;
+import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.commons.lang3.mutable.MutableBoolean;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.util.StringUtil;
/**
* Utility methods for number handling
@@ -66,36 +72,77 @@
}
/**
- * Parses string as bigint
+ * Parses string as float
* @param textPtr input string
* @param result placeholder for the result
* @return {@code true} if parsing was successful, {@code false} otherwise
*/
- public static boolean parseInt64(UTF8StringPointable textPtr, AMutableInt64 result) {
+ public static boolean parseFloat(UTF8StringPointable textPtr, AMutableFloat result) {
+ float v;
+ if (POSITIVE_INF.compareTo(textPtr) == 0) {
+ v = Float.POSITIVE_INFINITY;
+ } else if (NEGATIVE_INF.compareTo(textPtr) == 0) {
+ v = Float.NEGATIVE_INFINITY;
+ } else if (NAN.compareTo(textPtr) == 0) {
+ v = Float.NaN;
+ } else {
+ try {
+ v = Float.parseFloat(textPtr.toString());
+ } catch (NumberFormatException e) {
+ return false;
+ }
+ }
+ result.setValue(v);
+ return true;
+ }
+
+ /**
+ * Parses string as bigint
+ * @param textPtr input string
+ * @param result placeholder for the result
+ * @param maybeNumeric if parsing was unsuccessful indicates whether the input string might
+ * contain a non-integer numeric value
+ * @return {@code true} if parsing was successful, {@code false} otherwise
+ */
+ public static boolean parseInt64(UTF8StringPointable textPtr, AMutableInt64 result, MutableBoolean maybeNumeric) {
byte[] bytes = textPtr.getByteArray();
int offset = textPtr.getCharStartOffset();
+ int end = textPtr.getStartOffset() + textPtr.getLength();
+ return parseInt64(bytes, offset, end, StringUtil.getByteArrayAsCharAccessor(), result, maybeNumeric);
+ }
+
+ public static <T> boolean parseInt64(T input, int begin, int end, StringUtil.ICharAccessor<T> charAccessor,
+ AMutableInt64 result, MutableBoolean maybeNumeric) {
+ if (maybeNumeric != null) {
+ maybeNumeric.setFalse();
+ }
+ int offset = begin;
//accumulating value in negative domain
//otherwise Long.MIN_VALUE = -(Long.MAX_VALUE + 1) would have caused overflow
long value = 0;
boolean positive = true;
long limit = -Long.MAX_VALUE;
- if (bytes[offset] == '+') {
+ char c = charAccessor.charAt(input, offset);
+ if (c == '+') {
offset++;
- } else if (bytes[offset] == '-') {
+ } else if (c == '-') {
offset++;
positive = false;
limit = Long.MIN_VALUE;
}
- int end = textPtr.getStartOffset() + textPtr.getLength();
for (; offset < end; offset++) {
int digit;
- if (bytes[offset] >= '0' && bytes[offset] <= '9') {
+ c = charAccessor.charAt(input, offset);
+ if (c >= '0' && c <= '9') {
value *= 10;
- digit = bytes[offset] - '0';
- } else if (bytes[offset] == 'i' && bytes[offset + 1] == '6' && bytes[offset + 2] == '4'
- && offset + 3 == end) {
+ digit = c - '0';
+ } else if (c == 'i' && charAccessor.charAt(input, offset + 1) == '6'
+ && charAccessor.charAt(input, offset + 2) == '4' && offset + 3 == end) {
break;
} else {
+ if (maybeNumeric != null) {
+ maybeNumeric.setValue(isNumericNonDigitOrSignChar(c));
+ }
return false;
}
if (value < limit + digit) {
@@ -109,7 +156,173 @@
if (value < 0 && positive) {
value *= -1;
}
+ result.setValue(value);
+ return true;
+ }
+ /**
+ * Parses string as integer
+ * @param textPtr input string
+ * @param result placeholder for the result
+ * @param maybeNumeric if parsing was unsuccessful indicates whether the input string might
+ * contain a non-integer numeric value
+ * @return {@code true} if parsing was successful, {@code false} otherwise
+ */
+ public static boolean parseInt32(UTF8StringPointable textPtr, AMutableInt32 result, MutableBoolean maybeNumeric) {
+ if (maybeNumeric != null) {
+ maybeNumeric.setFalse();
+ }
+ byte[] bytes = textPtr.getByteArray();
+ int offset = textPtr.getCharStartOffset();
+ //accumulating value in negative domain
+ //otherwise Integer.MIN_VALUE = -(Integer.MAX_VALUE + 1) would have caused overflow
+ int value = 0;
+ boolean positive = true;
+ int limit = -Integer.MAX_VALUE;
+ if (bytes[offset] == '+') {
+ offset++;
+ } else if (bytes[offset] == '-') {
+ offset++;
+ positive = false;
+ limit = Integer.MIN_VALUE;
+ }
+ int end = textPtr.getStartOffset() + textPtr.getLength();
+ for (; offset < end; offset++) {
+ int digit;
+ if (bytes[offset] >= '0' && bytes[offset] <= '9') {
+ value *= 10;
+ digit = bytes[offset] - '0';
+ } else if (bytes[offset] == 'i' && bytes[offset + 1] == '3' && bytes[offset + 2] == '2'
+ && offset + 3 == end) {
+ break;
+ } else {
+ if (maybeNumeric != null) {
+ maybeNumeric.setValue(isNumericNonDigitOrSignChar(bytes[offset]));
+ }
+ return false;
+ }
+ if (value < limit + digit) {
+ return false;
+ }
+ value -= digit;
+ }
+ if (value > 0) {
+ return false;
+ }
+ if (value < 0 && positive) {
+ value *= -1;
+ }
+ result.setValue(value);
+ return true;
+ }
+
+ /**
+ * Parses string as smallint
+ * @param textPtr input string
+ * @param result placeholder for the result
+ * @param maybeNumeric if parsing was unsuccessful indicates whether the input string might
+ * contain a non-integer numeric value
+ * @return {@code true} if parsing was successful, {@code false} otherwise
+ */
+ public static boolean parseInt16(UTF8StringPointable textPtr, AMutableInt16 result, MutableBoolean maybeNumeric) {
+ if (maybeNumeric != null) {
+ maybeNumeric.setFalse();
+ }
+ byte[] bytes = textPtr.getByteArray();
+ int offset = textPtr.getCharStartOffset();
+ //accumulating value in negative domain
+ //otherwise Short.MIN_VALUE = -(Short.MAX_VALUE + 1) would have caused overflow
+ short value = 0;
+ boolean positive = true;
+ short limit = -Short.MAX_VALUE;
+ if (bytes[offset] == '+') {
+ offset++;
+ } else if (bytes[offset] == '-') {
+ offset++;
+ positive = false;
+ limit = Short.MIN_VALUE;
+ }
+ int end = textPtr.getStartOffset() + textPtr.getLength();
+ for (; offset < end; offset++) {
+ int digit;
+ if (bytes[offset] >= '0' && bytes[offset] <= '9') {
+ value = (short) (value * 10);
+ digit = bytes[offset] - '0';
+ } else if (bytes[offset] == 'i' && bytes[offset + 1] == '1' && bytes[offset + 2] == '6'
+ && offset + 3 == end) {
+ break;
+ } else {
+ if (maybeNumeric != null) {
+ maybeNumeric.setValue(isNumericNonDigitOrSignChar(bytes[offset]));
+ }
+ return false;
+ }
+ if (value < limit + digit) {
+ return false;
+ }
+ value = (short) (value - digit);
+ }
+ if (value > 0) {
+ return false;
+ }
+ if (value < 0 && positive) {
+ value *= -1;
+ }
+ result.setValue(value);
+ return true;
+ }
+
+ /**
+ * Parses string as tinyint
+ * @param textPtr input string
+ * @param result placeholder for the result
+ * @param maybeNumeric if parsing was unsuccessful indicates whether the input string might
+ * contain a non-integer numeric value
+ * @return {@code true} if parsing was successful, {@code false} otherwise
+ */
+ public static boolean parseInt8(UTF8StringPointable textPtr, AMutableInt8 result, MutableBoolean maybeNumeric) {
+ if (maybeNumeric != null) {
+ maybeNumeric.setFalse();
+ }
+ byte[] bytes = textPtr.getByteArray();
+ int offset = textPtr.getCharStartOffset();
+ //accumulating value in negative domain
+ //otherwise Byte.MIN_VALUE = -(Byte.MAX_VALUE + 1) would have caused overflow
+ byte value = 0;
+ boolean positive = true;
+ byte limit = -Byte.MAX_VALUE;
+ if (bytes[offset] == '+') {
+ offset++;
+ } else if (bytes[offset] == '-') {
+ offset++;
+ positive = false;
+ limit = Byte.MIN_VALUE;
+ }
+ int end = textPtr.getStartOffset() + textPtr.getLength();
+ for (; offset < end; offset++) {
+ int digit;
+ if (bytes[offset] >= '0' && bytes[offset] <= '9') {
+ value = (byte) (value * 10);
+ digit = bytes[offset] - '0';
+ } else if (bytes[offset] == 'i' && bytes[offset + 1] == '8' && offset + 2 == end) {
+ break;
+ } else {
+ if (maybeNumeric != null) {
+ maybeNumeric.setValue(isNumericNonDigitOrSignChar(bytes[offset]));
+ }
+ return false;
+ }
+ if (value < limit + digit) {
+ return false;
+ }
+ value = (byte) (value - digit);
+ }
+ if (value > 0) {
+ return false;
+ }
+ if (value < 0 && positive) {
+ value *= -1;
+ }
result.setValue(value);
return true;
}
@@ -161,4 +374,21 @@
return false;
}
}
+
+ private static boolean isNumericNonDigitOrSignChar(char v) {
+ switch (v) {
+ case '.':
+ case 'E':
+ case 'e':
+ case 'I': // INF
+ case 'N': // NaN
+ return true;
+ default:
+ return false;
+ }
+ }
+
+ private static boolean isNumericNonDigitOrSignChar(byte v) {
+ return isNumericNonDigitOrSignChar((char) v);
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java
index 7eb7358..192cf95 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java
@@ -21,6 +21,7 @@
import org.apache.asterix.dataflow.data.nontagged.Coordinate;
import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import org.apache.asterix.om.base.ARectangle;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class SpatialUtils {
@@ -177,4 +178,18 @@
public static void setTriangleYCoordinate(DoubleArray trianglesY, int triangleId, int point, double value) {
trianglesY.get()[triangleId * 3 + point] = value;
}
+
+ public static boolean intersects(ARectangle rect1, ARectangle rect2) {
+ // If one rectangle is on left side of other
+ if ((rect1.getP1().getX() > rect2.getP2().getX()) || (rect2.getP1().getX() > rect1.getP2().getX())) {
+ return false;
+ }
+
+ // If one rectangle is above other
+ if ((rect1.getP1().getY() > rect2.getP2().getY()) || (rect2.getP1().getY() > rect1.getP2().getY())) {
+ return false;
+ }
+
+ return true;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
index d2f7665..479d603 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringConstructorDescriptor.java
@@ -19,8 +19,8 @@
package org.apache.asterix.runtime.evaluators.constructors;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.base.AMutableBinary;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -28,17 +28,14 @@
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.parsers.ByteArrayBase64ParserFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.util.bytes.Base64Parser;
@MissingNullInOutFunction
public class ABinaryBase64StringConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ABinaryBase64StringConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ABinaryBase64StringConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -47,8 +44,7 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new ABinaryHexStringConstructorDescriptor.ABinaryConstructorEvaluator(args[0],
- ByteArrayBase64ParserFactory.INSTANCE, ctx, sourceLoc);
+ return new ABinaryBase64StringConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc);
}
};
}
@@ -57,4 +53,31 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.BINARY_BASE64_CONSTRUCTOR;
}
+
+ protected static class ABinaryBase64StringConstructorEvaluator extends AbstractBinaryConstructorEvaluator {
+
+ private final Base64Parser parser = new Base64Parser();
+
+ protected ABinaryBase64StringConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected boolean parseBinary(UTF8StringPointable textPtr, AMutableBinary result) {
+ try {
+ parser.generatePureByteArrayFromBase64String(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+ textPtr.getUTF8Length());
+ } catch (IllegalArgumentException e) {
+ return false;
+ }
+ result.setValue(parser.getByteArray(), 0, parser.getLength());
+ return true;
+ }
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.BINARY_BASE64_CONSTRUCTOR;
+ }
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..02a03d3
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryBase64StringDefaultNullConstructorDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ABinaryBase64StringDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ABinaryBase64StringDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new ABinaryBase64StringConstructorDescriptor.ABinaryBase64StringConstructorEvaluator(ctx,
+ args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ABinaryBase64StringDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
index 505b650..404258a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
@@ -19,42 +19,23 @@
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.base.AMutableBinary;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import org.apache.hyracks.dataflow.common.data.parsers.ByteArrayHexParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParser;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import org.apache.hyracks.util.bytes.HexParser;
@MissingNullInOutFunction
public class ABinaryHexStringConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ABinaryHexStringConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ABinaryHexStringConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -63,7 +44,26 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new ABinaryConstructorEvaluator(args[0], ByteArrayHexParserFactory.INSTANCE, ctx, sourceLoc);
+ return new AbstractBinaryConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+ private final HexParser parser = new HexParser();
+
+ @Override
+ protected boolean parseBinary(UTF8StringPointable textPtr, AMutableBinary result) {
+ try {
+ parser.generateByteArrayFromHexString(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+ textPtr.getUTF8Length());
+ } catch (IllegalArgumentException e) {
+ return false;
+ }
+ result.setValue(parser.getByteArray(), 0, parser.getLength());
+ return true;
+ }
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ABinaryHexStringConstructorDescriptor.this.getIdentifier();
+ }
+ };
}
};
}
@@ -72,59 +72,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.BINARY_HEX_CONSTRUCTOR;
}
-
- static class ABinaryConstructorEvaluator implements IScalarEvaluator {
- private final SourceLocation sourceLoc;
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private final DataOutput out = resultStorage.getDataOutput();
- private final IPointable inputArg = new VoidPointable();
- private final IScalarEvaluator eval;
- private IValueParser byteArrayParser;
- private UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
- public ABinaryConstructorEvaluator(IScalarEvaluatorFactory copyEvaluatorFactory,
- IValueParserFactory valueParserFactory, IEvaluatorContext context, SourceLocation sourceLoc)
- throws HyracksDataException {
- this.sourceLoc = sourceLoc;
- eval = copyEvaluatorFactory.createScalarEvaluator(context);
- byteArrayParser = valueParserFactory.createValueParser();
- }
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] binary = inputArg.getByteArray();
- int startOffset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = binary[startOffset];
- if (tt == ATypeTag.SERIALIZED_BINARY_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(inputArg.getByteArray(), startOffset + 1, len - 1);
- char[] buffer = utf8Ptr.toString().toCharArray();
- out.write(ATypeTag.BINARY.serialize());
- if (!byteArrayParser.parse(buffer, 0, buffer.length, out)) {
- PointableHelper.setNull(result);
- return;
- }
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, 0, tt,
- ATypeTag.SERIALIZED_BINARY_TYPE_TAG, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, e,
- ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
- }
- }
- }
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
index b08ae46..be5e3f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanConstructorDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
@MissingNullInOutFunction
public class ABooleanConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ABooleanConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ABooleanConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -46,7 +40,7 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
+ return new AbstractBooleanConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
protected FunctionIdentifier getIdentifier() {
return ABooleanConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..1bec6a7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABooleanDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ABooleanDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ABooleanDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractBooleanConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ABooleanDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.BOOLEAN_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
index 58daa47..5891798 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ACircleConstructorDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableCircle;
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -51,12 +50,7 @@
public class ACircleConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ACircleConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ACircleConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
index 90ec400..b9a4590 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorDescriptor.java
@@ -18,45 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADate;
-import org.apache.asterix.om.base.AMutableDate;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class ADateConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ADateConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ADateConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -65,73 +40,10 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableDate aDate = new AMutableDate(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADate> dateSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
-
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
+ return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- resultStorage.reset();
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[offset];
- if (tt == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- utf8Ptr.set(serString, offset + 1, len - 1);
- int stringLength = utf8Ptr.getUTF8Length();
-
- // the string to be parsed should be at least 8 characters: YYYYMMDD
- if (stringLength < 8) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_DATE_TYPE_TAG);
- }
-
- int startOffset = utf8Ptr.getCharStartOffset();
- while (serString[startOffset] == ' ') {
- startOffset++;
- }
- int endOffset = startOffset + stringLength - 1;
- while (serString[endOffset] == ' ') {
- endOffset--;
- }
-
- long chrononTimeInMs = ADateParserFactory.parseDatePart(serString, startOffset,
- endOffset - startOffset + 1);
- short temp = 0;
- if (chrononTimeInMs < 0
- && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
- temp = 1;
- }
- aDate.setValue((int) (chrononTimeInMs / GregorianCalendarSystem.CHRONON_OF_DAY) - temp);
- dateSerde.serialize(aDate, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_DATE_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return ADateConstructorDescriptor.this.getIdentifier();
}
};
}
@@ -142,5 +54,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.DATE_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..29cb552
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateConstructorWithFormatDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public class ADateConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADateConstructorWithFormatDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDateConstructorWithFormatEvaluator(ctx, args, sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADateConstructorWithFormatDescriptor.this.getIdentifier();
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DATE_CONSTRUCTOR_WITH_FORMAT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..da7db6b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ADateDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADateDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDateConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADateDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateDefaultNullConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateDefaultNullConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..e1ac605
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateDefaultNullConstructorWithFormatDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ADateDefaultNullConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADateDefaultNullConstructorWithFormatDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDateConstructorWithFormatEvaluator(ctx, args, sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADateDefaultNullConstructorWithFormatDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg, formatArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DATE_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
index 8d6d151..025281c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorDescriptor.java
@@ -18,45 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
-import org.apache.asterix.om.base.temporal.ADateParserFactory;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class ADateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ADateTimeConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ADateTimeConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -65,73 +40,10 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableDateTime aDateTime = new AMutableDateTime(0L);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADateTime> datetimeSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
+ return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- resultStorage.reset();
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[offset];
- if (tt == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- utf8Ptr.set(serString, offset + 1, len - 1);
- int stringLength = utf8Ptr.getUTF8Length();
- int startOffset = utf8Ptr.getCharStartOffset();
- // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
- if (stringLength < 14) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
- }
- // +1 if it is negative (-)
- short timeOffset = (short) ((serString[startOffset] == '-') ? 1 : 0);
-
- timeOffset += 8;
-
- if (serString[startOffset + timeOffset] != 'T') {
- timeOffset += 2;
- if (serString[startOffset + timeOffset] != 'T') {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
- }
- }
-
- long chrononTimeInMs =
- ADateParserFactory.parseDatePart(serString, startOffset, timeOffset);
-
- chrononTimeInMs += ATimeParserFactory.parseTimePart(serString,
- startOffset + timeOffset + 1, stringLength - timeOffset - 1);
-
- aDateTime.setValue(chrononTimeInMs);
- datetimeSerde.serialize(aDateTime, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return ADateTimeConstructorDescriptor.this.getIdentifier();
}
};
}
@@ -142,5 +54,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.DATETIME_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..af8bba4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ADateTimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADateTimeConstructorWithFormatDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDateTimeConstructorWithFormatEvaluator(ctx, args, sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADateTimeConstructorWithFormatDescriptor.this.getIdentifier();
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DATETIME_CONSTRUCTOR_WITH_FORMAT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..fd3ac96
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ADateTimeDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADateTimeDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDateTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADateTimeDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeDefaultNullConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeDefaultNullConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..04810f6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADateTimeDefaultNullConstructorWithFormatDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ADateTimeDefaultNullConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADateTimeDefaultNullConstructorWithFormatDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDateTimeConstructorWithFormatEvaluator(ctx, args, sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADateTimeDefaultNullConstructorWithFormatDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg, formatArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DATETIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
index fde5c23..a961bf9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
@@ -18,46 +18,21 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADayTimeDuration;
-import org.apache.asterix.om.base.AMutableDayTimeDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class ADayTimeDurationConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ADayTimeDurationConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ADayTimeDurationConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -66,67 +41,20 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde =
- SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADAYTIMEDURATION);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractDayTimeDurationConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx),
+ sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[offset];
- if (tt == ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, offset + 1, len - 1);
- int stringLength = utf8Ptr.getUTF8Length();
- int startOffset = utf8Ptr.getCharStartOffset();
-
- ADurationParserFactory.parseDuration(serString, startOffset, stringLength,
- aDayTimeDuration, ADurationParseOption.DAY_TIME);
-
- dayTimeDurationSerde.serialize(aDayTimeDuration, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (Exception e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return ADayTimeDurationConstructorDescriptor.this.getIdentifier();
}
-
};
}
};
}
- /* (non-Javadoc)
- * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
- */
@Override
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.DAY_TIME_DURATION_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..d17c80b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADayTimeDurationDefaultNullConstructorDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ADayTimeDurationDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADayTimeDurationDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDayTimeDurationConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx),
+ sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADayTimeDurationDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DAY_TIME_DURATION_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
index 700fbfe..c85cff3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleConstructorDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
@MissingNullInOutFunction
public class ADoubleConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ADoubleConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ADoubleConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -46,7 +40,7 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
+ return new AbstractDoubleConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
protected FunctionIdentifier getIdentifier() {
return ADoubleConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..578be20
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADoubleDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ADoubleDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADoubleDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDoubleConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADoubleDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DOUBLE_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
index d87ce13..67742c2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
@@ -18,45 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADuration;
-import org.apache.asterix.om.base.AMutableDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class ADurationConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ADurationConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ADurationConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -65,49 +40,11 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableDuration aDuration = new AMutableDuration(0, 0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADuration> durationSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADURATION);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractDurationConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[offset];
- if (tt == ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, offset + 1, len - 1);
- int stringLength = utf8Ptr.getUTF8Length();
- ADurationParserFactory.parseDuration(serString, utf8Ptr.getCharStartOffset(),
- stringLength, aDuration, ADurationParseOption.All);
- durationSerde.serialize(aDuration, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return ADurationConstructorDescriptor.this.getIdentifier();
}
};
}
@@ -118,5 +55,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.DURATION_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..429fa0a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ADurationDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ADurationDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ADurationDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractDurationConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ADurationDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.DURATION_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
index 05abb30..ccd71b9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatConstructorDescriptor.java
@@ -18,44 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AFloat;
-import org.apache.asterix.om.base.AMutableFloat;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.NumberUtils;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class AFloatConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AFloatConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AFloatConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -64,57 +40,11 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableFloat aFloat = new AMutableFloat(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AFloat> floatSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractFloatConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[offset];
- if (tt == ATypeTag.SERIALIZED_FLOAT_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- int utf8offset = offset + 1;
- int utf8len = len - 1;
- if (NumberUtils.POSITIVE_INF.compareTo(serString, utf8offset, utf8len) == 0) {
- aFloat.setValue(Float.POSITIVE_INFINITY);
- } else if (NumberUtils.NEGATIVE_INF.compareTo(serString, utf8offset, utf8len) == 0) {
- aFloat.setValue(Float.NEGATIVE_INFINITY);
- } else if (NumberUtils.NAN.compareTo(serString, utf8offset, utf8len) == 0) {
- aFloat.setValue(Float.NaN);
- } else {
- utf8Ptr.set(serString, utf8offset, utf8len);
- aFloat.setValue(Float.parseFloat(utf8Ptr.toString()));
- }
- floatSerde.serialize(aFloat, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return AFloatConstructorDescriptor.this.getIdentifier();
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..09a1e5a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AFloatDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AFloatDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AFloatDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractFloatConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return AFloatDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.FLOAT_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
index a55db9f..87f377e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16ConstructorDescriptor.java
@@ -18,44 +18,21 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AInt16;
-import org.apache.asterix.om.base.AMutableInt16;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class AInt16ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AInt16ConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AInt16ConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -64,100 +41,19 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private short value;
- private int offset;
- private boolean positive;
- private AMutableInt16 aInt16 = new AMutableInt16((short) 0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AInt16> int16Serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractInt16ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int startOffset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[startOffset];
- if (tt == ATypeTag.SERIALIZED_INT16_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, startOffset + 1, len - 1);
- offset = utf8Ptr.getCharStartOffset();
- //accumulating value in negative domain
- //otherwise Short.MIN_VALUE = -(Short.MAX_VALUE + 1) would have caused overflow
- value = 0;
- positive = true;
- short limit = -Short.MAX_VALUE;
- if (serString[offset] == '+') {
- offset++;
- } else if (serString[offset] == '-') {
- offset++;
- positive = false;
- limit = Short.MIN_VALUE;
- }
- int end = startOffset + len;
- for (; offset < end; offset++) {
- int digit;
- if (serString[offset] >= '0' && serString[offset] <= '9') {
- value = (short) (value * 10);
- digit = serString[offset] - '0';
- } else if (serString[offset] == 'i' && serString[offset + 1] == '1'
- && serString[offset + 2] == '6' && offset + 3 == end) {
- break;
- } else {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT16_TYPE_TAG);
- }
- if (value < limit + digit) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT16_TYPE_TAG);
- }
- value = (short) (value - digit);
- }
- if (value > 0) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT16_TYPE_TAG);
- }
- if (value < 0 && positive) {
- value *= -1;
- }
-
- aInt16.setValue(value);
- int16Serde.serialize(aInt16, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_INT16_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return AInt16ConstructorDescriptor.this.getIdentifier();
}
};
}
};
-
}
@Override
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.INT16_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16DefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16DefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..d7ae6b8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt16DefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AInt16DefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AInt16DefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractInt16ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return AInt16DefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.INT16_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
index 8a9c91c..d9c13ab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32ConstructorDescriptor.java
@@ -18,43 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class AInt32ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AInt32ConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AInt32ConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -63,90 +40,11 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private int value, offset;
- private boolean positive;
- private AMutableInt32 aInt32 = new AMutableInt32(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AInt32> int32Serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractInt32ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int startOffset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[startOffset];
- if (tt == ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, startOffset + 1, len - 1);
- offset = utf8Ptr.getCharStartOffset();
- //accumulating value in negative domain
- //otherwise Integer.MIN_VALUE = -(Integer.MAX_VALUE + 1) would have caused overflow
- value = 0;
- positive = true;
- int limit = -Integer.MAX_VALUE;
- if (serString[offset] == '+') {
- offset++;
- } else if (serString[offset] == '-') {
- offset++;
- positive = false;
- limit = Integer.MIN_VALUE;
- }
- int end = startOffset + len;
- for (; offset < end; offset++) {
- int digit;
- if (serString[offset] >= '0' && serString[offset] <= '9') {
- value *= 10;
- digit = serString[offset] - '0';
- } else if (serString[offset] == 'i' && serString[offset + 1] == '3'
- && serString[offset + 2] == '2' && offset + 3 == end) {
- break;
- } else {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT32_TYPE_TAG);
- }
- if (value < limit + digit) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT32_TYPE_TAG);
- }
- value -= digit;
- }
- if (value > 0) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT32_TYPE_TAG);
- }
- if (value < 0 && positive) {
- value *= -1;
- }
-
- aInt32.setValue(value);
- int32Serde.serialize(aInt32, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_INT32_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return AInt32ConstructorDescriptor.this.getIdentifier();
}
};
}
@@ -157,5 +55,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.INT32_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32DefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32DefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..bebf803
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt32DefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AInt32DefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AInt32DefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractInt32ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return AInt32DefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.INT32_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
index 265ccf6..e46b5b0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64ConstructorDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -32,12 +31,7 @@
@MissingNullInOutFunction
public class AInt64ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AInt64ConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AInt64ConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -46,7 +40,7 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
+ return new AbstractInt64ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
public FunctionIdentifier getIdentifier() {
return AInt64ConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64DefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64DefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..0b851fe
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt64DefaultNullConstructorDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AInt64DefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AInt64DefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractInt64ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return AInt64DefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.INT64_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
index c2986fe..425623a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8ConstructorDescriptor.java
@@ -18,43 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AInt8;
-import org.apache.asterix.om.base.AMutableInt8;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class AInt8ConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AInt8ConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AInt8ConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -63,101 +40,19 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private byte value;
- private int offset;
- private boolean positive;
- private AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AInt8> int8Serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractInt8ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int startOffset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[startOffset];
- if (tt == ATypeTag.SERIALIZED_INT8_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, startOffset + 1, len - 1);
- offset = utf8Ptr.getCharStartOffset();
- //accumulating value in negative domain
- //otherwise Byte.MIN_VALUE = -(Byte.MAX_VALUE + 1) would have caused overflow
- value = 0;
- positive = true;
- byte limit = -Byte.MAX_VALUE;
- if (serString[offset] == '+') {
- offset++;
- } else if (serString[offset] == '-') {
- offset++;
- positive = false;
- limit = Byte.MIN_VALUE;
- }
- int end = startOffset + len;
- for (; offset < end; offset++) {
- int digit;
- if (serString[offset] >= '0' && serString[offset] <= '9') {
- value = (byte) (value * 10);
- digit = serString[offset] - '0';
- } else if (serString[offset] == 'i' && serString[offset + 1] == '8'
- && offset + 2 == end) {
- break;
- } else {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT8_TYPE_TAG);
- }
- if (value < limit + digit) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT8_TYPE_TAG);
- }
- value = (byte) (value - digit);
- }
- if (value > 0) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_INT8_TYPE_TAG);
- }
- if (value < 0 && positive) {
- value *= -1;
- }
-
- aInt8.setValue(value);
- int8Serde.serialize(aInt8, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e1) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e1,
- ATypeTag.SERIALIZED_INT8_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return AInt8ConstructorDescriptor.this.getIdentifier();
}
};
}
};
-
}
@Override
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.INT8_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8DefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8DefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..5b38574
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AInt8DefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AInt8DefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AInt8DefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractInt8ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return AInt8DefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.INT8_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
index f548626..2c44692 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.AInterval;
import org.apache.asterix.om.base.AMutableInterval;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -54,12 +53,7 @@
public class AIntervalConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.INTERVAL_CONSTRUCTOR;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AIntervalConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AIntervalConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
index a95e474..6654166 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
@@ -36,7 +36,6 @@
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -61,12 +60,7 @@
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.INTERVAL_CONSTRUCTOR_START_FROM_DATE;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AIntervalStartFromDateConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AIntervalStartFromDateConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
index 11e850c..1fa1242 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
@@ -36,7 +36,6 @@
import org.apache.asterix.om.base.temporal.ATimeParserFactory;
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -60,12 +59,7 @@
public class AIntervalStartFromDateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.INTERVAL_CONSTRUCTOR_START_FROM_DATETIME;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AIntervalStartFromDateTimeConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AIntervalStartFromDateTimeConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
index 881fed0..40dfb67 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
@@ -35,7 +35,6 @@
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -59,12 +58,7 @@
public class AIntervalStartFromTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.INTERVAL_CONSTRUCTOR_START_FROM_TIME;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AIntervalStartFromTimeConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AIntervalStartFromTimeConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
index 9101223..853e538 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ALineConstructorDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableLine;
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -51,12 +50,7 @@
public class ALineConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ALineConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ALineConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
index 4d627fb..34d0b9f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APoint3DConstructorDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AMutablePoint3D;
import org.apache.asterix.om.base.APoint3D;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,12 +49,7 @@
public class APoint3DConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new APoint3DConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = APoint3DConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
index 47d2b11..abf62d8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APointConstructorDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.APoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -49,12 +48,7 @@
@MissingNullInOutFunction
public class APointConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new APointConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = APointConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
index 61008c3..41e8bc7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/APolygonConstructorDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -50,12 +49,7 @@
private static final Pattern WS = Pattern.compile("\\s+");
private static final Pattern COMMA = Pattern.compile(",");
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new APolygonConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = APolygonConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
index 6eed606..729fc4f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ARectangleConstructorDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableRectangle;
import org.apache.asterix.om.base.ARectangle;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,12 +49,7 @@
@MissingNullInOutFunction
public class ARectangleConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ARectangleConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ARectangleConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
index 1ed8d43..93fe469 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringConstructorDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
public class AStringConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AStringConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AStringConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -47,7 +41,7 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
+ return new AbstractStringConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
protected FunctionIdentifier getIdentifier() {
return AStringConstructorDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..64aed11
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AStringDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AStringDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AStringDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractStringConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return AStringDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.STRING_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
index 277cf6a..d648ce3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorDescriptor.java
@@ -18,45 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableTime;
-import org.apache.asterix.om.base.ATime;
-import org.apache.asterix.om.base.temporal.ATimeParserFactory;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class ATimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ATimeConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ATimeConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -65,64 +40,10 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableTime aTime = new AMutableTime(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ATime> timeSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
-
+ return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[offset];
- if (tt == ATypeTag.SERIALIZED_TIME_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, offset + 1, len - 1);
- int stringLength = utf8Ptr.getUTF8Length();
- int startOffset = utf8Ptr.getCharStartOffset();
-
- // the string to be parsed should be at least 6 characters: hhmmss
- if (stringLength < 6) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(),
- ATypeTag.SERIALIZED_TIME_TYPE_TAG);
- }
-
- int chrononTimeInMs =
- ATimeParserFactory.parseTimePart(serString, startOffset, stringLength);
-
- if (chrononTimeInMs < 0) {
- chrononTimeInMs += GregorianCalendarSystem.CHRONON_OF_DAY;
- }
-
- aTime.setValue(chrononTimeInMs);
- timeSerde.serialize(aTime, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_POLYGON_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return ATimeConstructorDescriptor.this.getIdentifier();
}
};
}
@@ -133,5 +54,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.TIME_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..9244f7e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeConstructorWithFormatDescriptor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ATimeConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ATimeConstructorWithFormatDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractTimeWithFormatConstructorEvaluator(ctx, args, sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ATimeConstructorWithFormatDescriptor.this.getIdentifier();
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.TIME_CONSTRUCTOR_WITH_FORMAT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..930b048
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeDefaultNullConstructorDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ATimeDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ATimeDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractTimeConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ATimeDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeDefaultNullConstructorWithFormatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeDefaultNullConstructorWithFormatDescriptor.java
new file mode 100644
index 0000000..5200ea6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ATimeDefaultNullConstructorWithFormatDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class ATimeDefaultNullConstructorWithFormatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = ATimeDefaultNullConstructorWithFormatDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractTimeWithFormatConstructorEvaluator(ctx, args, sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return ATimeDefaultNullConstructorWithFormatDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg, formatArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.TIME_DEFAULT_NULL_CONSTRUCTOR_WITH_FORMAT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
index 6693877..299c928 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringConstructorDescriptor.java
@@ -18,30 +18,15 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableUUID;
-import org.apache.asterix.om.base.AUUID;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
/**
* Receives a canonical representation of UUID and construct a UUID value.
@@ -61,52 +46,12 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableUUID uuid = new AMutableUUID();
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AUUID> uuidSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AUUID);
-
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractUUIDFromStringConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx),
+ sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int start = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[start];
- if (tt == ATypeTag.SERIALIZED_UUID_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, start + 1, len - 1);
-
- // first byte: tag, next x bytes: length
- int offset = utf8Ptr.getCharStartOffset();
- uuid.parseUUIDHexBytes(serString, offset);
- uuidSerde.serialize(uuid, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (HyracksDataException e) {
- e.setSourceLocation(sourceLoc);
- throw e;
- }
+ protected FunctionIdentifier getIdentifier() {
+ return AUUIDFromStringConstructorDescriptor.this.getIdentifier();
}
};
}
@@ -117,5 +62,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.UUID_CONSTRUCTOR;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..f982037
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AUUIDFromStringDefaultNullConstructorDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+/**
+ * Receives a canonical representation of UUID and construct a UUID value.
+ * a UUID is represented by 32 lowercase hexadecimal digits (8-4-4-4-12). (E.g.
+ * uuid("02a199ca-bf58-412e-bd9f-60a0c975a8ac"))
+ */
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AUUIDFromStringDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AUUIDFromStringDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractUUIDFromStringConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx),
+ sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return AUUIDFromStringDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
index aa304ce..5adb725 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
@@ -18,46 +18,20 @@
*/
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableYearMonthDuration;
-import org.apache.asterix.om.base.AYearMonthDuration;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory;
-import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class AYearMonthDurationConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AYearMonthDurationConstructorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AYearMonthDurationConstructorDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -66,61 +40,18 @@
@Override
public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable inputArg = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
- private AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde =
- SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AYEARMONTHDURATION);
- private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ return new AbstractYearMonthDurationConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx),
+ sourceLoc) {
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- eval.evaluate(tuple, inputArg);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- int len = inputArg.getLength();
-
- byte tt = serString[offset];
- if (tt == ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- resultStorage.reset();
- utf8Ptr.set(serString, offset + 1, len - 1);
- int stringLength = utf8Ptr.getUTF8Length();
- ADurationParserFactory.parseDuration(serString, utf8Ptr.getCharStartOffset(),
- stringLength, aYearMonthDuration, ADurationParseOption.YEAR_MONTH);
- yearMonthDurationSerde.serialize(aYearMonthDuration, out);
- result.set(resultStorage);
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt,
- ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e,
- ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG);
- }
+ protected FunctionIdentifier getIdentifier() {
+ return AYearMonthDurationConstructorDescriptor.this.getIdentifier();
}
-
};
}
};
}
- /* (non-Javadoc)
- * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
- */
@Override
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.YEAR_MONTH_DURATION_CONSTRUCTOR;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationDefaultNullConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationDefaultNullConstructorDescriptor.java
new file mode 100644
index 0000000..4360f0b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AYearMonthDurationDefaultNullConstructorDescriptor.java
@@ -0,0 +1,69 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+@MissingNullInOutFunction(onMissing = MissingNullInOutFunction.MissingNullType.NULL)
+public class AYearMonthDurationDefaultNullConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = AYearMonthDurationDefaultNullConstructorDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new AbstractYearMonthDurationConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx),
+ sourceLoc) {
+
+ @Override
+ protected FunctionIdentifier getIdentifier() {
+ return AYearMonthDurationDefaultNullConstructorDescriptor.this.getIdentifier();
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetNull(result, inputArg);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR;
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBinaryConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBinaryConstructorEvaluator.java
new file mode 100644
index 0000000..6960273
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBinaryConstructorEvaluator.java
@@ -0,0 +1,81 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABinary;
+import org.apache.asterix.om.base.AMutableBinary;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+abstract class AbstractBinaryConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ protected final AMutableBinary aBinary = new AMutableBinary(new byte[0], 0, 0);
+ @SuppressWarnings("unchecked")
+ protected final ISerializerDeserializer<ABinary> binarySerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABINARY);
+ protected final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ protected AbstractBinaryConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case BINARY:
+ result.set(inputArg);
+ break;
+ case STRING:
+ utf8Ptr.set(inputArg.getByteArray(), startOffset + 1, len - 1);
+ if (parseBinary(utf8Ptr, aBinary)) {
+ resultStorage.reset();
+ binarySerde.serialize(aBinary, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected BuiltinType getTargetType() {
+ return BuiltinType.ABINARY;
+ }
+
+ protected abstract boolean parseBinary(UTF8StringPointable textPtr, AMutableBinary result);
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
index 7e1e35b..6777feb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractBooleanConstructorEvaluator.java
@@ -19,32 +19,33 @@
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.common.NumberUtils;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.util.string.UTF8StringUtil;
-public abstract class AbstractBooleanConstructorEvaluator implements IScalarEvaluator {
+public abstract class AbstractBooleanConstructorEvaluator extends AbstractConstructorEvaluator {
+
@SuppressWarnings("unchecked")
- protected static final ISerializerDeserializer<ABoolean> BOOLEAN_SERDE =
+ protected final ISerializerDeserializer<ABoolean> booleanSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
protected static final IBinaryComparator UTF8_BINARY_CMP =
@@ -53,60 +54,85 @@
protected static final byte[] TRUE = UTF8StringUtil.writeStringToBytes("true");
protected static final byte[] FALSE = UTF8StringUtil.writeStringToBytes("false");
- protected final IScalarEvaluator inputEval;
- protected final SourceLocation sourceLoc;
- protected final IPointable inputArg;
- protected final ArrayBackedValueStorage resultStorage;
- protected final DataOutput out;
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
- protected AbstractBooleanConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
- this.inputEval = inputEval;
- this.sourceLoc = sourceLoc;
- inputArg = new VoidPointable();
- resultStorage = new ArrayBackedValueStorage();
- out = resultStorage.getDataOutput();
+ protected AbstractBooleanConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- inputEval.evaluate(tuple, inputArg);
- resultStorage.reset();
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- evaluateImpl(result);
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
- }
- }
-
protected void evaluateImpl(IPointable result) throws HyracksDataException {
byte[] bytes = inputArg.getByteArray();
int startOffset = inputArg.getStartOffset();
- byte tt = bytes[startOffset];
- if (tt == ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- int len = inputArg.getLength();
- if (UTF8_BINARY_CMP.compare(bytes, startOffset + 1, len - 1, TRUE, 0, TRUE.length) == 0) {
- setBoolean(result, true);
- } else if (UTF8_BINARY_CMP.compare(bytes, startOffset + 1, len - 1, FALSE, 0, FALSE.length) == 0) {
- setBoolean(result, false);
- } else {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG);
- }
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case BOOLEAN:
+ result.set(inputArg);
+ break;
+ case TINYINT:
+ setInteger(AInt8SerializerDeserializer.getByte(bytes, startOffset + 1), result);
+ break;
+ case SMALLINT:
+ setInteger(AInt16SerializerDeserializer.getShort(bytes, startOffset + 1), result);
+ break;
+ case INTEGER:
+ setInteger(AInt32SerializerDeserializer.getInt(bytes, startOffset + 1), result);
+ break;
+ case BIGINT:
+ setInteger(AInt64SerializerDeserializer.getLong(bytes, startOffset + 1), result);
+ break;
+ case FLOAT:
+ setDouble(AFloatSerializerDeserializer.getFloat(bytes, startOffset + 1), result);
+ break;
+ case DOUBLE:
+ setDouble(ADoubleSerializerDeserializer.getDouble(bytes, startOffset + 1), result);
+ break;
+ case STRING:
+ Boolean v = parseBoolean(bytes, startOffset, len);
+ if (v != null) {
+ setBoolean(result, v);
+ } else {
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
}
}
- protected void setBoolean(IPointable result, boolean value) throws HyracksDataException {
- BOOLEAN_SERDE.serialize(ABoolean.valueOf(value), out);
+ protected Boolean parseBoolean(byte[] bytes, int offset, int len) throws HyracksDataException {
+ if (UTF8_BINARY_CMP.compare(bytes, offset + 1, len - 1, TRUE, 0, TRUE.length) == 0) {
+ return true;
+ } else if (UTF8_BINARY_CMP.compare(bytes, offset + 1, len - 1, FALSE, 0, FALSE.length) == 0) {
+ return false;
+ } else {
+ return null;
+ }
+ }
+
+ protected final void setBoolean(IPointable result, boolean value) throws HyracksDataException {
+ resultStorage.reset();
+ booleanSerde.serialize(ABoolean.valueOf(value), out);
result.set(resultStorage);
}
- protected abstract FunctionIdentifier getIdentifier();
+ protected final void setInteger(long v, IPointable result) throws HyracksDataException {
+ setBoolean(result, v != 0);
+ }
+
+ protected final void setDouble(double v, IPointable result) throws HyracksDataException {
+ long bits = Double.doubleToLongBits(v);
+ boolean zeroOrNaN = bits == NumberUtils.POSITIVE_ZERO_BITS || bits == NumberUtils.NEGATIVE_ZERO_BITS
+ || bits == NumberUtils.NAN_BITS;
+ setBoolean(result, !zeroOrNaN);
+ }
+
+ @Override
+ protected BuiltinType getTargetType() {
+ return BuiltinType.ABOOLEAN;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
new file mode 100644
index 0000000..e7c4cd5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractConstructorEvaluator.java
@@ -0,0 +1,106 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.LogRedactionUtil;
+
+public abstract class AbstractConstructorEvaluator implements IScalarEvaluator {
+
+ protected final IEvaluatorContext ctx;
+ protected final IScalarEvaluator inputEval;
+ protected final SourceLocation sourceLoc;
+ protected final IPointable inputArg;
+ protected final ArrayBackedValueStorage resultStorage;
+ protected final DataOutput out;
+
+ protected AbstractConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ this.ctx = ctx;
+ this.inputEval = inputEval;
+ this.sourceLoc = sourceLoc;
+ this.inputArg = new VoidPointable();
+ this.resultStorage = new ArrayBackedValueStorage();
+ this.out = resultStorage.getDataOutput();
+ }
+
+ @Override
+ public final void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ evaluateInput(tuple);
+ if (checkAndSetMissingOrNull(result)) {
+ return;
+ }
+ evaluateImpl(result);
+ }
+
+ protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+ inputEval.evaluate(tuple, inputArg);
+ }
+
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetMissingOrNull(result, inputArg);
+ }
+
+ protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
+
+ protected abstract FunctionIdentifier getIdentifier();
+
+ protected abstract BuiltinType getTargetType();
+
+ protected void handleUnsupportedType(ATypeTag inputType, IPointable result) throws HyracksDataException {
+ warnUnsupportedType(inputType);
+ PointableHelper.setNull(result);
+ }
+
+ protected void warnUnsupportedType(ATypeTag inputType) {
+ ExceptionUtil.warnUnsupportedType(ctx, sourceLoc, getIdentifier().getName() + "()", inputType);
+ }
+
+ protected void handleParseError(UTF8StringPointable textPtr, IPointable result) {
+ warnParseError(textPtr);
+ PointableHelper.setNull(result);
+ }
+
+ protected void warnParseError(UTF8StringPointable textPtr) {
+ IWarningCollector warningCollector = ctx.getWarningCollector();
+ if (warningCollector.shouldWarn()) {
+ warningCollector.warn(Warning.of(sourceLoc, ErrorCode.INVALID_FORMAT,
+ getTargetType().getTypeTag().toString(), LogRedactionUtil.userData(textPtr.toString())));
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
new file mode 100644
index 0000000..bb0dbd1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorEvaluator.java
@@ -0,0 +1,107 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADate;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableDate aDate = new AMutableDate(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADate> dateSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ private final GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
+
+ protected AbstractDateConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case DATE:
+ result.set(inputArg);
+ break;
+ case DATETIME:
+ long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+ int chrononInDays = cal.getChrononInDays(chronon);
+ aDate.setValue(chrononInDays);
+ resultStorage.reset();
+ dateSerde.serialize(aDate, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (parseDate(utf8Ptr, aDate)) {
+ resultStorage.reset();
+ dateSerde.serialize(aDate, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+ int stringLength = textPtr.getUTF8Length();
+ if (stringLength < 8) {
+ return false;
+ }
+ try {
+ long chronon = ADateParserFactory.parseDatePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+ stringLength);
+ int chrononInDays = cal.getChrononInDays(chronon);
+ result.setValue(chrononInDays);
+ return true;
+ } catch (HyracksDataException e) {
+ return false;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.ADATE;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorWithFormatEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorWithFormatEvaluator.java
new file mode 100644
index 0000000..7e9a6b3
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateConstructorWithFormatEvaluator.java
@@ -0,0 +1,87 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractDateConstructorWithFormatEvaluator extends AbstractDateConstructorEvaluator {
+
+ private final IScalarEvaluator formatEval;
+ protected final IPointable formatArg = new VoidPointable();
+ private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+ private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+ protected AbstractDateConstructorWithFormatEvaluator(IEvaluatorContext ctx, IScalarEvaluatorFactory[] args,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(ctx, args[0].createScalarEvaluator(ctx), sourceLoc);
+ formatEval = args[1].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+ super.evaluateInput(tuple);
+ formatEval.evaluate(tuple, formatArg);
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+ }
+
+ @Override
+ protected boolean parseDate(UTF8StringPointable textPtr, AMutableDate result) {
+ byte[] formatBytes = formatArg.getByteArray();
+ int formatStartOffset = formatArg.getStartOffset();
+ int formatLength = formatArg.getLength();
+ if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ return false;
+ }
+ formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+ try {
+ if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+ textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+ formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+ DateTimeFormatUtils.DateTimeParseMode.DATE_ONLY, false)) {
+ result.setValue((int) (aInt64.getLongValue() / GregorianCalendarSystem.CHRONON_OF_DAY));
+ return true;
+ } else {
+ return false;
+ }
+ } catch (AsterixTemporalTypeParseException e) {
+ // shouldn't happen
+ return false;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
new file mode 100644
index 0000000..6765d90
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorEvaluator.java
@@ -0,0 +1,119 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.temporal.ADateParserFactory;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDateTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableDateTime aDateTime = new AMutableDateTime(0L);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADateTime> datetimeSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ protected AbstractDateTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case DATETIME:
+ result.set(inputArg);
+ break;
+ case DATE:
+ int chrononInDays = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1);
+ aDateTime.setValue(chrononInDays * GregorianCalendarSystem.CHRONON_OF_DAY);
+ resultStorage.reset();
+ datetimeSerde.serialize(aDateTime, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (parseDateTime(utf8Ptr, aDateTime)) {
+ resultStorage.reset();
+ datetimeSerde.serialize(aDateTime, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+ int stringLength = textPtr.getUTF8Length();
+ // the string to be parsed should be at least 14 characters: YYYYMMDDhhmmss
+ if (stringLength < 14) {
+ return false;
+ }
+ byte[] bytes = textPtr.getByteArray();
+ int charStartOffset = textPtr.getCharStartOffset();
+ // +1 if it is negative (-)
+ short timeOffset = (short) ((bytes[charStartOffset] == '-') ? 1 : 0);
+ timeOffset += 8;
+ if (bytes[charStartOffset + timeOffset] != 'T') {
+ timeOffset += 2;
+ if (bytes[charStartOffset + timeOffset] != 'T') {
+ return false;
+ }
+ }
+ try {
+ long chronon = ADateParserFactory.parseDatePart(bytes, charStartOffset, timeOffset);
+ int timeInMs = ATimeParserFactory.parseTimePart(bytes, charStartOffset + timeOffset + 1,
+ stringLength - timeOffset - 1);
+ chronon += timeInMs;
+ result.setValue(chronon);
+ return true;
+ } catch (HyracksDataException e) {
+ return false;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.ADATETIME;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorWithFormatEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorWithFormatEvaluator.java
new file mode 100644
index 0000000..8c8da2e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDateTimeConstructorWithFormatEvaluator.java
@@ -0,0 +1,86 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractDateTimeConstructorWithFormatEvaluator extends AbstractDateTimeConstructorEvaluator {
+
+ private final IScalarEvaluator formatEval;
+ protected final IPointable formatArg = new VoidPointable();
+ private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+ private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+ protected AbstractDateTimeConstructorWithFormatEvaluator(IEvaluatorContext ctx, IScalarEvaluatorFactory[] args,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(ctx, args[0].createScalarEvaluator(ctx), sourceLoc);
+ formatEval = args[1].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+ super.evaluateInput(tuple);
+ formatEval.evaluate(tuple, formatArg);
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+ }
+
+ @Override
+ protected boolean parseDateTime(UTF8StringPointable textPtr, AMutableDateTime result) {
+ byte[] formatBytes = formatArg.getByteArray();
+ int formatStartOffset = formatArg.getStartOffset();
+ int formatLength = formatArg.getLength();
+ if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ return false;
+ }
+ formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+ try {
+ if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+ textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+ formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+ DateTimeFormatUtils.DateTimeParseMode.DATETIME, false)) {
+ result.setValue(aInt64.getLongValue());
+ return true;
+ } else {
+ return false;
+ }
+ } catch (AsterixTemporalTypeParseException e) {
+ // shouldn't happen
+ return false;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDayTimeDurationConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDayTimeDurationConstructorEvaluator.java
new file mode 100644
index 0000000..2706b5e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDayTimeDurationConstructorEvaluator.java
@@ -0,0 +1,98 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADayTimeDuration;
+import org.apache.asterix.om.base.AMutableDayTimeDuration;
+import org.apache.asterix.om.base.temporal.ADurationParserFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDayTimeDurationConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADAYTIMEDURATION);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ protected AbstractDayTimeDurationConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case DAYTIMEDURATION:
+ result.set(inputArg);
+ break;
+ case DURATION:
+ long millis = ADurationSerializerDeserializer.getDayTime(bytes, startOffset + 1);
+ aDayTimeDuration.setMilliseconds(millis);
+ resultStorage.reset();
+ dayTimeDurationSerde.serialize(aDayTimeDuration, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (parseDayTimeDuration(utf8Ptr, aDayTimeDuration)) {
+ resultStorage.reset();
+ dayTimeDurationSerde.serialize(aDayTimeDuration, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.ADAYTIMEDURATION;
+ }
+
+ private static boolean parseDayTimeDuration(UTF8StringPointable textPtr, ADayTimeDuration result) {
+ try {
+ ADurationParserFactory.parseDuration(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+ textPtr.getUTF8Length(), result, ADurationParserFactory.ADurationParseOption.DAY_TIME);
+ return true;
+ } catch (HyracksDataException e) {
+ return false;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
index 04220ce..b38b5e2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDoubleConstructorEvaluator.java
@@ -19,90 +19,87 @@
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-public abstract class AbstractDoubleConstructorEvaluator implements IScalarEvaluator {
+public abstract class AbstractDoubleConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ protected final AMutableDouble aDouble = new AMutableDouble(0);
@SuppressWarnings("unchecked")
- protected static final ISerializerDeserializer<ADouble> DOUBLE_SERDE =
+ protected final ISerializerDeserializer<ADouble> doubleSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+ protected final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
- protected final IScalarEvaluator inputEval;
- protected final SourceLocation sourceLoc;
- protected final ArrayBackedValueStorage resultStorage;
- protected final DataOutput out;
- protected final IPointable inputArg;
- protected final AMutableDouble aDouble;
- protected final UTF8StringPointable utf8Ptr;
-
- protected AbstractDoubleConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
- this.inputEval = inputEval;
- this.sourceLoc = sourceLoc;
- resultStorage = new ArrayBackedValueStorage();
- out = resultStorage.getDataOutput();
- inputArg = new VoidPointable();
- aDouble = new AMutableDouble(0);
- utf8Ptr = new UTF8StringPointable();
+ protected AbstractDoubleConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- inputEval.evaluate(tuple, inputArg);
- resultStorage.reset();
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- evaluateImpl(result);
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
- }
- }
-
- protected void evaluateImpl(IPointable result) throws IOException {
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
byte[] bytes = inputArg.getByteArray();
int startOffset = inputArg.getStartOffset();
- byte tt = bytes[startOffset];
- if (tt == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- utf8Ptr.set(bytes, startOffset + 1, inputArg.getLength() - 1);
- if (NumberUtils.parseDouble(utf8Ptr, aDouble)) {
- DOUBLE_SERDE.serialize(aDouble, out);
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case DOUBLE:
+ result.set(inputArg);
+ break;
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ resultStorage.reset();
+ try {
+ ATypeHierarchy.getTypePromoteComputer(inputType, ATypeTag.DOUBLE).convertType(bytes,
+ startOffset + 1, len - 1, out);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
result.set(resultStorage);
- } else {
- handleUparseableString(result);
- }
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ break;
+ case BOOLEAN:
+ resultStorage.reset();
+ boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, startOffset + 1);
+ aDouble.setValue(b ? 1 : 0);
+ doubleSerde.serialize(aDouble, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (NumberUtils.parseDouble(utf8Ptr, aDouble)) {
+ resultStorage.reset();
+ doubleSerde.serialize(aDouble, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
}
}
- protected void handleUparseableString(IPointable result) throws HyracksDataException {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+ @Override
+ protected BuiltinType getTargetType() {
+ return BuiltinType.ADOUBLE;
}
-
- protected abstract FunctionIdentifier getIdentifier();
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDurationConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDurationConstructorEvaluator.java
new file mode 100644
index 0000000..ee0affd
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractDurationConstructorEvaluator.java
@@ -0,0 +1,106 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADuration;
+import org.apache.asterix.om.base.AMutableDuration;
+import org.apache.asterix.om.base.temporal.ADurationParserFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractDurationConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableDuration aDuration = new AMutableDuration(0, 0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADuration> durationSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADURATION);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ protected AbstractDurationConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case DURATION:
+ result.set(inputArg);
+ break;
+ case YEARMONTHDURATION:
+ int months = AYearMonthDurationSerializerDeserializer.getYearMonth(bytes, startOffset + 1);
+ aDuration.setValue(months, 0);
+ resultStorage.reset();
+ durationSerde.serialize(aDuration, out);
+ result.set(resultStorage);
+ break;
+ case DAYTIMEDURATION:
+ long millis = ADayTimeDurationSerializerDeserializer.getDayTime(bytes, startOffset + 1);
+ aDuration.setValue(0, millis);
+ resultStorage.reset();
+ durationSerde.serialize(aDuration, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (parseDuration(utf8Ptr, aDuration)) {
+ resultStorage.reset();
+ durationSerde.serialize(aDuration, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.ADURATION;
+ }
+
+ private static boolean parseDuration(UTF8StringPointable textPtr, ADuration result) {
+ try {
+ ADurationParserFactory.parseDuration(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+ textPtr.getUTF8Length(), result, ADurationParserFactory.ADurationParseOption.All);
+ return true;
+ } catch (HyracksDataException e) {
+ return false;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractFloatConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractFloatConstructorEvaluator.java
new file mode 100644
index 0000000..3d50a6c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractFloatConstructorEvaluator.java
@@ -0,0 +1,114 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.common.NumberUtils;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractFloatConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableFloat aFloat = new AMutableFloat(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AFloat> floatSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ protected AbstractFloatConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case FLOAT:
+ result.set(inputArg);
+ break;
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ resultStorage.reset();
+ try {
+ ATypeHierarchy.getTypePromoteComputer(inputType, ATypeTag.FLOAT).convertType(bytes, startOffset + 1,
+ len - 1, out);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ break;
+ case DOUBLE:
+ resultStorage.reset();
+ try {
+ ATypeHierarchy.getTypeDemoteComputer(inputType, ATypeTag.FLOAT, false).convertType(bytes,
+ startOffset + 1, len - 1, out);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ break;
+ case BOOLEAN:
+ boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, startOffset + 1);
+ aFloat.setValue(b ? 1 : 0);
+ resultStorage.reset();
+ floatSerde.serialize(aFloat, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (NumberUtils.parseFloat(utf8Ptr, aFloat)) {
+ resultStorage.reset();
+ floatSerde.serialize(aFloat, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.AFLOAT;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt16ConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt16ConstructorEvaluator.java
new file mode 100644
index 0000000..ac75877
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt16ConstructorEvaluator.java
@@ -0,0 +1,113 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.base.AMutableInt16;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.common.NumberUtils;
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractInt16ConstructorEvaluator extends AbstractIntConstructorEvaluator {
+
+ private final AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt16> int16Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
+
+ private final AMutableFloat aFloat = new AMutableFloat(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AFloat> floatSerdeNonTagged =
+ SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AFLOAT);
+
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ private final MutableBoolean maybeNumeric = new MutableBoolean();
+
+ protected AbstractInt16ConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case SMALLINT:
+ result.set(inputArg);
+ break;
+ case TINYINT:
+ promoteNumeric(inputType, bytes, startOffset + 1, len - 1, result);
+ break;
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ demoteNumeric(inputType, bytes, startOffset + 1, len - 1, result);
+ break;
+ case BOOLEAN:
+ boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, startOffset + 1);
+ aInt16.setValue((short) (b ? 1 : 0));
+ resultStorage.reset();
+ int16Serde.serialize(aInt16, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (NumberUtils.parseInt16(utf8Ptr, aInt16, maybeNumeric)) {
+ resultStorage.reset();
+ int16Serde.serialize(aInt16, out);
+ result.set(resultStorage);
+ } else if (maybeNumeric.booleanValue() && NumberUtils.parseFloat(utf8Ptr, aFloat)) {
+ tmpStorage.reset();
+ floatSerdeNonTagged.serialize(aFloat, tmpOut);
+ demoteNumeric(ATypeTag.FLOAT, tmpStorage.getByteArray(), tmpStorage.getStartOffset(),
+ tmpStorage.getLength(), result);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.AINT16;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt32ConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt32ConstructorEvaluator.java
new file mode 100644
index 0000000..7b2e198
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt32ConstructorEvaluator.java
@@ -0,0 +1,113 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.common.NumberUtils;
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractInt32ConstructorEvaluator extends AbstractIntConstructorEvaluator {
+
+ private final AMutableInt32 aInt32 = new AMutableInt32(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt32> int32Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+ private final AMutableDouble aDouble = new AMutableDouble(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADouble> doubleSerdeNonTagged =
+ SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.ADOUBLE);
+
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ private final MutableBoolean maybeNumeric = new MutableBoolean();
+
+ protected AbstractInt32ConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case INTEGER:
+ result.set(inputArg);
+ break;
+ case TINYINT:
+ case SMALLINT:
+ promoteNumeric(inputType, bytes, startOffset + 1, len - 1, result);
+ break;
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ demoteNumeric(inputType, bytes, startOffset + 1, len - 1, result);
+ break;
+ case BOOLEAN:
+ boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, startOffset + 1);
+ aInt32.setValue(b ? 1 : 0);
+ resultStorage.reset();
+ int32Serde.serialize(aInt32, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (NumberUtils.parseInt32(utf8Ptr, aInt32, maybeNumeric)) {
+ resultStorage.reset();
+ int32Serde.serialize(aInt32, out);
+ result.set(resultStorage);
+ } else if (maybeNumeric.booleanValue() && NumberUtils.parseDouble(utf8Ptr, aDouble)) {
+ tmpStorage.reset();
+ doubleSerdeNonTagged.serialize(aDouble, tmpOut);
+ demoteNumeric(ATypeTag.DOUBLE, tmpStorage.getByteArray(), tmpStorage.getStartOffset(),
+ tmpStorage.getLength(), result);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.AINT32;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
index cab2db4..d02f304 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt64ConstructorEvaluator.java
@@ -19,91 +19,95 @@
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
-import java.io.IOException;
-
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-public abstract class AbstractInt64ConstructorEvaluator implements IScalarEvaluator {
+public abstract class AbstractInt64ConstructorEvaluator extends AbstractIntConstructorEvaluator {
+
+ protected final AMutableInt64 aInt64 = new AMutableInt64(0);
@SuppressWarnings("unchecked")
- protected static final ISerializerDeserializer<AInt64> INT64_SERDE =
+ protected final ISerializerDeserializer<AInt64> int64Serde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
- protected final IScalarEvaluator inputEval;
- protected final SourceLocation sourceLoc;
- protected final IPointable inputArg;
- protected final ArrayBackedValueStorage resultStorage;
- protected final DataOutput out;
- protected final AMutableInt64 aInt64;
- protected final UTF8StringPointable utf8Ptr;
+ private final AMutableDouble aDouble = new AMutableDouble(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADouble> doubleSerdeNonTagged =
+ SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.ADOUBLE);
- protected AbstractInt64ConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
- this.inputEval = inputEval;
- this.sourceLoc = sourceLoc;
- inputArg = new VoidPointable();
- resultStorage = new ArrayBackedValueStorage();
- out = resultStorage.getDataOutput();
- aInt64 = new AMutableInt64(0);
- utf8Ptr = new UTF8StringPointable();
+ protected final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ private final MutableBoolean maybeNumeric = new MutableBoolean();
+
+ protected AbstractInt64ConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- inputEval.evaluate(tuple, inputArg);
- resultStorage.reset();
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- evaluateImpl(result);
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
- }
- }
-
- protected void evaluateImpl(IPointable result) throws IOException {
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
byte[] bytes = inputArg.getByteArray();
int startOffset = inputArg.getStartOffset();
-
- byte tt = bytes[startOffset];
- if (tt == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
- result.set(inputArg);
- } else if (tt == ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
- utf8Ptr.set(bytes, startOffset + 1, inputArg.getLength() - 1);
- if (NumberUtils.parseInt64(utf8Ptr, aInt64)) {
- INT64_SERDE.serialize(aInt64, out);
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case BIGINT:
+ result.set(inputArg);
+ break;
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ promoteNumeric(inputType, bytes, startOffset + 1, len - 1, result);
+ break;
+ case FLOAT:
+ case DOUBLE:
+ demoteNumeric(inputType, bytes, startOffset + 1, len - 1, result);
+ break;
+ case BOOLEAN:
+ resultStorage.reset();
+ boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, startOffset + 1);
+ aInt64.setValue(b ? 1 : 0);
+ int64Serde.serialize(aInt64, out);
result.set(resultStorage);
- } else {
- handleUnparseableString(result);
- }
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, tt, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (NumberUtils.parseInt64(utf8Ptr, aInt64, maybeNumeric)) {
+ resultStorage.reset();
+ int64Serde.serialize(aInt64, out);
+ result.set(resultStorage);
+ } else if (maybeNumeric.booleanValue() && NumberUtils.parseDouble(utf8Ptr, aDouble)) {
+ tmpStorage.reset();
+ doubleSerdeNonTagged.serialize(aDouble, tmpOut);
+ demoteNumeric(ATypeTag.DOUBLE, tmpStorage.getByteArray(), tmpStorage.getStartOffset(),
+ tmpStorage.getLength(), result);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
}
}
- protected void handleUnparseableString(IPointable result) throws HyracksDataException {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ @Override
+ protected BuiltinType getTargetType() {
+ return BuiltinType.AINT64;
}
-
- protected abstract FunctionIdentifier getIdentifier();
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt8ConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt8ConstructorEvaluator.java
new file mode 100644
index 0000000..e8c73f5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractInt8ConstructorEvaluator.java
@@ -0,0 +1,112 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.common.NumberUtils;
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractInt8ConstructorEvaluator extends AbstractIntConstructorEvaluator {
+
+ private final AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt8> int8Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+
+ private final AMutableFloat aFloat = new AMutableFloat(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AFloat> floatSerdeNonTagged =
+ SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AFLOAT);
+
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ private final MutableBoolean maybeNumeric = new MutableBoolean();
+
+ protected AbstractInt8ConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case TINYINT:
+ result.set(inputArg);
+ break;
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ demoteNumeric(inputType, bytes, startOffset + 1, len - 1, result);
+ break;
+ case BOOLEAN:
+ boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, startOffset + 1);
+ aInt8.setValue((byte) (b ? 1 : 0));
+ resultStorage.reset();
+ int8Serde.serialize(aInt8, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (NumberUtils.parseInt8(utf8Ptr, aInt8, maybeNumeric)) {
+ resultStorage.reset();
+ int8Serde.serialize(aInt8, out);
+ result.set(resultStorage);
+ } else if (maybeNumeric.booleanValue() && NumberUtils.parseFloat(utf8Ptr, aFloat)) {
+ tmpStorage.reset();
+ floatSerdeNonTagged.serialize(aFloat, tmpOut);
+ demoteNumeric(ATypeTag.FLOAT, tmpStorage.getByteArray(), tmpStorage.getStartOffset(),
+ tmpStorage.getLength(), result);
+ break;
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.AINT8;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractIntConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractIntConstructorEvaluator.java
new file mode 100644
index 0000000..484fae2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractIntConstructorEvaluator.java
@@ -0,0 +1,69 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+public abstract class AbstractIntConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ protected final ArrayBackedValueStorage tmpStorage;
+ protected final DataOutput tmpOut;
+
+ public AbstractIntConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ tmpStorage = new ArrayBackedValueStorage();
+ tmpOut = tmpStorage.getDataOutput();
+ }
+
+ protected void promoteNumeric(ATypeTag inputType, byte[] bytes, int startOffset, int len, IPointable result)
+ throws HyracksDataException {
+ resultStorage.reset();
+ try {
+ ATypeHierarchy.getTypePromoteComputer(inputType, getTargetType().getTypeTag()).convertType(bytes,
+ startOffset, len, out);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ protected void demoteNumeric(ATypeTag inputType, byte[] bytes, int startOffset, int len, IPointable result)
+ throws HyracksDataException {
+ resultStorage.reset();
+ try {
+ ATypeHierarchy.getTypeDemoteComputer(inputType, getTargetType().getTypeTag(), false).convertType(bytes,
+ startOffset, len, out);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
index 5ce7853..e1bd65b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractStringConstructorEvaluator.java
@@ -19,103 +19,85 @@
package org.apache.asterix.runtime.evaluators.constructors;
-import java.io.DataOutput;
import java.io.IOException;
+import org.apache.asterix.dataflow.data.nontagged.serde.ABinarySerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
+import org.apache.asterix.om.base.AUUID;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.data.std.util.GrowableArray;
import org.apache.hyracks.data.std.util.UTF8StringBuilder;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.bytes.Base64Printer;
-public abstract class AbstractStringConstructorEvaluator implements IScalarEvaluator {
+public abstract class AbstractStringConstructorEvaluator extends AbstractConstructorEvaluator {
- protected final IScalarEvaluator inputEval;
- protected final SourceLocation sourceLoc;
- protected final IPointable inputArg;
- protected final ArrayBackedValueStorage resultStorage;
- protected final DataOutput out;
- protected final UTF8StringBuilder builder;
- protected final GrowableArray baaos;
+ protected final UTF8StringBuilder builder = new UTF8StringBuilder();
+ protected final GrowableArray baaos = new GrowableArray();
+ protected final StringBuilder sb = new StringBuilder(32);
- protected AbstractStringConstructorEvaluator(IScalarEvaluator inputEval, SourceLocation sourceLoc) {
- this.inputEval = inputEval;
- this.sourceLoc = sourceLoc;
- resultStorage = new ArrayBackedValueStorage();
- out = resultStorage.getDataOutput();
- inputArg = new VoidPointable();
- builder = new UTF8StringBuilder();
- baaos = new GrowableArray();
+ protected AbstractStringConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
}
@Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- try {
- inputEval.evaluate(tuple, inputArg);
- resultStorage.reset();
-
- if (PointableHelper.checkAndSetMissingOrNull(result, inputArg)) {
- return;
- }
-
- evaluateImpl(result);
- } catch (IOException e) {
- throw new InvalidDataFormatException(sourceLoc, getIdentifier(), e, ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- }
- }
-
- protected void evaluateImpl(IPointable result) throws IOException {
- byte[] serString = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
-
- ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[serString[offset]];
- if (tt == ATypeTag.STRING) {
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ if (inputType == ATypeTag.STRING) {
result.set(inputArg);
- } else {
- int len = inputArg.getLength();
+ return;
+ }
+ try {
baaos.reset();
builder.reset(baaos, len);
- int startOffset = offset + 1;
- switch (tt) {
- case TINYINT: {
- int i = AInt8SerializerDeserializer.getByte(serString, startOffset);
- builder.appendString(String.valueOf(i));
+ sb.setLength(0);
+ switch (inputType) {
+ case TINYINT:
+ int i = AInt8SerializerDeserializer.getByte(bytes, startOffset + 1);
+ sb.append(i);
+ builder.appendString(sb);
break;
- }
- case SMALLINT: {
- int i = AInt16SerializerDeserializer.getShort(serString, startOffset);
- builder.appendString(String.valueOf(i));
+ case SMALLINT:
+ i = AInt16SerializerDeserializer.getShort(bytes, startOffset + 1);
+ sb.append(i);
+ builder.appendString(sb);
break;
- }
- case INTEGER: {
- int i = AInt32SerializerDeserializer.getInt(serString, startOffset);
- builder.appendString(String.valueOf(i));
+ case INTEGER:
+ i = AInt32SerializerDeserializer.getInt(bytes, startOffset + 1);
+ sb.append(i);
+ builder.appendString(sb);
break;
- }
- case BIGINT: {
- long l = AInt64SerializerDeserializer.getLong(serString, startOffset);
- builder.appendString(String.valueOf(l));
+ case BIGINT:
+ long l = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
+ sb.append(l);
+ builder.appendString(sb);
break;
- }
- case DOUBLE: {
- double d = ADoubleSerializerDeserializer.getDouble(serString, startOffset);
+ case DOUBLE:
+ double d = ADoubleSerializerDeserializer.getDouble(bytes, startOffset + 1);
if (Double.isNaN(d)) {
builder.appendUtf8StringPointable(NumberUtils.NAN);
} else if (d == Double.POSITIVE_INFINITY) { // NOSONAR
@@ -123,12 +105,12 @@
} else if (d == Double.NEGATIVE_INFINITY) { // NOSONAR
builder.appendUtf8StringPointable(NumberUtils.NEGATIVE_INF);
} else {
- builder.appendString(String.valueOf(d));
+ sb.append(d);
+ builder.appendString(sb);
}
break;
- }
- case FLOAT: {
- float f = AFloatSerializerDeserializer.getFloat(serString, startOffset);
+ case FLOAT:
+ float f = AFloatSerializerDeserializer.getFloat(bytes, startOffset + 1);
if (Float.isNaN(f)) {
builder.appendUtf8StringPointable(NumberUtils.NAN);
} else if (f == Float.POSITIVE_INFINITY) { // NOSONAR
@@ -136,43 +118,75 @@
} else if (f == Float.NEGATIVE_INFINITY) { // NOSONAR
builder.appendUtf8StringPointable(NumberUtils.NEGATIVE_INF);
} else {
- builder.appendString(String.valueOf(f));
+ sb.append(f);
+ builder.appendString(sb);
}
break;
- }
- case BOOLEAN: {
- boolean b = ABooleanSerializerDeserializer.getBoolean(serString, startOffset);
+ case BOOLEAN:
+ boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, startOffset + 1);
builder.appendString(String.valueOf(b));
break;
- }
-
- // NotYetImplemented
- case CIRCLE:
case DATE:
- case DATETIME:
- case LINE:
+ l = ADateSerializerDeserializer.getChronon(bytes, startOffset + 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(l, sb,
+ GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.DAY);
+ builder.appendString(sb);
+ break;
case TIME:
- case DURATION:
+ i = ATimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(i, sb,
+ GregorianCalendarSystem.Fields.HOUR, GregorianCalendarSystem.Fields.MILLISECOND);
+ builder.appendString(sb);
+ break;
+ case DATETIME:
+ l = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+ GregorianCalendarSystem.getInstance().getExtendStringRepUntilField(l, sb,
+ GregorianCalendarSystem.Fields.YEAR, GregorianCalendarSystem.Fields.MILLISECOND);
+ builder.appendString(sb);
+ break;
case YEARMONTHDURATION:
+ i = AYearMonthDurationSerializerDeserializer.getYearMonth(bytes, startOffset + 1);
+ GregorianCalendarSystem.getInstance().getDurationExtendStringRepWithTimezoneUntilField(0, i, sb);
+ builder.appendString(sb);
+ break;
case DAYTIMEDURATION:
- case INTERVAL:
- case ARRAY:
- case POINT:
- case POINT3D:
- case RECTANGLE:
- case POLYGON:
- case OBJECT:
- case MULTISET:
+ l = ADayTimeDurationSerializerDeserializer.getDayTime(bytes, startOffset + 1);
+ GregorianCalendarSystem.getInstance().getDurationExtendStringRepWithTimezoneUntilField(l, 0, sb);
+ builder.appendString(sb);
+ break;
+ case DURATION:
+ i = ADurationSerializerDeserializer.getYearMonth(bytes, startOffset + 1);
+ l = ADurationSerializerDeserializer.getDayTime(bytes, startOffset + 1);
+ GregorianCalendarSystem.getInstance().getDurationExtendStringRepWithTimezoneUntilField(l, i, sb);
+ builder.appendString(sb);
+ break;
case UUID:
+ AUUID.appendLiteralOnly(bytes, startOffset + 1, sb);
+ builder.appendString(sb);
+ break;
+ case BINARY:
+ int contentLength = ABinarySerializerDeserializer.getContentLength(bytes, startOffset + 1);
+ int metaLength = ABinarySerializerDeserializer.getMetaLength(contentLength);
+ Base64Printer.printBase64Binary(bytes, startOffset + 1 + metaLength, contentLength, sb);
+ builder.appendString(sb);
+ break;
default:
- throw new UnsupportedTypeException(sourceLoc, getIdentifier(), serString[offset]);
+ handleUnsupportedType(inputType, result);
+ return;
}
builder.finish();
+ resultStorage.reset();
out.write(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
out.write(baaos.getByteArray(), 0, baaos.getLength());
result.set(resultStorage);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
}
}
- protected abstract FunctionIdentifier getIdentifier();
+ @Override
+ protected BuiltinType getTargetType() {
+ return BuiltinType.ASTRING;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
new file mode 100644
index 0000000..765f491
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeConstructorEvaluator.java
@@ -0,0 +1,110 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.base.temporal.ATimeParserFactory;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractTimeConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableTime aTime = new AMutableTime(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ATime> timeSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ private final GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
+
+ protected AbstractTimeConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case TIME:
+ result.set(inputArg);
+ break;
+ case DATETIME:
+ long chronon = ADateTimeSerializerDeserializer.getChronon(bytes, startOffset + 1);
+ int chrononTime = cal.getTimeChronon(chronon);
+ aTime.setValue(chrononTime);
+ resultStorage.reset();
+ timeSerde.serialize(aTime, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (parseTime(utf8Ptr, aTime)) {
+ resultStorage.reset();
+ timeSerde.serialize(aTime, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+ int stringLength = textPtr.getUTF8Length();
+ // the string to be parsed should be at least 6 characters: hhmmss
+ if (stringLength < 6) {
+ return false;
+ }
+ try {
+ int chronon = ATimeParserFactory.parseTimePart(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+ stringLength);
+ if (chronon < 0) {
+ chronon += GregorianCalendarSystem.CHRONON_OF_DAY;
+ }
+ result.setValue(chronon);
+ return true;
+ } catch (HyracksDataException e) {
+ return false;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.ATIME;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeWithFormatConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeWithFormatConstructorEvaluator.java
new file mode 100644
index 0000000..f6e945c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractTimeWithFormatConstructorEvaluator.java
@@ -0,0 +1,86 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.temporal.AsterixTemporalTypeParseException;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractTimeWithFormatConstructorEvaluator extends AbstractTimeConstructorEvaluator {
+
+ private final IScalarEvaluator formatEval;
+ protected final IPointable formatArg = new VoidPointable();
+ private final UTF8StringPointable formatTextPtr = new UTF8StringPointable();
+ private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+ protected AbstractTimeWithFormatConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluatorFactory[] args,
+ SourceLocation sourceLoc) throws HyracksDataException {
+ super(ctx, args[0].createScalarEvaluator(ctx), sourceLoc);
+ formatEval = args[1].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ protected void evaluateInput(IFrameTupleReference tuple) throws HyracksDataException {
+ super.evaluateInput(tuple);
+ formatEval.evaluate(tuple, formatArg);
+ }
+
+ @Override
+ protected boolean checkAndSetMissingOrNull(IPointable result) throws HyracksDataException {
+ return PointableHelper.checkAndSetMissingOrNull(result, inputArg, formatArg);
+ }
+
+ @Override
+ protected boolean parseTime(UTF8StringPointable textPtr, AMutableTime result) {
+ byte[] formatBytes = formatArg.getByteArray();
+ int formatStartOffset = formatArg.getStartOffset();
+ int formatLength = formatArg.getLength();
+ if (formatBytes[formatStartOffset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ return false;
+ }
+ formatTextPtr.set(formatBytes, formatStartOffset + 1, formatLength - 1);
+ try {
+ if (DateTimeFormatUtils.getInstance().parseDateTime(aInt64, textPtr.getByteArray(),
+ textPtr.getCharStartOffset(), textPtr.getUTF8Length(), formatBytes,
+ formatTextPtr.getCharStartOffset(), formatTextPtr.getUTF8Length(),
+ DateTimeFormatUtils.DateTimeParseMode.TIME_ONLY, false)) {
+ result.setValue((int) aInt64.getLongValue());
+ return true;
+ } else {
+ return false;
+ }
+ } catch (AsterixTemporalTypeParseException e) {
+ // shouldn't happen
+ return false;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractUUIDFromStringConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractUUIDFromStringConstructorEvaluator.java
new file mode 100644
index 0000000..e429c1d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractUUIDFromStringConstructorEvaluator.java
@@ -0,0 +1,89 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableUUID;
+import org.apache.asterix.om.base.AUUID;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractUUIDFromStringConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableUUID uuid = new AMutableUUID();
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AUUID> uuidSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AUUID);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ protected AbstractUUIDFromStringConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case UUID:
+ result.set(inputArg);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (parseUUID(utf8Ptr, uuid)) {
+ resultStorage.reset();
+ uuidSerde.serialize(uuid, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.AUUID;
+ }
+
+ private static boolean parseUUID(UTF8StringPointable textPtr, AMutableUUID result) {
+ try {
+ // first byte: tag, next x bytes: length
+ result.parseUUIDHexBytes(textPtr.getByteArray(), textPtr.getCharStartOffset());
+ return true;
+ } catch (HyracksDataException e) {
+ return false;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractYearMonthDurationConstructorEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractYearMonthDurationConstructorEvaluator.java
new file mode 100644
index 0000000..5c39367
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AbstractYearMonthDurationConstructorEvaluator.java
@@ -0,0 +1,98 @@
+/*
+ * 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.runtime.evaluators.constructors;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableYearMonthDuration;
+import org.apache.asterix.om.base.AYearMonthDuration;
+import org.apache.asterix.om.base.temporal.ADurationParserFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractYearMonthDurationConstructorEvaluator extends AbstractConstructorEvaluator {
+
+ private final AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AYEARMONTHDURATION);
+ private final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+
+ protected AbstractYearMonthDurationConstructorEvaluator(IEvaluatorContext ctx, IScalarEvaluator inputEval,
+ SourceLocation sourceLoc) {
+ super(ctx, inputEval, sourceLoc);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ byte[] bytes = inputArg.getByteArray();
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ ATypeTag inputType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[startOffset]);
+ switch (inputType) {
+ case YEARMONTHDURATION:
+ result.set(inputArg);
+ break;
+ case DURATION:
+ int months = ADurationSerializerDeserializer.getYearMonth(bytes, startOffset + 1);
+ aYearMonthDuration.setMonths(months);
+ resultStorage.reset();
+ yearMonthDurationSerde.serialize(aYearMonthDuration, out);
+ result.set(resultStorage);
+ break;
+ case STRING:
+ utf8Ptr.set(bytes, startOffset + 1, len - 1);
+ if (parseYearMonthDuration(utf8Ptr, aYearMonthDuration)) {
+ resultStorage.reset();
+ yearMonthDurationSerde.serialize(aYearMonthDuration, out);
+ result.set(resultStorage);
+ } else {
+ handleParseError(utf8Ptr, result);
+ }
+ break;
+ default:
+ handleUnsupportedType(inputType, result);
+ break;
+ }
+ }
+
+ @Override
+ protected final BuiltinType getTargetType() {
+ return BuiltinType.AYEARMONTHDURATION;
+ }
+
+ private static boolean parseYearMonthDuration(UTF8StringPointable textPtr, AMutableYearMonthDuration result) {
+ try {
+ ADurationParserFactory.parseDuration(textPtr.getByteArray(), textPtr.getCharStartOffset(),
+ textPtr.getUTF8Length(), result, ADurationParserFactory.ADurationParseOption.YEAR_MONTH);
+ return true;
+ } catch (HyracksDataException e) {
+ return false;
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
index a0c28b9..ef9c77b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AndDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -45,12 +44,7 @@
public class AndDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AndDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AndDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
index 734966e..d2dcb9e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AnyCollectionMemberDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -44,12 +43,7 @@
public class AnyCollectionMemberDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AnyCollectionMemberDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = AnyCollectionMemberDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIfNullDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIfNullDescriptor.java
index aa0b567..2f66203 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIfNullDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIfNullDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -59,12 +58,7 @@
public class ArrayIfNullDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ArrayIfNullDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ArrayIfNullDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
index c27423b..6a0d6fb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CastTypeDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.functions.IFunctionTypeInferer;
-import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.functions.FunctionTypeInferers;
@@ -64,14 +63,6 @@
public void setImmutableStates(Object... states) {
reqType = (IAType) states[0];
inputType = (IAType) states[1];
- // If reqType or inputType is null, or they are the same, it indicates there is a bug in the compiler.
- if (reqType == null || inputType == null || reqType.equals(inputType)) {
- throw new IllegalStateException(
- "Invalid types for casting, required type " + reqType + ", input type " + inputType);
- }
- // NULLs and MISSINGs are handled by the generated code, therefore we only need to handle actual types here.
- this.reqType = TypeComputeUtils.getActualType(reqType);
- this.inputType = TypeComputeUtils.getActualType(inputType);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
index 88be867..ee6883e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CheckUnknownDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -42,12 +41,7 @@
public class CheckUnknownDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CheckUnknownDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CheckUnknownDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
index c45f90c..b92bc11 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateCircleDescriptor.java
@@ -30,7 +30,6 @@
import org.apache.asterix.om.base.AMutableCircle;
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,12 +51,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CreateCircleDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CreateCircleDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
index efcce7a..233d8f2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateLineDescriptor.java
@@ -30,7 +30,6 @@
import org.apache.asterix.om.base.AMutableLine;
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,12 +51,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CreateLineDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CreateLineDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
index 4048802..8266199 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateMBRDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.CreateMBREvalFactory;
@@ -31,11 +30,7 @@
public class CreateMBRDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CreateMBRDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CreateMBRDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
index 200c16c..a1c9b69 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePointDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutablePoint;
import org.apache.asterix.om.base.APoint;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -47,12 +46,7 @@
@MissingNullInOutFunction
public class CreatePointDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CreatePointDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CreatePointDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
index 2792f3c..11d776c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreatePolygonDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,12 +51,7 @@
public class CreatePolygonDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CreatePolygonDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CreatePolygonDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
index 054d477..779cb7b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateRectangleDescriptor.java
@@ -30,7 +30,6 @@
import org.apache.asterix.om.base.AMutableRectangle;
import org.apache.asterix.om.base.ARectangle;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,12 +51,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CreateRectangleDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CreateRectangleDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java
index e895ddb..ffa0155 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/CreateUUIDDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.base.AGeneratedUUID;
import org.apache.asterix.om.base.AUUID;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -42,12 +41,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CreateUUIDDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CreateUUIDDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetIntersectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetIntersectionDescriptor.java
new file mode 100644
index 0000000..e37fa05
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetIntersectionDescriptor.java
@@ -0,0 +1,157 @@
+/*
+ * 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.runtime.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class GetIntersectionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = () -> new GetIntersectionDescriptor();
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.GET_INTERSECTION;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ final IHyracksTaskContext hyracksTaskContext = ctx.getTaskContext();
+
+ return new IScalarEvaluator() {
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final IPointable inputArg0 = new VoidPointable();
+ private final IPointable inputArg1 = new VoidPointable();
+ private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+
+ private final AMutableRectangle aRectangle =
+ new AMutableRectangle(new AMutablePoint(0.0, 0.0), new AMutablePoint(0.0, 0.0));
+ private final AMutablePoint[] aPoint = { new AMutablePoint(0, 0), new AMutablePoint(0, 0) };
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ARectangle> rectangleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1)) {
+ return;
+ }
+
+ byte[] bytes0 = inputArg0.getByteArray();
+ byte[] bytes1 = inputArg1.getByteArray();
+
+ int offset0 = inputArg0.getStartOffset();
+ int offset1 = inputArg1.getStartOffset();
+
+ ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+
+ if (tag0 != ATypeTag.RECTANGLE) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ if (tag1 != ATypeTag.RECTANGLE) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+
+ double ax1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double ay1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double ax2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double ay2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+ double bx1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double by1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double bx2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double by2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+ // Bottom-left of the intersection rectangle
+ double ix1 = Math.max(ax1, bx1);
+ double iy1 = Math.max(ay1, by1);
+
+ // Top-right of the intersection rectangle
+ double ix2 = Math.min(ax2, bx2);
+ double iy2 = Math.min(ay2, by2);
+
+ // Update the intersection rectangle.
+ // If there is no intersection, return default rectangle [(0,0),(0,0)]
+ if ((ix1 < ix2) && (iy1 < iy2)) {
+ aPoint[0].setValue(ix1, iy1);
+ aPoint[1].setValue(ix2, iy2);
+ } else {
+ aPoint[0].setValue(0.0, 0.0);
+ aPoint[1].setValue(0.0, 0.0);
+ }
+ aRectangle.setValue(aPoint[0], aPoint[1]);
+ resultStorage.reset();
+ rectangleSerde.serialize(aRectangle, resultStorage.getDataOutput());
+ result.set(resultStorage);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index fb71eae..86204f0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -46,12 +45,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GetItemDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GetItemDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
index ce781fb..6201218 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetJobParameterByNameDescriptor.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -35,12 +34,7 @@
public class GetJobParameterByNameDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GetJobParameterByNameDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GetJobParameterByNameDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetTypeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetTypeDescriptor.java
index 843ef62..cbb253a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetTypeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetTypeDescriptor.java
@@ -51,9 +51,7 @@
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private DataOutput dataOutput = resultStorage.getDataOutput();
private IScalarEvaluator inputEval = args[0].createScalarEvaluator(ctx);
- final IPointable inputArg = new VoidPointable();
-
- private StringBuilder stringBuilder = new StringBuilder();
+ private final IPointable inputArg = new VoidPointable();
private final UTF8StringWriter writer = new UTF8StringWriter();
@Override
@@ -61,13 +59,11 @@
resultStorage.reset();
inputEval.evaluate(tuple, inputArg);
- byte[] bytes = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- stringBuilder.append(ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]].toString());
-
try {
+ byte[] bytes = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
dataOutput.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
- writer.writeUTF8(stringBuilder.toString(), dataOutput);
+ writer.writeUTF8(ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]].toString(), dataOutput);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
index 075c707..e70aaa8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/InjectFailureDescriptor.java
@@ -22,7 +22,6 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -45,12 +44,7 @@
private static final Logger LOGGER = LogManager.getLogger();
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new InjectFailureDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = InjectFailureDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
index f94bc32..6143f25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LenDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -48,12 +47,7 @@
@MissingNullInOutFunction
public class LenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new LenDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = LenDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
index bba8cac..4b4cbc2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NotDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -45,12 +44,7 @@
@MissingNullInOutFunction
public class NotDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NotDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NotDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
index 15ce5f8..146e89d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericACosDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -33,12 +32,7 @@
@MissingNullInOutFunction
public class NumericACosDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericACosDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericACosDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
index 6e48b90..3f132a1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericASinDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericASinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericASinDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericASinDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java
index 815b479..d042cdc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATan2Descriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -48,12 +47,7 @@
public class NumericATan2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericATan2Descriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericATan2Descriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
index bb8fd12..99acea7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericATanDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericATanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericATanDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericATanDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
index 4140410..d988982 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAbsDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -38,12 +37,7 @@
@MissingNullInOutFunction
public class NumericAbsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericAbsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericAbsDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
index f82cb7c..d0691f5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericAddDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.exceptions.OverflowException;
@@ -36,11 +35,7 @@
public class NumericAddDescriptor extends AbstractNumericArithmeticEval {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericAddDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericAddDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
index 209ae75..6e6cbdc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCeilingDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
public class NumericCeilingDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericCeilingDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericCeilingDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
index a5056ae..0057445 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCosDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericCosDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericCosDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericCosDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCoshDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCoshDescriptor.java
index 9ad9daa..40b784e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCoshDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCoshDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericCoshDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericCoshDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericCoshDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
index 96d3f9e..173d41e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericDegreesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericDegreesDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericDegreesDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
index dbd5251..8215d8e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.exceptions.OverflowException;
@@ -36,11 +35,7 @@
public class NumericDivDescriptor extends AbstractNumericArithmeticEval {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericDivDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericDivDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index b23c2ac..a03792a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,11 +33,7 @@
public class NumericDivideDescriptor extends AbstractNumericArithmeticEval {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericDivideDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericDivideDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
index 20054cd..6468cd7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericExpDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericExpDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericExpDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericExpDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
index b1f19d7..fbfc738 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericFloorDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -38,12 +37,7 @@
@MissingNullInOutFunction
public class NumericFloorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericFloorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericFloorDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
index dfd6a8d..0e3c8c9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLnDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericLnDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericLnDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericLnDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
index e12b948..db4a4f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericLogDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericLogDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericLogDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericLogDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
index 3932eda..a6b6db4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericModuloDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,13 +33,7 @@
public class NumericModuloDescriptor extends AbstractNumericArithmeticEval {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericModuloDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericModuloDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
index 7bfb983..95410d5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericMultiplyDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.exceptions.OverflowException;
@@ -35,11 +34,7 @@
public class NumericMultiplyDescriptor extends AbstractNumericArithmeticEval {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericMultiplyDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericMultiplyDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
index be97a65..ecb003b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.exceptions.OverflowException;
@@ -38,11 +37,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericPowerDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericPowerDescriptor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
index 38b07ce..7bab807 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericRadiansDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericRadiansDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericRadiansDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
index b1879a6..944850a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfToEvenDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -39,12 +38,7 @@
@MissingNullInOutFunction
public class NumericRoundHalfToEvenDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericRoundHalfToEvenDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericRoundHalfToEvenDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfUp2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfUp2Descriptor.java
new file mode 100644
index 0000000..2ebc302
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRoundHalfUp2Descriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.evaluators.functions;
+
+import java.math.BigDecimal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public class NumericRoundHalfUp2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = NumericRoundHalfUp2Descriptor::new;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NUMERIC_ROUND_HALF_UP2;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ return new NumericRoundTruncEvaluator(ctx, args, BigDecimal.ROUND_HALF_UP, getIdentifier(), sourceLoc);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
index 05d5723..ee65375 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSignDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericSignDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericSignDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericSignDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
index 2d9e578..b5a22ad 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericSinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericSinDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericSinDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinhDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinhDescriptor.java
index d8d610e..6d66a2c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinhDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSinhDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericSinhDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericSinhDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericSinhDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
index fe1e673..e4206ed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSqrtDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericSqrtDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericSqrtDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericSqrtDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
index 6305074..275dfca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.exceptions.OverflowException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,11 +33,7 @@
public class NumericSubDescriptor extends AbstractNumericArithmeticEval {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericSubDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericSubDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
index e1aaf38..bc69fac 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericSubtractDescriptor.java
@@ -35,7 +35,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.AMutableInt8;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -57,12 +56,7 @@
public class NumericSubtractDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericSubtractDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericSubtractDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
index ececf63..c20c46e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericTanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericTanDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericTanDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanhDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanhDescriptor.java
index 2d569b7..923e9c9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanhDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericTanhDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
@MissingNullInOutFunction
public class NumericTanhDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericTanhDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericTanhDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
index 29a16d9..06a5b76 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericUnaryMinusDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
public class NumericUnaryMinusDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericUnaryMinusDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = NumericUnaryMinusDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
index 71069f8..091fd4b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/OrDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -45,12 +44,7 @@
public class OrDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new OrDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = OrDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
index 01aa181..7a017bd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
@@ -170,43 +170,49 @@
// 1 pointable check
public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1)
throws HyracksDataException {
- return checkAndSetMissingOrNull(result, null, pointable1, null, null, null);
+ return checkAndSetMissingOrNull(result, null, pointable1, null, null, null, null);
}
// 2 pointables check
public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2)
throws HyracksDataException {
- return checkAndSetMissingOrNull(result, null, pointable1, pointable2, null, null);
+ return checkAndSetMissingOrNull(result, null, pointable1, pointable2, null, null, null);
}
// 3 pointables check
public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2,
IPointable pointable3) throws HyracksDataException {
- return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, null);
+ return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, null, null);
}
// 4 pointables check
public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2,
IPointable pointable3, IPointable pointable4) throws HyracksDataException {
- return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, pointable4);
+ return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, pointable4, null);
+ }
+
+ // 5 pointables check
+ public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2,
+ IPointable pointable3, IPointable pointable4, IPointable pointable5) throws HyracksDataException {
+ return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, pointable4, pointable5);
}
// 1 pointable check (check list members for missing values)
public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1)
throws HyracksDataException {
- return checkAndSetMissingOrNull(result, listAccessor, pointable1, null, null, null);
+ return checkAndSetMissingOrNull(result, listAccessor, pointable1, null, null, null, null);
}
// 2 pointables check (check list members for missing values)
public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1,
IPointable pointable2) throws HyracksDataException {
- return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, null, null);
+ return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, null, null, null);
}
// 3 pointables check (check list members for missing values)
public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1,
IPointable pointable2, IPointable pointable3) throws HyracksDataException {
- return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, pointable3, null);
+ return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, pointable3, null, null);
}
/**
@@ -227,11 +233,13 @@
* @param pointable2 the second pointable to be checked
* @param pointable3 the third pointable to be checked
* @param pointable4 the fourth pointable to be checked
+ * @param pointable5 the fourth pointable to be checked
*
* @return {@code true} if the pointable value is missing or null, {@code false} otherwise.
*/
public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1,
- IPointable pointable2, IPointable pointable3, IPointable pointable4) throws HyracksDataException {
+ IPointable pointable2, IPointable pointable3, IPointable pointable4, IPointable pointable5)
+ throws HyracksDataException {
// this flag will keep an eye on whether a null value is encountered or not
boolean isMeetNull = false;
@@ -278,6 +286,17 @@
}
}
+ if (pointable5 != null) {
+ switch (getPointableValueState(pointable5, listAccessor)) {
+ case MISSING:
+ setMissing(result);
+ return true;
+ case NULL:
+ isMeetNull = true;
+ break;
+ }
+ }
+
// this is reached only if no missing is encountered in all the passed pointables
if (isMeetNull) {
setNull(result);
@@ -289,6 +308,41 @@
}
/**
+ * Checks whether the pointable {@param pointable1} is null or missing, and if true, assigns null to the
+ * {@param result}.
+ *
+ * @param result the result pointable that will hold the null value
+ * @param pointable1 the pointable to be checked
+ *
+ * @return {@code true} if the {@param pointable1} value is missing or null, {@code false} otherwise.
+ */
+ public static boolean checkAndSetNull(IPointable result, IPointable pointable1) throws HyracksDataException {
+ switch (getPointableValueState(pointable1, null)) {
+ case MISSING:
+ case NULL:
+ setNull(result);
+ return true;
+ default:
+ return false;
+ }
+ }
+
+ /**
+ * Checks whether any pointable argument is null or missing, and if true, assigns null to the
+ * {@param result}.
+ *
+ * @param result the result pointable that will hold the null value
+ * @param pointable1 the pointable to be checked
+ * @param pointable2 the pointable to be checked
+ *
+ * @return {@code true} if any pointable is missing or null, {@code false} otherwise.
+ */
+ public static boolean checkAndSetNull(IPointable result, IPointable pointable1, IPointable pointable2)
+ throws HyracksDataException {
+ return checkAndSetNull(result, pointable1) || checkAndSetNull(result, pointable2);
+ }
+
+ /**
* This method checks and returns the pointable value state.
*
* If a ListAccessor is passed to this function, it will check if the passed pointable is a list, and if so, it
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
index 82289ed..8184434 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
@@ -20,7 +20,6 @@
package org.apache.asterix.runtime.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RandomHelper;
@@ -35,12 +34,7 @@
public class RandomDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new RandomDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = RandomDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
index bd6bd3d..c47253b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -43,12 +42,7 @@
public class RandomWithSeedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new RandomWithSeedDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = RandomWithSeedDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ReferenceTileDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ReferenceTileDescriptor.java
new file mode 100644
index 0000000..c55aa75
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ReferenceTileDescriptor.java
@@ -0,0 +1,176 @@
+/*
+ * 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.runtime.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ReferenceTileDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ReferenceTileDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.REFERENCE_TILE;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ return new IScalarEvaluator() {
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final IPointable inputArg0 = new VoidPointable();
+ private final IPointable inputArg1 = new VoidPointable();
+ private final IPointable inputArg2 = new VoidPointable();
+ private final IPointable inputArg3 = new VoidPointable();
+ private final IPointable inputArg4 = new VoidPointable();
+ private final IPointable inputArg5 = new VoidPointable();
+ private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval2 = args[2].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval3 = args[3].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval4 = args[4].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval5 = args[5].createScalarEvaluator(ctx);
+
+ private final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer intSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+ eval2.evaluate(tuple, inputArg2);
+ eval3.evaluate(tuple, inputArg3);
+ eval4.evaluate(tuple, inputArg4);
+ eval5.evaluate(tuple, inputArg5);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1, inputArg2, inputArg3,
+ inputArg4)) {
+ return;
+ }
+
+ byte[] bytes0 = inputArg0.getByteArray();
+ byte[] bytes1 = inputArg1.getByteArray();
+ byte[] bytes2 = inputArg2.getByteArray();
+ byte[] bytes3 = inputArg3.getByteArray();
+ byte[] bytes4 = inputArg4.getByteArray();
+
+ int offset0 = inputArg0.getStartOffset();
+ int offset1 = inputArg1.getStartOffset();
+ int offset2 = inputArg2.getStartOffset();
+ int offset3 = inputArg3.getStartOffset();
+ int offset4 = inputArg4.getStartOffset();
+
+ ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[offset2]);
+
+ if (tag0 != ATypeTag.RECTANGLE) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ if (tag1 != ATypeTag.RECTANGLE) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ if (tag2 != ATypeTag.RECTANGLE) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes2[offset2],
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+
+ double ax1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double ay1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double bx1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double by1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double minX = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double minY = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double maxX = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double maxY = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+ int rows = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes3, offset3);
+ int columns = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes4, offset4);
+
+ // Compute the reference point
+ double x = Math.max(ax1, bx1);
+ double y = Math.max(ay1, by1);
+
+ // Compute the tile ID of the reference point
+ int row = (int) Math.ceil((y - minY) * rows / (maxY - minY));
+ int col = (int) Math.ceil((x - minX) * columns / (maxX - minX));
+
+ row = Math.min(Math.max(1, row), rows * columns);
+ col = Math.min(Math.max(1, col), rows * columns);
+
+ int tileId = (row - 1) * columns + col;
+ resultStorage.reset();
+ aInt32.setValue(tileId);
+ intSerde.serialize(aInt32, resultStorage.getDataOutput());
+ result.set(resultStorage);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
index 10ecb1f..b7164a1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialAreaDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -50,12 +49,7 @@
public class SpatialAreaDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SpatialAreaDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SpatialAreaDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
index d0c40c9..3753f58 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
@@ -30,7 +30,6 @@
import org.apache.asterix.om.base.AMutableRectangle;
import org.apache.asterix.om.base.ARectangle;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -51,12 +50,7 @@
public class SpatialCellDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SpatialCellDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SpatialCellDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
index 3c428e8..af858f4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialDistanceDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -45,12 +44,7 @@
public class SpatialDistanceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SpatialDistanceDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SpatialDistanceDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
index b4d8ab6..eeec70f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
public class StringContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringContainsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringContainsDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
index 0c952cf..333eb5a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
public class StringEndsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringEndsWithDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringEndsWithDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
index bbe7c59..4bee3ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
public class StringEqualDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringEqualDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringEqualDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
index b201c80..a385931 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringInitCapDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -38,12 +37,7 @@
public class StringInitCapDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringInitCapDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringInitCapDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
index 6aebf6b..9f561d4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrim2Descriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
@@ -38,12 +37,7 @@
public class StringLTrim2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringLTrim2Descriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringLTrim2Descriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
index 4d4fd93..d86287f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLTrimDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
@@ -39,12 +38,7 @@
public class StringLTrimDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringLTrimDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringLTrimDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
index a4fd667..47caf14 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -47,12 +46,7 @@
@MissingNullInOutFunction
public class StringLengthDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringLengthDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringLengthDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
index f37a385..b1c60bd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -22,7 +22,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -37,12 +36,7 @@
public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringLowerCaseDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringLowerCaseDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
index bf463a1..2817652 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrim2Descriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
@@ -38,12 +37,7 @@
public class StringRTrim2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringRTrim2Descriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringRTrim2Descriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
index e99c401..5f04340 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRTrimDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
@@ -39,12 +38,7 @@
public class StringRTrimDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringRTrimDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringRTrimDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
index 8a6587e..b5544cb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
@@ -36,12 +35,7 @@
public class StringRegExpLikeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringRegExpLikeDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringRegExpLikeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
index 046f80e..c8f8c61 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpLikeWithFlagDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
@@ -36,12 +35,7 @@
public class StringRegExpLikeWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringRegExpLikeWithFlagDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringRegExpLikeWithFlagDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
index e240bfa..f0f547f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringReplacer;
@@ -36,12 +35,7 @@
@MissingNullInOutFunction
public class StringReplaceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringReplaceDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringReplaceDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
index 6d81e59..bc514da 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -37,12 +36,7 @@
@MissingNullInOutFunction
public class StringReverseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringReverseDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringReverseDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
index aef92c5..2a56a57 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringSplitDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
@@ -49,12 +48,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringSplitDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringSplitDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
index 41eeeca..caa1e64 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringStartsWithDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
public class StringStartsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringStartsWithDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringStartsWithDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
index 8d03352..2f6a223 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringToCodePointDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
@@ -51,12 +50,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringToCodePointDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringToCodePointDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
index 3fad596..60c6c04 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrim2Descriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
@@ -38,12 +37,7 @@
public class StringTrim2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringTrim2Descriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringTrim2Descriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
index 3f0ec52..c252037 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringTrimDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.StringTrimmer;
@@ -39,12 +38,7 @@
public class StringTrimDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringTrimDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringTrimDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
index 102892b..300dd86 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringUpperCaseDescriptor.java
@@ -22,7 +22,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -37,12 +36,7 @@
public class StringUpperCaseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new StringUpperCaseDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = StringUpperCaseDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Eval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Eval.java
index 124ae40..e5a6853 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Eval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Eval.java
@@ -96,7 +96,7 @@
string.set(bytes, offset + 1, len - 1);
array.reset();
try {
- int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+ int actualStart = start >= 0 ? Math.max(start - baseOffset, 0) : string.getStringLength() + start;
boolean success = UTF8StringPointable.substr(string, actualStart, Integer.MAX_VALUE, builder, array);
if (success) {
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
index 44c1094..e106a1c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringAfterDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -44,12 +43,7 @@
@MissingNullInOutFunction
public class SubstringAfterDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SubstringAfterDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SubstringAfterDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
index 4da815f..6eef5ce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringBeforeDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -44,12 +43,7 @@
@MissingNullInOutFunction
public class SubstringBeforeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SubstringBeforeDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SubstringBeforeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringEval.java
index e8c2ff8..dec6cce 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringEval.java
@@ -108,7 +108,7 @@
string.set(bytes, offset + 1, length - 1);
array.reset();
try {
- int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+ int actualStart = start >= 0 ? Math.max(start - baseOffset, 0) : string.getStringLength() + start;
boolean success = UTF8StringPointable.substr(string, actualStart, len, builder, array);
if (success) {
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
index 3e1eb49..fbf52f2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SwitchCaseDescriptor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.runtime.evaluators.functions;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,12 +33,7 @@
public class SwitchCaseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SwitchCaseDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SwitchCaseDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
index 6040e1c..b1edd2c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBigIntDescriptor.java
@@ -19,16 +19,10 @@
package org.apache.asterix.runtime.evaluators.functions;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.types.hierachy.ITypeConvertComputer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AbstractInt64ConstructorEvaluator;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -41,12 +35,7 @@
@MissingNullInOutFunction
public class ToBigIntDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ToBigIntDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ToBigIntDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -55,54 +44,23 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractInt64ConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
+ return new AbstractInt64ConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- protected void evaluateImpl(IPointable result) throws IOException {
- byte[] bytes = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- switch (tt) {
- case BOOLEAN:
- boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
- aInt64.setValue(b ? 1 : 0);
- INT64_SERDE.serialize(aInt64, out);
- result.set(resultStorage);
- break;
-
- case TINYINT:
- case SMALLINT:
- case INTEGER:
- ITypeConvertComputer tpcc = ATypeHierarchy.getTypePromoteComputer(tt, ATypeTag.BIGINT);
- tpcc.convertType(bytes, offset + 1, inputArg.getLength() - 1, out);
- result.set(resultStorage);
- break;
-
- case FLOAT:
- case DOUBLE:
- ITypeConvertComputer tdcc =
- ATypeHierarchy.getTypeDemoteComputer(tt, ATypeTag.BIGINT, false);
- tdcc.convertType(bytes, offset + 1, inputArg.getLength() - 1, out);
- result.set(resultStorage);
- break;
-
+ protected void handleUnsupportedType(ATypeTag inputType, IPointable result)
+ throws HyracksDataException {
+ switch (inputType) {
case ARRAY:
case MULTISET:
case OBJECT:
PointableHelper.setNull(result);
break;
-
default:
- super.evaluateImpl(result);
+ super.handleUnsupportedType(inputType, result);
break;
}
}
@Override
- protected void handleUnparseableString(IPointable result) {
- PointableHelper.setNull(result);
- }
-
- @Override
protected FunctionIdentifier getIdentifier() {
return ToBigIntDescriptor.this.getIdentifier();
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
index 594b6d8..4954ece 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToBooleanDescriptor.java
@@ -20,21 +20,13 @@
package org.apache.asterix.runtime.evaluators.functions;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.NumberUtils;
import org.apache.asterix.runtime.evaluators.constructors.AbstractBooleanConstructorEvaluator;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
@@ -48,12 +40,7 @@
public class ToBooleanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ToBooleanDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ToBooleanDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -62,61 +49,35 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractBooleanConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
+ return new AbstractBooleanConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ protected void handleUnsupportedType(ATypeTag inputType, IPointable result)
+ throws HyracksDataException {
byte[] bytes = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
-
- ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- switch (tt) {
- case TINYINT:
- setInteger(AInt8SerializerDeserializer.getByte(bytes, offset + 1), result);
- break;
- case SMALLINT:
- setInteger(AInt16SerializerDeserializer.getShort(bytes, offset + 1), result);
- break;
- case INTEGER:
- setInteger(AInt32SerializerDeserializer.getInt(bytes, offset + 1), result);
- break;
- case BIGINT:
- setInteger(AInt64SerializerDeserializer.getLong(bytes, offset + 1), result);
- break;
- case FLOAT:
- setDouble(AFloatSerializerDeserializer.getFloat(bytes, offset + 1), result);
- break;
- case DOUBLE:
- setDouble(ADoubleSerializerDeserializer.getDouble(bytes, offset + 1), result);
- break;
- case STRING:
- setInteger(UTF8StringUtil.getStringLength(bytes, offset + 1), result);
- break;
+ int startOffset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+ switch (inputType) {
case ARRAY:
- setInteger(AOrderedListSerializerDeserializer.getNumberOfItems(bytes, offset), result);
+ int n = AOrderedListSerializerDeserializer.getNumberOfItems(bytes, startOffset);
+ setInteger(n, result);
break;
case MULTISET:
- setInteger(AUnorderedListSerializerDeserializer.getNumberOfItems(bytes, offset),
- result);
+ n = AUnorderedListSerializerDeserializer.getNumberOfItems(bytes, startOffset);
+ setInteger(n, result);
break;
case OBJECT:
- setBoolean(result, !ARecordSerializerDeserializer.hasNoFields(bytes, offset,
- inputArg.getLength()));
+ boolean empty = ARecordSerializerDeserializer.hasNoFields(bytes, startOffset, len);
+ setBoolean(result, !empty);
break;
default:
- super.evaluateImpl(result);
+ super.handleUnsupportedType(inputType, result);
break;
}
}
- private void setInteger(long v, IPointable result) throws HyracksDataException {
- setBoolean(result, v != 0);
- }
-
- private void setDouble(double v, IPointable result) throws HyracksDataException {
- long bits = Double.doubleToLongBits(v);
- boolean zeroOrNaN = bits == NumberUtils.POSITIVE_ZERO_BITS
- || bits == NumberUtils.NEGATIVE_ZERO_BITS || bits == NumberUtils.NAN_BITS;
- setBoolean(result, !zeroOrNaN);
+ @Override
+ protected Boolean parseBoolean(byte[] bytes, int offset, int len) {
+ return UTF8StringUtil.getStringLength(bytes, offset + 1) != 0;
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
index 70ed398..196afb6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToDoubleDescriptor.java
@@ -19,16 +19,10 @@
package org.apache.asterix.runtime.evaluators.functions;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.types.hierachy.ITypeConvertComputer;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AbstractDoubleConstructorEvaluator;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -41,12 +35,7 @@
@MissingNullInOutFunction
public class ToDoubleDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ToDoubleDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ToDoubleDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -55,48 +44,23 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractDoubleConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
+ return new AbstractDoubleConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- protected void evaluateImpl(IPointable result) throws IOException {
- byte[] bytes = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- switch (tt) {
- case BOOLEAN:
- boolean b = ABooleanSerializerDeserializer.getBoolean(bytes, offset + 1);
- aDouble.setValue(b ? 1 : 0);
- DOUBLE_SERDE.serialize(aDouble, out);
- result.set(resultStorage);
- break;
-
- case TINYINT:
- case SMALLINT:
- case INTEGER:
- case BIGINT:
- case FLOAT:
- ITypeConvertComputer tcc = ATypeHierarchy.getTypePromoteComputer(tt, ATypeTag.DOUBLE);
- tcc.convertType(bytes, offset + 1, inputArg.getLength() - 1, out);
- result.set(resultStorage);
- break;
-
+ protected void handleUnsupportedType(ATypeTag inputType, IPointable result)
+ throws HyracksDataException {
+ switch (inputType) {
case ARRAY:
case MULTISET:
case OBJECT:
PointableHelper.setNull(result);
break;
-
default:
- super.evaluateImpl(result);
+ super.handleUnsupportedType(inputType, result);
break;
}
}
@Override
- protected void handleUparseableString(IPointable result) {
- PointableHelper.setNull(result);
- }
-
- @Override
protected FunctionIdentifier getIdentifier() {
return ToDoubleDescriptor.this.getIdentifier();
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
index a5e66af..5a16efc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToNumberDescriptor.java
@@ -29,13 +29,13 @@
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.common.NumberUtils;
import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.commons.lang3.mutable.MutableBoolean;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
@@ -51,12 +51,7 @@
@MissingNullInOutFunction
public class ToNumberDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ToNumberDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ToNumberDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -73,6 +68,7 @@
final AMutableInt64 aInt64 = new AMutableInt64(0);
final AMutableDouble aDouble = new AMutableDouble(0);
final UTF8StringPointable utf8Ptr = new UTF8StringPointable();
+ final MutableBoolean maybeNumeric = new MutableBoolean();
@SuppressWarnings("unchecked")
final ISerializerDeserializer<AInt64> INT64_SERDE =
@@ -114,10 +110,10 @@
case STRING:
utf8Ptr.set(bytes, startOffset + 1, inputArg.getLength() - 1);
- if (NumberUtils.parseInt64(utf8Ptr, aInt64)) {
+ if (NumberUtils.parseInt64(utf8Ptr, aInt64, maybeNumeric)) {
INT64_SERDE.serialize(aInt64, out);
result.set(resultStorage);
- } else if (NumberUtils.parseDouble(utf8Ptr, aDouble)) {
+ } else if (maybeNumeric.booleanValue() && NumberUtils.parseDouble(utf8Ptr, aDouble)) {
DOUBLE_SERDE.serialize(aDouble, out);
result.set(resultStorage);
} else {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java
index 7c57c1e..9d0373b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToObjectDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -42,12 +41,7 @@
public class ToObjectDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = -4146417699529927812L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ToObjectDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ToObjectDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
index 6635082..84ce9d7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ToStringDescriptor.java
@@ -19,35 +19,23 @@
package org.apache.asterix.runtime.evaluators.functions;
-import java.io.IOException;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AbstractStringConstructorEvaluator;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
@MissingNullInOutFunction
public class ToStringDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ToStringDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ToStringDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -56,33 +44,21 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new AbstractStringConstructorEvaluator(args[0].createScalarEvaluator(ctx), sourceLoc) {
- @SuppressWarnings("unchecked")
- private final ISerializerDeserializer<ANull> nullSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-
+ return new AbstractStringConstructorEvaluator(ctx, args[0].createScalarEvaluator(ctx), sourceLoc) {
@Override
- protected void evaluateImpl(IPointable result) throws IOException {
- byte[] bytes = inputArg.getByteArray();
- int offset = inputArg.getStartOffset();
- ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- switch (tt) {
+ protected void handleUnsupportedType(ATypeTag inputType, IPointable result)
+ throws HyracksDataException {
+ switch (inputType) {
case ARRAY:
case MULTISET:
case OBJECT:
- setNull(result);
+ PointableHelper.setNull(result);
break;
default:
- super.evaluateImpl(result);
- break;
+ super.handleUnsupportedType(inputType, result);
}
}
- private void setNull(IPointable result) throws HyracksDataException {
- nullSerde.serialize(ANull.NULL, out);
- result.set(resultStorage);
- }
-
@Override
protected FunctionIdentifier getIdentifier() {
return ToStringDescriptor.this.getIdentifier();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
index aa42ede..2fee7e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -49,12 +48,7 @@
@MissingNullInOutFunction
public class TreatAsIntegerDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TreatAsIntegerDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TreatAsIntegerDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java
index 3076358..5437222 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/UUIDDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.base.AGeneratedUUID;
import org.apache.asterix.om.base.AUUID;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -43,12 +42,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new UUIDDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = UUIDDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
index 907b591..1b115e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryConcatDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AMissing;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,12 +49,7 @@
public class BinaryConcatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new BinaryConcatDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = BinaryConcatDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
@@ -100,10 +94,10 @@
}
try {
listAccessor.reset(data, offset);
+ boolean itemsAreSelfDescribing = listAccessor.itemsAreSelfDescribing();
int concatLength = 0;
boolean itemIsNull = false;
for (int i = 0; i < listAccessor.size(); i++) {
- // TODO(ali): itemOffset should be adjusted if list is heterogeneous like string_concat
int itemOffset = listAccessor.getItemOffset(i);
ATypeTag itemType = listAccessor.getItemType(itemOffset);
if (itemType != ATypeTag.BINARY) {
@@ -119,6 +113,9 @@
throw new UnsupportedItemTypeException(sourceLoc, getIdentifier(),
itemType.serialize());
}
+ if (itemsAreSelfDescribing) {
+ itemOffset++;
+ }
concatLength += ByteArrayPointable.getContentLength(data, itemOffset);
}
if (itemIsNull) {
@@ -132,6 +129,9 @@
for (int i = 0; i < listAccessor.size(); i++) {
int itemOffset = listAccessor.getItemOffset(i);
+ if (itemsAreSelfDescribing) {
+ itemOffset++;
+ }
int length = ByteArrayPointable.getContentLength(data, itemOffset);
dataOutput.write(data,
itemOffset + ByteArrayPointable.getNumberBytesToStoreMeta(length), length);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
index 7305fbf..7649ba5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/BinaryLengthDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -43,12 +42,7 @@
@MissingNullInOutFunction
public class BinaryLengthDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new BinaryLengthDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = BinaryLengthDescriptor::new;
private static final ATypeTag[] EXPECTED_TAGS = { ATypeTag.BINARY };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
index 83be52c..dea07fb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/ParseBinaryDescriptor.java
@@ -24,7 +24,6 @@
import org.apache.asterix.om.base.ABinary;
import org.apache.asterix.om.base.AMutableBinary;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -49,12 +48,7 @@
private static final UTF8StringPointable HEX_FORMAT = UTF8StringPointable.generateUTF8Pointable("hex");
private static final UTF8StringPointable BASE64_FORMAT = UTF8StringPointable.generateUTF8Pointable("base64");
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ParseBinaryDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = ParseBinaryDescriptor::new;
public static final ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.STRING, ATypeTag.STRING };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
index ee303ba..521c010 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/PrintBinaryDescriptor.java
@@ -23,7 +23,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -53,12 +52,7 @@
return BuiltinFunctions.PRINT_BINARY;
}
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PrintBinaryDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = PrintBinaryDescriptor::new;
public final static ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.BINARY, ATypeTag.STRING };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
index c5c7c97..10670dc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -30,11 +29,7 @@
public class FieldAccessByNameDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new FieldAccessByNameDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = FieldAccessByNameDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
index 401335c..b617085 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatDescriptor.java
@@ -46,18 +46,20 @@
}
};
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private ARecordType[] argTypes;
+ private ARecordType listItemRecordType;
@Override
public void setImmutableStates(Object... states) {
- argTypes = (ARecordType[]) states;
+ argTypes = (ARecordType[]) states[0];
+ listItemRecordType = (ARecordType) states[1];
}
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
- return new RecordConcatEvalFactory(args, argTypes, false, sourceLoc);
+ return new RecordConcatEvalFactory(args, argTypes, listItemRecordType, false, sourceLoc);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
index c4bc87e..bc1cacc 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatEvalFactory.java
@@ -33,6 +33,7 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
import org.apache.asterix.runtime.evaluators.functions.BinaryHashMap;
import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -49,20 +50,23 @@
class RecordConcatEvalFactory implements IScalarEvaluatorFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private final IScalarEvaluatorFactory[] args;
private final ARecordType[] argTypes;
+ private final ARecordType listItemRecordType;
+
private final boolean failOnArgTypeMismatch;
private final SourceLocation sourceLoc;
- RecordConcatEvalFactory(IScalarEvaluatorFactory[] args, ARecordType[] argTypes, boolean failOnArgTypeMismatch,
- SourceLocation sourceLoc) {
+ RecordConcatEvalFactory(IScalarEvaluatorFactory[] args, ARecordType[] argTypes, ARecordType listItemRecordType,
+ boolean failOnArgTypeMismatch, SourceLocation sourceLoc) {
this.args = args;
this.argTypes = argTypes;
+ this.listItemRecordType = listItemRecordType;
this.failOnArgTypeMismatch = failOnArgTypeMismatch;
this.sourceLoc = sourceLoc;
}
@@ -81,9 +85,16 @@
private static final int TABLE_FRAME_SIZE = 32768;
private static final int TABLE_SIZE = 100;
+ private ListAccessor listAccessor;
+ private ARecordVisitablePointable itemRecordPointable;
+ private final ArrayBackedValueStorage itemRecordStorage;
+ private boolean itemRecordCastRequired;
+
+ private ArgKind argKind;
+ private final IPointable firstArg;
private final IScalarEvaluator[] argEvals;
- private final IPointable[] argPointables;
- private final ARecordVisitablePointable[] argRecordPointables;
+ private IPointable[] argPointables;
+ private ARecordVisitablePointable[] argRecordPointables;
private final ARecordVisitablePointable openRecordPointable;
private final BitSet castRequired;
@@ -98,11 +109,12 @@
private final BinaryEntry keyEntry;
private final BinaryEntry valEntry;
+ private int numRecords;
+
private RecordConcatEvaluator(IScalarEvaluator[] argEvals) {
this.argEvals = argEvals;
- argPointables = new IPointable[args.length];
- argRecordPointables = new ARecordVisitablePointable[args.length];
+ firstArg = new VoidPointable();
openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
resultStorage = new ArrayBackedValueStorage();
@@ -117,45 +129,101 @@
valEntry.set(new byte[0], 0, 0);
castRequired = new BitSet();
- for (int i = 0; i < args.length; i++) {
- argPointables[i] = new VoidPointable();
- ARecordType argType = argTypes[i];
- if (argType != null) {
- argRecordPointables[i] = new ARecordVisitablePointable(argType);
- if (hasDerivedType(argType.getFieldTypes())) {
- castRequired.set(i);
- if (castVisitor == null) {
- castVisitor = new ACastVisitor();
- castVisitorArg = new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(),
- Boolean.FALSE);
+ itemRecordStorage = new ArrayBackedValueStorage();
+ if (listItemRecordType != null) {
+ // init if we know we will always get one list of records whose type is known at compile-time
+ itemRecordPointable = new ARecordVisitablePointable(listItemRecordType);
+ if (hasDerivedType(listItemRecordType.getFieldTypes())) {
+ itemRecordCastRequired = true;
+ initCastVisitor();
+ }
+ } else {
+ // otherwise, any kind of arguments are possible (and possibly a single open list of records)
+ argPointables = new IPointable[args.length];
+ argRecordPointables = new ARecordVisitablePointable[args.length];
+ for (int i = 0; i < args.length; i++) {
+ argPointables[i] = new VoidPointable();
+ ARecordType argType = argTypes[i];
+ if (argType != null) {
+ argRecordPointables[i] = new ARecordVisitablePointable(argType);
+ if (hasDerivedType(argType.getFieldTypes())) {
+ castRequired.set(i);
+ initCastVisitor();
}
}
}
}
}
+ private void initCastVisitor() {
+ if (castVisitor == null) {
+ castVisitor = new ACastVisitor();
+ castVisitorArg =
+ new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
+ }
+ }
+
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
resultStorage.reset();
- if (validateArgs(tuple)) {
- processArgs();
+ if (args.length == 0) {
+ writeTypeTag(ATypeTag.SERIALIZED_NULL_TYPE_TAG, result);
+ return;
}
+ if (!validateArgs(tuple, result)) {
+ return;
+ }
+ processArgs();
result.set(resultStorage);
}
- private boolean validateArgs(IFrameTupleReference tuple) throws HyracksDataException {
- if (args.length == 0) {
- writeTypeTag(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
- return false;
+ private boolean validateArgs(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ if (argEvals.length == 1) {
+ // either 1 list arg or 1 presumably record arg
+ argEvals[0].evaluate(tuple, firstArg);
+ byte[] data = firstArg.getByteArray();
+ int offset = firstArg.getStartOffset();
+ ATypeTag typeTag = ATypeTag.VALUE_TYPE_MAPPING[data[offset]];
+ if (typeTag.isListType()) {
+ if (listAccessor == null) {
+ listAccessor = new ListAccessor();
+ }
+ listAccessor.reset(data, offset);
+ argKind = ArgKind.SINGLE_ARG_LIST;
+ numRecords = listAccessor.size();
+ if (numRecords == 0) {
+ writeTypeTag(ATypeTag.SERIALIZED_NULL_TYPE_TAG, result);
+ return false;
+ }
+ } else {
+ argKind = ArgKind.SINGLE_ARG;
+ numRecords = 1;
+ }
+ } else {
+ // fixed number of args (presumably records)
+ argKind = ArgKind.MULTIPLE_ARGS;
+ numRecords = argEvals.length;
}
- boolean returnMissing = false, returnNull = false;
- for (int i = 0; i < argEvals.length; i++) {
- IPointable argPtr = argPointables[i];
- argEvals[i].evaluate(tuple, argPtr);
+ return validateRecords(tuple, result, argKind);
+ }
- byte[] data = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
- byte typeTag = data[offset];
+ private boolean validateRecords(IFrameTupleReference tuple, IPointable result, ArgKind argKind)
+ throws HyracksDataException {
+ boolean returnMissing = false, returnNull = false;
+ for (int i = 0; i < numRecords; i++) {
+ byte typeTag;
+ if (argKind == ArgKind.SINGLE_ARG_LIST) {
+ typeTag = listAccessor.getItemTypeAt(i).serialize();
+ } else if (argKind == ArgKind.SINGLE_ARG) {
+ // first arg has already been evaluated before
+ IPointable argPtr = argPointables[i];
+ argPtr.set(firstArg);
+ typeTag = argPtr.getByteArray()[argPtr.getStartOffset()];
+ } else {
+ IPointable argPtr = argPointables[i];
+ argEvals[i].evaluate(tuple, argPtr);
+ typeTag = argPtr.getByteArray()[argPtr.getStartOffset()];
+ }
if (typeTag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
returnMissing = true;
@@ -174,11 +242,11 @@
}
}
if (returnMissing) {
- writeTypeTag(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ writeTypeTag(ATypeTag.SERIALIZED_MISSING_TYPE_TAG, result);
return false;
}
if (returnNull) {
- writeTypeTag(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ writeTypeTag(ATypeTag.SERIALIZED_NULL_TYPE_TAG, result);
return false;
}
return true;
@@ -187,14 +255,34 @@
private void processArgs() throws HyracksDataException {
outRecordBuilder.init();
fieldMap.clear();
- for (int i = argEvals.length - 1; i >= 0; i--) {
+ if (argKind == ArgKind.SINGLE_ARG_LIST) {
+ processListRecords();
+ } else {
+ processArgsRecords();
+ }
+ outRecordBuilder.write(resultOutput, true);
+ }
+
+ private void processListRecords() throws HyracksDataException {
+ for (int i = numRecords - 1; i >= 0; i--) {
+ try {
+ itemRecordStorage.reset();
+ listAccessor.writeItem(i, itemRecordStorage.getDataOutput());
+ appendRecord(itemRecordStorage, itemRecordPointable, itemRecordCastRequired);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+
+ private void processArgsRecords() throws HyracksDataException {
+ for (int i = numRecords - 1; i >= 0; i--) {
try {
appendRecord(argPointables[i], argRecordPointables[i], castRequired.get(i));
} catch (IOException e) {
throw HyracksDataException.create(e);
}
}
- outRecordBuilder.write(resultOutput, true);
}
private void appendRecord(IPointable recordPtr, ARecordVisitablePointable argVisitablePointable,
@@ -239,12 +327,19 @@
return false;
}
- private void writeTypeTag(byte typeTag) throws HyracksDataException {
+ private void writeTypeTag(byte typeTag, IPointable result) throws HyracksDataException {
try {
resultOutput.writeByte(typeTag);
+ result.set(resultStorage);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
}
}
+
+ private enum ArgKind {
+ SINGLE_ARG_LIST,
+ SINGLE_ARG,
+ MULTIPLE_ARGS
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
index b51f66a..7c1a88d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordConcatStrictDescriptor.java
@@ -45,18 +45,20 @@
}
};
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private ARecordType[] argTypes;
+ private ARecordType listItemRecordType;
@Override
public void setImmutableStates(Object... states) {
- argTypes = (ARecordType[]) states;
+ argTypes = (ARecordType[]) states[0];
+ listItemRecordType = (ARecordType) states[1];
}
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
- return new RecordConcatEvalFactory(args, argTypes, true, sourceLoc);
+ return new RecordConcatEvalFactory(args, argTypes, listItemRecordType, true, sourceLoc);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentDateEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentDateEval.java
new file mode 100644
index 0000000..f9bb37c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentDateEval.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADate;
+import org.apache.asterix.om.base.AMutableDate;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+abstract class AbstractCurrentDateEval extends AbstractCurrentTemporalValueEval {
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADate> dateSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+ private final AMutableDate aDate = new AMutableDate(0);
+
+ AbstractCurrentDateEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier funcId) {
+ super(ctx, sourceLoc, funcId);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ int dateChronon = getCurrentDateChrononAdjusted();
+ aDate.setValue(dateChronon);
+ dateSerde.serialize(aDate, out);
+ result.set(resultStorage);
+ }
+
+ protected abstract int getCurrentDateChrononAdjusted() throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentDateTimeEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentDateTimeEval.java
new file mode 100644
index 0000000..3080e81
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentDateTimeEval.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+abstract class AbstractCurrentDateTimeEval extends AbstractCurrentTemporalValueEval {
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADateTime> datetimeSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+ private final AMutableDateTime aDateTime = new AMutableDateTime(0);
+
+ AbstractCurrentDateTimeEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier funcId) {
+ super(ctx, sourceLoc, funcId);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ long chronon = getCurrentDateTimeChrononAdjusted();
+ aDateTime.setValue(chronon);
+ datetimeSerde.serialize(aDateTime, out);
+ result.set(resultStorage);
+ }
+
+ protected abstract long getCurrentDateTimeChrononAdjusted() throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentTemporalValueEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentTemporalValueEval.java
new file mode 100644
index 0000000..1da7c91
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentTemporalValueEval.java
@@ -0,0 +1,95 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.time.DateTimeException;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.zone.ZoneRules;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.runtime.evaluators.functions.AbstractScalarEval;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+abstract class AbstractCurrentTemporalValueEval extends AbstractScalarEval {
+
+ protected final IEvaluatorContext ctx;
+ protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ protected final DataOutput out = resultStorage.getDataOutput();
+ protected final GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
+
+ private long jobStartTime = Long.MIN_VALUE;
+ private ZoneId jobStartTimeZoneId;
+ private ZoneRules jobStartTimeZoneRules;
+
+ AbstractCurrentTemporalValueEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier funcId) {
+ super(sourceLoc, funcId);
+ this.ctx = ctx;
+ }
+
+ protected final long getSystemCurrentTimeAsAdjustedChronon() throws HyracksDataException {
+ return getChrononAdjusted(System.currentTimeMillis());
+ }
+
+ protected final long getJobStartTimeAsAdjustedChronon() throws HyracksDataException {
+ return getChrononAdjusted(getJobStartTime());
+ }
+
+ private long getJobStartTime() throws HyracksDataException {
+ if (jobStartTime == Long.MIN_VALUE) {
+ IHyracksTaskContext taskCtx = ctx.getTaskContext();
+ if (taskCtx == null) {
+ throw new HyracksDataException(ErrorCode.ILLEGAL_STATE, srcLoc, "job-start-time");
+ }
+ jobStartTime = taskCtx.getJobletContext().getJobStartTime();
+ }
+ return jobStartTime;
+ }
+
+ private void ensureJobStartTimeZone() throws HyracksDataException {
+ if (jobStartTimeZoneId == null) {
+ IHyracksTaskContext taskCtx = ctx.getTaskContext();
+ if (taskCtx == null) {
+ throw new HyracksDataException(ErrorCode.ILLEGAL_STATE, srcLoc, "job-start-timezone");
+ }
+ try {
+ jobStartTimeZoneId = ZoneId.of(taskCtx.getJobletContext().getJobStartTimeZoneId());
+ jobStartTimeZoneRules = jobStartTimeZoneId.getRules();
+ } catch (DateTimeException e) {
+ throw new HyracksDataException(ErrorCode.ILLEGAL_STATE, e, srcLoc, "job-start-timezone");
+ }
+ }
+ }
+
+ private long getChrononAdjusted(long chronon) throws HyracksDataException {
+ ensureJobStartTimeZone();
+ ZoneOffset tzOffset = jobStartTimeZoneRules.getOffset(Instant.ofEpochMilli(chronon));
+ return cal.adjustChrononByTimezone(chronon, (int) -TimeUnit.SECONDS.toMillis(tzOffset.getTotalSeconds()));
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentTimeEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentTimeEval.java
new file mode 100644
index 0000000..367b999
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractCurrentTimeEval.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableTime;
+import org.apache.asterix.om.base.ATime;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+abstract class AbstractCurrentTimeEval extends AbstractCurrentTemporalValueEval {
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ATime> timeSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+ private final AMutableTime aTime = new AMutableTime(0);
+
+ public AbstractCurrentTimeEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier funcId) {
+ super(ctx, sourceLoc, funcId);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ int timeChronon = getCurrentTimeChrononAdjusted();
+ aTime.setValue(timeChronon);
+ timeSerde.serialize(aTime, out);
+ result.set(resultStorage);
+ }
+
+ protected abstract int getCurrentTimeChrononAdjusted() throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractDatetimeFromUnixTimeEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractDatetimeFromUnixTimeEval.java
new file mode 100644
index 0000000..f04c97b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractDatetimeFromUnixTimeEval.java
@@ -0,0 +1,146 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.zone.ZoneRules;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADateTime;
+import org.apache.asterix.om.base.AMutableDateTime;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.functions.AbstractScalarEval;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+abstract class AbstractDatetimeFromUnixTimeEval extends AbstractScalarEval {
+
+ private final IScalarEvaluator arg0;
+ private final IScalarEvaluator arg1;
+
+ private final IPointable argPtr0;
+ private final IPointable argPtr1;
+ private final UTF8StringPointable utf8Ptr;
+
+ protected final GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
+ protected final TimezoneHelper tzHelper;
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<ADateTime> datetimeSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+ private final AMutableDateTime aDatetime = new AMutableDateTime(0);
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+
+ AbstractDatetimeFromUnixTimeEval(IScalarEvaluator arg0, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ this(arg0, null, sourceLoc, fid);
+ }
+
+ AbstractDatetimeFromUnixTimeEval(IScalarEvaluator arg0, IScalarEvaluator arg1, SourceLocation sourceLoc,
+ FunctionIdentifier fid) {
+ super(sourceLoc, fid);
+ this.arg0 = arg0;
+ this.arg1 = arg1;
+ this.argPtr0 = new VoidPointable();
+ this.argPtr1 = arg1 != null ? new VoidPointable() : null;
+ this.utf8Ptr = arg1 != null ? new UTF8StringPointable() : null;
+ this.tzHelper = new TimezoneHelper(sourceLoc, fid);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ arg0.evaluate(tuple, argPtr0);
+ if (arg1 != null) {
+ arg1.evaluate(tuple, argPtr1);
+ }
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, argPtr0, argPtr1)) {
+ return;
+ }
+
+ long unixTime;
+ byte[] bytes = argPtr0.getByteArray();
+ int offset = argPtr0.getStartOffset();
+ ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
+ switch (argPtrTypeTag) {
+ case TINYINT:
+ unixTime = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
+ break;
+ case SMALLINT:
+ unixTime = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
+ break;
+ case INTEGER:
+ unixTime = AInt32SerializerDeserializer.getInt(bytes, offset + 1);
+ break;
+ case BIGINT:
+ unixTime = AInt64SerializerDeserializer.getLong(bytes, offset + 1);
+ break;
+ default:
+ throw new TypeMismatchException(srcLoc, funID, 0, bytes[offset], ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT16_TYPE_TAG, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ }
+
+ long chrononUTC = chrononFromUnixTime(unixTime);
+ long chrononAdjusted;
+
+ if (arg1 != null) {
+ byte[] bytes1 = argPtr1.getByteArray();
+ int offset1 = argPtr1.getStartOffset();
+ int len1 = argPtr1.getLength();
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(srcLoc, funID, 1, bytes1[offset1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
+ ZoneRules tzRules = tzHelper.parseTimeZone(utf8Ptr);
+ ZoneOffset tzOffset = tzRules.getOffset(Instant.ofEpochMilli(chrononUTC));
+ int tzOffsetMillis = (int) TimeUnit.SECONDS.toMillis(tzOffset.getTotalSeconds());
+ chrononAdjusted = cal.adjustChrononByTimezone(chrononUTC, -tzOffsetMillis);
+ } else {
+ chrononAdjusted = chrononUTC;
+ }
+
+ resultStorage.reset();
+ aDatetime.setValue(chrononAdjusted);
+ datetimeSerde.serialize(aDatetime, out);
+ result.set(resultStorage);
+ }
+
+ protected abstract long chrononFromUnixTime(long argValue);
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractUnixTimeFromDatetimeEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractUnixTimeFromDatetimeEval.java
new file mode 100644
index 0000000..2723a64
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractUnixTimeFromDatetimeEval.java
@@ -0,0 +1,142 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_DATETIME_TYPE_TAG;
+
+import java.io.DataOutput;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.zone.ZoneRules;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.functions.AbstractScalarEval;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+abstract class AbstractUnixTimeFromDatetimeEval extends AbstractScalarEval {
+
+ private final IScalarEvaluator arg0;
+ private final IScalarEvaluator arg1;
+
+ private final IPointable argPtr0;
+ private final IPointable argPtr1;
+ private final UTF8StringPointable utf8Ptr;
+
+ protected final GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
+ protected final TimezoneHelper tzHelper;
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt64> int64Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ private final AMutableInt64 aInt64 = new AMutableInt64(0);
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+
+ AbstractUnixTimeFromDatetimeEval(IScalarEvaluator arg0, SourceLocation sourceLoc,
+ FunctionIdentifier functionIdentifier) {
+ this(arg0, null, sourceLoc, functionIdentifier);
+ }
+
+ AbstractUnixTimeFromDatetimeEval(IScalarEvaluator arg0, IScalarEvaluator arg1, SourceLocation sourceLoc,
+ FunctionIdentifier fid) {
+ super(sourceLoc, fid);
+ this.arg0 = arg0;
+ this.arg1 = arg1;
+ this.argPtr0 = new VoidPointable();
+ this.argPtr1 = arg1 != null ? new VoidPointable() : null;
+ this.utf8Ptr = arg1 != null ? new UTF8StringPointable() : null;
+ this.tzHelper = new TimezoneHelper(sourceLoc, fid);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ arg0.evaluate(tuple, argPtr0);
+ if (arg1 != null) {
+ arg1.evaluate(tuple, argPtr1);
+ }
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, argPtr0, argPtr1)) {
+ return;
+ }
+
+ byte[] bytes0 = argPtr0.getByteArray();
+ int offset0 = argPtr0.getStartOffset();
+ if (bytes0[offset0] != SERIALIZED_DATETIME_TYPE_TAG) {
+ throw new TypeMismatchException(srcLoc, funID, 0, bytes0[offset0], ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
+ }
+
+ long chrononLocal = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
+ long chrononUTC;
+ if (arg1 != null) {
+ byte[] bytes1 = argPtr1.getByteArray();
+ int offset1 = argPtr1.getStartOffset();
+ int len1 = argPtr1.getLength();
+ if (bytes1[offset1] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(srcLoc, funID, 1, bytes1[offset1], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
+ ZoneRules tzRules = tzHelper.parseTimeZone(utf8Ptr);
+ LocalDateTime dt = toLocalDateTime(chrononLocal, cal);
+ ZoneOffset tzOffset = tzRules.getOffset(dt);
+ int tzOffsetMillis = (int) TimeUnit.SECONDS.toMillis(tzOffset.getTotalSeconds());
+ chrononUTC = cal.adjustChrononByTimezone(chrononLocal, tzOffsetMillis);
+ } else {
+ chrononUTC = chrononLocal;
+ }
+ long unixTime = chrononToUnixTime(chrononUTC);
+
+ resultStorage.reset();
+ aInt64.setValue(unixTime);
+ int64Serde.serialize(aInt64, out);
+ result.set(resultStorage);
+ }
+
+ private static LocalDateTime toLocalDateTime(long chronon, GregorianCalendarSystem cal) {
+ int year = cal.getYear(chronon);
+ int month = cal.getMonthOfYear(chronon, year);
+ int day = cal.getDayOfMonthYear(chronon, year, month);
+ int hour = cal.getHourOfDay(chronon);
+ int minute = cal.getMinOfHour(chronon);
+ int second = cal.getSecOfMin(chronon);
+ int milli = cal.getMillisOfSec(chronon);
+ int nano = (int) TimeUnit.MILLISECONDS.toNanos(milli);
+ return LocalDateTime.of(year, month, day, hour, minute, second, nano);
+ }
+
+ abstract long chrononToUnixTime(long chronon);
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
index 90c1eb0..da8d94f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustDateTimeForTimeZoneDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem.Fields;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -51,13 +50,7 @@
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.ADJUST_DATETIME_FOR_TIMEZONE;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AdjustDateTimeForTimeZoneDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = AdjustDateTimeForTimeZoneDescriptor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
@@ -124,8 +117,8 @@
chronon = calInstance.adjustChrononByTimezone(chronon, timezone);
StringBuilder sbder = new StringBuilder();
- calInstance.getExtendStringRepUntilField(chronon, timezone, sbder, Fields.YEAR,
- Fields.MILLISECOND, true);
+ calInstance.getExtendStringRepUntilField(chronon, sbder, Fields.YEAR, Fields.MILLISECOND);
+ GregorianCalendarSystem.getInstance().getTimezoneStringRep(timezone, sbder);
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
utf8Writer.writeUTF8(sbder, out);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
index 39235e6..44ef832 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AdjustTimeForTimeZoneDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem.Fields;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -50,13 +49,7 @@
public class AdjustTimeForTimeZoneDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.ADJUST_TIME_FOR_TIMEZONE;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new AdjustTimeForTimeZoneDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = AdjustTimeForTimeZoneDescriptor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
@@ -122,8 +115,8 @@
StringBuilder sbder = new StringBuilder();
- calInstance.getExtendStringRepUntilField(chronon, timezone, sbder, Fields.HOUR,
- Fields.MILLISECOND, true);
+ calInstance.getExtendStringRepUntilField(chronon, sbder, Fields.HOUR, Fields.MILLISECOND);
+ GregorianCalendarSystem.getInstance().getTimezoneStringRep(timezone, sbder);
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
writer.writeUTF8(sbder, out);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateDescriptor.java
index 8c5a303..3a1fe52 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -51,13 +50,7 @@
public class CalendarDurationFromDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.CALENDAR_DURATION_FROM_DATE;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CalendarDurationFromDateDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = CalendarDurationFromDateDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
index 67c0c05..bf450f0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CalendarDurationFromDateTimeDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -70,13 +69,7 @@
public class CalendarDurationFromDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public static final FunctionIdentifier FID = BuiltinFunctions.CALENDAR_DURATION_FROM_DATETIME;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CalendarDurationFromDateTimeDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = CalendarDurationFromDateTimeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java
index 00aa69c..76ca677 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateDescriptor.java
@@ -18,67 +18,28 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import java.io.DataOutput;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADate;
-import org.apache.asterix.om.base.AMutableDate;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public class CurrentDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public final static IFunctionDescriptorFactory FACTORY = CurrentDateDescriptor::new;
private static final long serialVersionUID = 1L;
private final static FunctionIdentifier FID = BuiltinFunctions.CURRENT_DATE;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CurrentDateDescriptor();
- }
- };
-
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
-
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
-
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADate> dateSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
- private AMutableDate aDate = new AMutableDate(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- int dateChronon = (int) (System.currentTimeMillis() / GregorianCalendarSystem.CHRONON_OF_DAY);
- aDate.setValue(dateChronon);
- dateSerde.serialize(aDate, out);
- result.set(resultStorage);
- }
- };
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) {
+ return new CurrentDateEval(ctx, sourceLoc, getIdentifier());
}
};
}
@@ -87,5 +48,4 @@
public FunctionIdentifier getIdentifier() {
return FID;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateEval.java
new file mode 100644
index 0000000..da0ef15
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateEval.java
@@ -0,0 +1,42 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+class CurrentDateEval extends AbstractCurrentDateEval {
+
+ private int dateChrononAdjusted = Integer.MIN_VALUE;
+
+ CurrentDateEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(ctx, sourceLoc, fid);
+ }
+
+ @Override
+ protected int getCurrentDateChrononAdjusted() throws HyracksDataException {
+ if (dateChrononAdjusted == Integer.MIN_VALUE) {
+ dateChrononAdjusted = cal.getChrononInDays(getJobStartTimeAsAdjustedChronon());
+ }
+ return dateChrononAdjusted;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateImmediateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateImmediateDescriptor.java
new file mode 100644
index 0000000..4fdba3c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateImmediateDescriptor.java
@@ -0,0 +1,52 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+
+public class CurrentDateImmediateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public final static IFunctionDescriptorFactory FACTORY = CurrentDateImmediateDescriptor::new;
+ private static final long serialVersionUID = 1L;
+ private final static FunctionIdentifier FID = BuiltinFunctions.CURRENT_DATE_IMMEDIATE;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) {
+ return new CurrentDateImmediateEval(ctx, sourceLoc, getIdentifier());
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateImmediateEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateImmediateEval.java
new file mode 100644
index 0000000..4b3a84e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateImmediateEval.java
@@ -0,0 +1,37 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+class CurrentDateImmediateEval extends AbstractCurrentDateEval {
+
+ CurrentDateImmediateEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(ctx, sourceLoc, fid);
+ }
+
+ @Override
+ protected int getCurrentDateChrononAdjusted() throws HyracksDataException {
+ return cal.getChrononInDays(getSystemCurrentTimeAsAdjustedChronon());
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java
index e0c22c3..b43227f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeDescriptor.java
@@ -18,77 +18,35 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import java.io.DataOutput;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public class CurrentDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public final static IFunctionDescriptorFactory FACTORY = CurrentDateTimeDescriptor::new;
private static final long serialVersionUID = 1L;
private final static FunctionIdentifier FID = BuiltinFunctions.CURRENT_DATETIME;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CurrentDateTimeDescriptor();
- }
- };
-
- private CurrentDateTimeDescriptor() {
- }
-
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
-
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADateTime> datetimeSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
- private AMutableDateTime aDateTime = new AMutableDateTime(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- aDateTime.setValue(System.currentTimeMillis());
- datetimeSerde.serialize(aDateTime, out);
- result.set(resultStorage);
- }
- };
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) {
+ return new CurrentDateTimeEval(ctx, CurrentDateTimeDescriptor.this.sourceLoc,
+ CurrentDateTimeDescriptor.this.getIdentifier());
}
};
}
- /* (non-Javadoc)
- * @see org.apache.asterix.om.functions.IFunctionDescriptor#getIdentifier()
- */
@Override
public FunctionIdentifier getIdentifier() {
return FID;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeEval.java
new file mode 100644
index 0000000..713f002
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeEval.java
@@ -0,0 +1,42 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+class CurrentDateTimeEval extends AbstractCurrentDateTimeEval {
+
+ private long chrononAdjusted = Long.MIN_VALUE;
+
+ CurrentDateTimeEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(ctx, sourceLoc, fid);
+ }
+
+ @Override
+ protected long getCurrentDateTimeChrononAdjusted() throws HyracksDataException {
+ if (chrononAdjusted == Long.MIN_VALUE) {
+ chrononAdjusted = getJobStartTimeAsAdjustedChronon();
+ }
+ return chrononAdjusted;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeImmediateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeImmediateDescriptor.java
new file mode 100644
index 0000000..0fcfd55
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeImmediateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+
+public class CurrentDateTimeImmediateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public final static IFunctionDescriptorFactory FACTORY = CurrentDateTimeImmediateDescriptor::new;
+ private static final long serialVersionUID = 1L;
+ private final static FunctionIdentifier FID = BuiltinFunctions.CURRENT_DATETIME_IMMEDIATE;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) {
+ return new CurrentDateTimeImmediateEval(ctx, CurrentDateTimeImmediateDescriptor.this.sourceLoc,
+ CurrentDateTimeImmediateDescriptor.this.getIdentifier());
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeImmediateEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeImmediateEval.java
new file mode 100644
index 0000000..56c70e1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentDateTimeImmediateEval.java
@@ -0,0 +1,36 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+class CurrentDateTimeImmediateEval extends AbstractCurrentDateTimeEval {
+ CurrentDateTimeImmediateEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(ctx, sourceLoc, fid);
+ }
+
+ @Override
+ protected long getCurrentDateTimeChrononAdjusted() throws HyracksDataException {
+ return getSystemCurrentTimeAsAdjustedChronon();
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java
index fe6348a..b47f915 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeDescriptor.java
@@ -18,79 +18,34 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import java.io.DataOutput;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AMutableTime;
-import org.apache.asterix.om.base.ATime;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public class CurrentTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- private static final long serialVersionUID = 1L;
+ public final static IFunctionDescriptorFactory FACTORY = CurrentTimeDescriptor::new;
public final static FunctionIdentifier FID = BuiltinFunctions.CURRENT_TIME;
+ private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new CurrentTimeDescriptor();
- }
- };
-
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@Override
- public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
-
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ATime> timeSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
- private AMutableTime aTime = new AMutableTime(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- int timeChronon = (int) (System.currentTimeMillis() % GregorianCalendarSystem.CHRONON_OF_DAY);
- aTime.setValue(timeChronon);
- timeSerde.serialize(aTime, out);
- result.set(resultStorage);
- }
- };
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) {
+ return new CurrentTimeEval(ctx, sourceLoc, getIdentifier());
}
};
}
- /* (non-Javadoc)
- * @see org.apache.asterix.om.functions.IFunctionDescriptor#getIdentifier()
- */
@Override
public FunctionIdentifier getIdentifier() {
return FID;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeEval.java
new file mode 100644
index 0000000..c73a2c0
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeEval.java
@@ -0,0 +1,42 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+class CurrentTimeEval extends AbstractCurrentTimeEval {
+
+ private int timeChrononAdjusted = Integer.MIN_VALUE;
+
+ CurrentTimeEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(ctx, sourceLoc, fid);
+ }
+
+ @Override
+ protected int getCurrentTimeChrononAdjusted() throws HyracksDataException {
+ if (timeChrononAdjusted == Integer.MIN_VALUE) {
+ timeChrononAdjusted = cal.getTimeChronon(getJobStartTimeAsAdjustedChronon());
+ }
+ return timeChrononAdjusted;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeImmediateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeImmediateDescriptor.java
new file mode 100644
index 0000000..b423f9f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeImmediateDescriptor.java
@@ -0,0 +1,52 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+
+public class CurrentTimeImmediateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public final static IFunctionDescriptorFactory FACTORY = CurrentTimeImmediateDescriptor::new;
+ public final static FunctionIdentifier FID = BuiltinFunctions.CURRENT_TIME_IMMEDIATE;
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) {
+ return new CurrentTimeImmediateEval(ctx, sourceLoc, getIdentifier());
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeImmediateEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeImmediateEval.java
new file mode 100644
index 0000000..8c5d3f2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/CurrentTimeImmediateEval.java
@@ -0,0 +1,37 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+class CurrentTimeImmediateEval extends AbstractCurrentTimeEval {
+
+ CurrentTimeImmediateEval(IEvaluatorContext ctx, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(ctx, sourceLoc, fid);
+ }
+
+ @Override
+ protected int getCurrentTimeChrononAdjusted() throws HyracksDataException {
+ return cal.getTimeChronon(getSystemCurrentTimeAsAdjustedChronon());
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
index d4ffb09..079d8a0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromDatetimeDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableDate;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -49,14 +48,7 @@
public class DateFromDatetimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DATE_FROM_DATETIME;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DateFromDatetimeDescriptor();
- }
-
- };
+ public final static IFunctionDescriptorFactory FACTORY = DateFromDatetimeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -78,6 +70,7 @@
private ISerializerDeserializer<ADate> dateSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
private AMutableDate aDate = new AMutableDate(0);
+ private final GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
@@ -96,10 +89,7 @@
ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- int dateChrononInDays = (int) (datetimeChronon / GregorianCalendarSystem.CHRONON_OF_DAY);
- if (dateChrononInDays < 0 && datetimeChronon % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
- dateChrononInDays -= 1;
- }
+ int dateChrononInDays = cal.getChrononInDays(datetimeChronon);
aDate.setValue(dateChrononInDays);
dateSerde.serialize(aDate, out);
result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java
index 4f05b1b..a4d62a0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DateFromUnixTimeInDaysDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.base.ADate;
import org.apache.asterix.om.base.AMutableDate;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -47,13 +46,7 @@
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DATE_FROM_UNIX_TIME_IN_DAYS;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DateFromUnixTimeInDaysDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = DateFromUnixTimeInDaysDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
index 4fdef14..c288e2c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromDateAndTimeDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,14 +49,7 @@
public class DatetimeFromDateAndTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DATETIME_FROM_DATE_TIME;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DatetimeFromDateAndTimeDescriptor();
- }
-
- };
+ public final static IFunctionDescriptorFactory FACTORY = DatetimeFromDateAndTimeDescriptor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
index 5da4475..79a6f5a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsDescriptor.java
@@ -18,52 +18,24 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import java.io.DataOutput;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
-public class DatetimeFromUnixTimeInMsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public final class DatetimeFromUnixTimeInMsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DATETIME_FROM_UNIX_TIME_IN_MS;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public final static IFunctionDescriptorFactory FACTORY = DatetimeFromUnixTimeInMsDescriptor::new;
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DatetimeFromUnixTimeInMsDescriptor();
- }
- };
-
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
@@ -72,64 +44,14 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable argPtr = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
-
- // possible output types
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADateTime> datetimeSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
-
- private AMutableDateTime aDatetime = new AMutableDateTime(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- eval.evaluate(tuple, argPtr);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, argPtr)) {
- return;
- }
-
- byte[] bytes = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
- ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
- switch (argPtrTypeTag) {
- case TINYINT:
- aDatetime.setValue(AInt8SerializerDeserializer.getByte(bytes, offset + 1));
- break;
- case SMALLINT:
- aDatetime.setValue(AInt16SerializerDeserializer.getShort(bytes, offset + 1));
- break;
- case INTEGER:
- aDatetime.setValue(AInt32SerializerDeserializer.getInt(bytes, offset + 1));
- break;
- case BIGINT:
- aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1));
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
- ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
- ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
- }
- datetimeSerde.serialize(aDatetime, out);
- result.set(resultStorage);
- }
- };
+ IScalarEvaluator argEval = args[0].createScalarEvaluator(ctx);
+ return new DatetimeFromUnixTimeInMsEval(argEval, sourceLoc, getIdentifier());
}
};
}
- /* (non-Javadoc)
- * @see org.apache.asterix.om.functions.IFunctionDescriptor#getIdentifier()
- */
@Override
public FunctionIdentifier getIdentifier() {
return FID;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsEval.java
new file mode 100644
index 0000000..a7d5338
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsEval.java
@@ -0,0 +1,41 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+final class DatetimeFromUnixTimeInMsEval extends AbstractDatetimeFromUnixTimeEval {
+
+ DatetimeFromUnixTimeInMsEval(IScalarEvaluator arg0, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(arg0, sourceLoc, fid);
+ }
+
+ DatetimeFromUnixTimeInMsEval(IScalarEvaluator arg0, IScalarEvaluator arg1, SourceLocation sourceLoc,
+ FunctionIdentifier fid) {
+ super(arg0, arg1, sourceLoc, fid);
+ }
+
+ @Override
+ protected long chrononFromUnixTime(long argValue) {
+ return argValue;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsWithTzDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsWithTzDescriptor.java
new file mode 100644
index 0000000..3eca492
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInMsWithTzDescriptor.java
@@ -0,0 +1,58 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public final class DatetimeFromUnixTimeInMsWithTzDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public final static FunctionIdentifier FID = BuiltinFunctions.DATETIME_FROM_UNIX_TIME_IN_MS_WITH_TZ;
+ public final static IFunctionDescriptorFactory FACTORY = DatetimeFromUnixTimeInMsWithTzDescriptor::new;
+ private final static long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ IScalarEvaluator arg0Eval = args[0].createScalarEvaluator(ctx);
+ IScalarEvaluator arg1Eval = args[1].createScalarEvaluator(ctx);
+ return new DatetimeFromUnixTimeInMsEval(arg0Eval, arg1Eval, sourceLoc, getIdentifier());
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
index e8190d5..4e625f1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsDescriptor.java
@@ -18,52 +18,24 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import java.io.DataOutput;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ADateTime;
-import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
-public class DatetimeFromUnixTimeInSecsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public final class DatetimeFromUnixTimeInSecsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DATETIME_FROM_UNIX_TIME_IN_SECS;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public final static IFunctionDescriptorFactory FACTORY = DatetimeFromUnixTimeInSecsDescriptor::new;
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DatetimeFromUnixTimeInSecsDescriptor();
- }
- };
-
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
@@ -71,65 +43,14 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable argPtr = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
-
- // possible output types
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<ADateTime> datetimeSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
-
- private AMutableDateTime aDatetime = new AMutableDateTime(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- eval.evaluate(tuple, argPtr);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, argPtr)) {
- return;
- }
-
- byte[] bytes = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
- ATypeTag argPtrTypeTag = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
-
- switch (argPtrTypeTag) {
- case TINYINT:
- aDatetime.setValue(AInt8SerializerDeserializer.getByte(bytes, offset + 1) * 1000l);
- break;
- case SMALLINT:
- aDatetime.setValue(AInt16SerializerDeserializer.getShort(bytes, offset + 1) * 1000l);
- break;
- case INTEGER:
- aDatetime.setValue(AInt32SerializerDeserializer.getInt(bytes, offset + 1) * 1000l);
- break;
- case BIGINT:
- aDatetime.setValue(AInt64SerializerDeserializer.getLong(bytes, offset + 1) * 1000l);
- break;
- default:
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
- ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
- ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG);
- }
- datetimeSerde.serialize(aDatetime, out);
- result.set(resultStorage);
- }
- };
+ IScalarEvaluator argEval = args[0].createScalarEvaluator(ctx);
+ return new DatetimeFromUnixTimeInSecsEval(argEval, sourceLoc, getIdentifier());
}
};
}
- /* (non-Javadoc)
- * @see org.apache.asterix.om.functions.IFunctionDescriptor#getIdentifier()
- */
@Override
public FunctionIdentifier getIdentifier() {
return FID;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsEval.java
new file mode 100644
index 0000000..7294ecb
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsEval.java
@@ -0,0 +1,41 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+final class DatetimeFromUnixTimeInSecsEval extends AbstractDatetimeFromUnixTimeEval {
+
+ DatetimeFromUnixTimeInSecsEval(IScalarEvaluator arg0, SourceLocation sourceLoc, FunctionIdentifier fid) {
+ super(arg0, sourceLoc, fid);
+ }
+
+ DatetimeFromUnixTimeInSecsEval(IScalarEvaluator arg0, IScalarEvaluator arg1, SourceLocation sourceLoc,
+ FunctionIdentifier fid) {
+ super(arg0, arg1, sourceLoc, fid);
+ }
+
+ @Override
+ protected long chrononFromUnixTime(long argValue) {
+ return argValue * 1000;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsWithTzDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsWithTzDescriptor.java
new file mode 100644
index 0000000..68bbcf6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DatetimeFromUnixTimeInSecsWithTzDescriptor.java
@@ -0,0 +1,57 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public final class DatetimeFromUnixTimeInSecsWithTzDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private final static long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = BuiltinFunctions.DATETIME_FROM_UNIX_TIME_IN_SECS_WITH_TZ;
+ public final static IFunctionDescriptorFactory FACTORY = DatetimeFromUnixTimeInSecsWithTzDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ IScalarEvaluator arg0Eval = args[0].createScalarEvaluator(ctx);
+ IScalarEvaluator arg1Eval = args[1].createScalarEvaluator(ctx);
+ return new DatetimeFromUnixTimeInSecsEval(arg0Eval, arg1Eval, sourceLoc, getIdentifier());
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeek2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeek2Descriptor.java
new file mode 100644
index 0000000..0e7feaf
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeek2Descriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public class DayOfWeek2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public final static FunctionIdentifier FID = BuiltinFunctions.DAY_OF_WEEK2;
+ public final static IFunctionDescriptorFactory FACTORY = DayOfWeek2Descriptor::new;
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ return new DayOfWeekEval(args[0].createScalarEvaluator(ctx), args[1].createScalarEvaluator(ctx), FID,
+ sourceLoc);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
index 17c8077..41442e4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekDescriptor.java
@@ -18,47 +18,22 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import java.io.DataOutput;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
public class DayOfWeekDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DAY_OF_WEEK;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DayOfWeekDescriptor();
- }
-
- };
+ public final static IFunctionDescriptorFactory FACTORY = DayOfWeekDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -67,56 +42,7 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable argPtr = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
-
- private GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
-
- // possible returning types
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AInt64> int64Serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
- private AMutableInt64 aInt64 = new AMutableInt64(0);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- eval.evaluate(tuple, argPtr);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, argPtr)) {
- return;
- }
-
- byte[] bytes = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
-
- long chronon;
- if (bytes[offset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- chronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- } else if (bytes[offset] == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
- chronon = ADateSerializerDeserializer.getChronon(bytes, offset + 1)
- * GregorianCalendarSystem.CHRONON_OF_DAY;
- } else {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
- ATypeTag.SERIALIZED_DATETIME_TYPE_TAG, ATypeTag.SERIALIZED_DATE_TYPE_TAG);
- }
-
- int weekday = cal.getDayOfWeek(chronon);
-
- // convert from 0-based to 1-based (so 7 = Sunday)
- if (weekday == 0) {
- weekday = GregorianCalendarSystem.DAYS_IN_A_WEEK;
- }
-
- aInt64.setValue(weekday);
- int64Serde.serialize(aInt64, out);
- result.set(resultStorage);
- }
- };
+ return new DayOfWeekEval(args[0].createScalarEvaluator(ctx), null, FID, sourceLoc);
}
};
}
@@ -125,5 +51,4 @@
public FunctionIdentifier getIdentifier() {
return FID;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekEval.java
new file mode 100644
index 0000000..d412181
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfWeekEval.java
@@ -0,0 +1,152 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.util.Objects;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class DayOfWeekEval implements IScalarEvaluator {
+
+ private final IScalarEvaluator eval0;
+ private final IScalarEvaluator eval1;
+ private final IPointable arg0Ptr = new VoidPointable();
+ private final IPointable arg1Ptr;
+ private final UTF8StringPointable str1Ptr;
+
+ private final GregorianCalendarSystem calSystem = GregorianCalendarSystem.getInstance();
+
+ // possible returning types
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt64> int64Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+
+ private final FunctionIdentifier fid;
+ private final SourceLocation sourceLoc;
+
+ public DayOfWeekEval(IScalarEvaluator eval0, IScalarEvaluator eval1, FunctionIdentifier fid,
+ SourceLocation sourceLoc) {
+ this.eval0 = Objects.requireNonNull(eval0);
+ this.eval1 = eval1;
+ arg1Ptr = eval1 != null ? new VoidPointable() : null;
+ str1Ptr = eval1 != null ? new UTF8StringPointable() : null;
+ this.fid = Objects.requireNonNull(fid);
+ this.sourceLoc = sourceLoc;
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval0.evaluate(tuple, arg0Ptr);
+ if (eval1 != null) {
+ eval1.evaluate(tuple, arg1Ptr);
+ }
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, arg0Ptr, arg1Ptr)) {
+ return;
+ }
+
+ byte[] bytes0 = arg0Ptr.getByteArray();
+ int offset0 = arg0Ptr.getStartOffset();
+
+ long chronon;
+ if (bytes0[offset0] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
+ } else if (bytes0[offset0] == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ chronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ } else {
+ throw new TypeMismatchException(sourceLoc, fid, 0, bytes0[offset0], ATypeTag.SERIALIZED_DATETIME_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG);
+ }
+
+ int weekStart = 0;
+ if (eval1 != null) {
+ byte[] bytes1 = arg1Ptr.getByteArray();
+ int offset1 = arg1Ptr.getStartOffset();
+ ATypeTag tt1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ switch (tt1) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ int v = ATypeHierarchy.getIntegerValue(fid.getName(), 1, bytes1, offset1);
+ weekStart = v - 1;
+ break;
+ case STRING:
+ int len1 = arg1Ptr.getLength();
+ str1Ptr.set(bytes1, offset1 + 1, len1 - 1);
+ int str1Len = str1Ptr.getStringLength();
+ weekStart = DateTimeFormatUtils.weekdayIDSearch(str1Ptr.getByteArray(),
+ str1Ptr.getCharStartOffset(), str1Len, str1Len == 3);
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, fid, 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+ ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ boolean weekStartValid = weekStart >= 0 && weekStart < GregorianCalendarSystem.DAYS_IN_A_WEEK;
+ if (!weekStartValid) {
+ throw new InvalidDataFormatException(sourceLoc, fid, "week_start_day");
+ }
+ }
+
+ int dowRaw = calSystem.getDayOfWeek(chronon);
+ int dowRawOffset = dowRaw - weekStart;
+ int dow0 = dowRawOffset < 0 ? dowRawOffset + GregorianCalendarSystem.DAYS_IN_A_WEEK : dowRawOffset;
+ int dow1 = dow0 + 1;
+
+ resultStorage.reset();
+ aInt64.setValue(dow1);
+ int64Serde.serialize(aInt64, out);
+ result.set(resultStorage);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfYearDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfYearDescriptor.java
new file mode 100644
index 0000000..cd82cf8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayOfYearDescriptor.java
@@ -0,0 +1,121 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+@MissingNullInOutFunction
+public class DayOfYearDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = BuiltinFunctions.DAY_OF_YEAR;
+
+ public final static IFunctionDescriptorFactory FACTORY = DayOfYearDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new IScalarEvaluator() {
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+ private final IPointable argPtr = new VoidPointable();
+ private final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
+
+ private final GregorianCalendarSystem calSystem = GregorianCalendarSystem.getInstance();
+
+ // for output: type integer
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt64> intSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, argPtr);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, argPtr)) {
+ return;
+ }
+
+ byte[] bytes = argPtr.getByteArray();
+ int startOffset = argPtr.getStartOffset();
+
+ resultStorage.reset();
+ try {
+ long chrononTimeInMs;
+ if (bytes[startOffset] == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ chrononTimeInMs = AInt32SerializerDeserializer.getInt(bytes, startOffset + 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
+ } else {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
+ }
+
+ int year = calSystem.getYear(chrononTimeInMs);
+ int day = calSystem.getDayOfYear(chrononTimeInMs, year);
+
+ aMutableInt64.setValue(day);
+ intSerde.serialize(aMutableInt64, out);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationGreaterThanComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationGreaterThanComparatorDescriptor.java
index 9157f80..5f2b7d2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationGreaterThanComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationGreaterThanComparatorDescriptor.java
@@ -20,20 +20,13 @@
package org.apache.asterix.runtime.evaluators.functions.temporal;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@MissingNullInOutFunction
public class DayTimeDurationGreaterThanComparatorDescriptor extends DayTimeDurationComparatorDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DayTimeDurationGreaterThanComparatorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = DayTimeDurationGreaterThanComparatorDescriptor::new;
protected DayTimeDurationGreaterThanComparatorDescriptor() {
super(true);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationLessThanComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationLessThanComparatorDescriptor.java
index cb662e3..ee368f3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationLessThanComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DayTimeDurationLessThanComparatorDescriptor.java
@@ -20,20 +20,13 @@
package org.apache.asterix.runtime.evaluators.functions.temporal;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@MissingNullInOutFunction
public class DayTimeDurationLessThanComparatorDescriptor extends DayTimeDurationComparatorDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DayTimeDurationLessThanComparatorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = DayTimeDurationLessThanComparatorDescriptor::new;
protected DayTimeDurationLessThanComparatorDescriptor() {
super(false);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
index fc6c39b..3044b12 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationEqualDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -47,13 +46,7 @@
public class DurationEqualDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DURATION_EQUAL;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DurationEqualDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = DurationEqualDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
index fc58cc2..baaf058 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromIntervalDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.AMutableDayTimeDuration;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -49,14 +48,7 @@
public class DurationFromIntervalDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DURATION_FROM_INTERVAL;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DurationFromIntervalDescriptor();
- }
-
- };
+ public final static IFunctionDescriptorFactory FACTORY = DurationFromIntervalDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
index d075562..4a35594 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMillisecondsDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.ADuration;
import org.apache.asterix.om.base.AMutableDuration;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -53,13 +52,7 @@
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DURATION_FROM_MILLISECONDS;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DurationFromMillisecondsDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = DurationFromMillisecondsDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java
index e71714b..5ae3123 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/DurationFromMonthsDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.base.ADuration;
import org.apache.asterix.om.base.AMutableDuration;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -47,13 +46,7 @@
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.DURATION_FROM_MONTHS;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new DurationFromMonthsDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = DurationFromMonthsDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
index 0af84df..358aecd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.ADayTimeDuration;
import org.apache.asterix.om.base.AMutableDayTimeDuration;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -48,13 +47,7 @@
public class GetDayTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.GET_DAY_TIME_DURATION;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GetDayTimeDurationDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = GetDayTimeDurationDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
index dc08502..0010b9e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AMutableInterval;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.asterix.om.types.ATypeTag;
@@ -49,12 +48,7 @@
@MissingNullInOutFunction
public class GetOverlappingIntervalDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GetOverlappingIntervalDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = GetOverlappingIntervalDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
index 0944e5d..43fa3fd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AMutableYearMonthDuration;
import org.apache.asterix.om.base.AYearMonthDuration;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -48,13 +47,7 @@
public class GetYearMonthDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.GET_YEAR_MONTH_DURATION;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new GetYearMonthDurationDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = GetYearMonthDurationDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalAfterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalAfterDescriptor.java
index bb95153..5f141d4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalAfterDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalAfterDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalAfterDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalAfterDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBeforeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBeforeDescriptor.java
index 1eee566..718ad59 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBeforeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBeforeDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private final static long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalBeforeDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = IntervalBeforeDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
index d3ecac7..2cd6300 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalBinDescriptor.java
@@ -32,7 +32,6 @@
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -59,13 +58,7 @@
public class IntervalBinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalBinDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalBinDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoveredByDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoveredByDescriptor.java
index 7d0c590..60d14ac 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoveredByDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoveredByDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalCoveredByDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalCoveredByDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoversDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoversDescriptor.java
index 4eff00a..504baac 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoversDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalCoversDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalCoversDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalCoversDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndedByDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndedByDescriptor.java
index f5b6589..b9e414d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndedByDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndedByDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalEndedByDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalEndedByDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndsDescriptor.java
index e5d1a27..7fc8ffb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalEndsDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -30,13 +29,7 @@
public class IntervalEndsDescriptor extends AbstractIntervalLogicFuncDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalEndsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalEndsDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMeetsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMeetsDescriptor.java
index 8fec043..ba3e1ed 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMeetsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMeetsDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalMeetsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalMeetsDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMetByDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMetByDescriptor.java
index b526c20..c92bd24 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMetByDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalMetByDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalMetByDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalMetByDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
index ec335e8..7cd9775 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalOverlappedByDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalOverlappedByDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
index 1968908..6884309 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalOverlapsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalOverlapsDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartedByDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartedByDescriptor.java
index a474f7e..33386b4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartedByDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartedByDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalStartedByDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalStartedByDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartsDescriptor.java
index 77dec6c..8021a16 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalStartsDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,13 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new IntervalStartsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = IntervalStartsDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
index 6a71c60..adb0a1a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -48,13 +47,7 @@
public class MillisecondsFromDayTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.MILLISECONDS_FROM_DAY_TIME_DURATION;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new MillisecondsFromDayTimeDurationDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = MillisecondsFromDayTimeDurationDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
index 96d7184..c428b4d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -48,13 +47,7 @@
public class MonthsFromYearMonthDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.MONTHS_FROM_YEAR_MONTH_DURATION;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new MonthsFromYearMonthDurationDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = MonthsFromYearMonthDurationDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
index 56f6633..a470464 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapBinsDescriptor.java
@@ -35,7 +35,6 @@
import org.apache.asterix.om.base.temporal.DurationArithmeticOperations;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
@@ -63,13 +62,7 @@
public class OverlapBinsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new OverlapBinsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = OverlapBinsDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
index 5c12752..f886697 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -31,12 +30,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new OverlapDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = OverlapDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
index f31cef7..58273ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateDescriptor.java
@@ -29,7 +29,6 @@
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -61,12 +60,7 @@
public class ParseDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ParseDateDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = ParseDateDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
index 49dec79..8596dab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseDateTimeDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,12 +51,7 @@
public class ParseDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ParseDateTimeDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = ParseDateTimeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
index 606e65e..15bd366 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/ParseTimeDescriptor.java
@@ -28,7 +28,6 @@
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -52,12 +51,7 @@
public class ParseTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ParseTimeDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = ParseTimeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
index a2311f4..ff787ee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -49,13 +48,7 @@
public class PrintDateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PrintDateDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = PrintDateDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -108,7 +101,7 @@
int formatLength = UTF8StringUtil.getUTFLength(bytes1, offset1 + 1);
int offset = UTF8StringUtil.getNumBytesToStoreLength(formatLength);
sbder.delete(0, sbder.length());
- util.printDateTime(chronon, 0, bytes1, offset1 + 1 + offset, formatLength, sbder,
+ util.printDateTime(chronon, bytes1, offset1 + 1 + offset, formatLength, sbder,
DateTimeParseMode.DATE_ONLY);
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
index 43b240d..f1d28df 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintDateTimeDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -48,13 +47,7 @@
public class PrintDateTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PrintDateTimeDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = PrintDateTimeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -107,7 +100,7 @@
utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
int formatLength = utf8Ptr.getUTF8Length();
sbder.delete(0, sbder.length());
- util.printDateTime(chronon, 0, utf8Ptr.getByteArray(), utf8Ptr.getCharStartOffset(),
+ util.printDateTime(chronon, utf8Ptr.getByteArray(), utf8Ptr.getCharStartOffset(),
formatLength, sbder, DateTimeParseMode.DATETIME);
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
index c74664e..568ffcb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/PrintTimeDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
import org.apache.asterix.om.base.temporal.DateTimeFormatUtils.DateTimeParseMode;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
@@ -48,13 +47,7 @@
public class PrintTimeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new PrintTimeDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = PrintTimeDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -107,7 +100,7 @@
utf8Ptr.set(bytes1, offset1 + 1, len1 - 1);
int formatLength = utf8Ptr.getUTF8Length();
sbder.delete(0, sbder.length());
- util.printDateTime(chronon, 0, utf8Ptr.getByteArray(), utf8Ptr.getCharStartOffset(),
+ util.printDateTime(chronon, utf8Ptr.getByteArray(), utf8Ptr.getCharStartOffset(),
formatLength, sbder, DateTimeParseMode.TIME_ONLY);
out.writeByte(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/QuarterOfYearDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/QuarterOfYearDescriptor.java
new file mode 100644
index 0000000..bd54db0
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/QuarterOfYearDescriptor.java
@@ -0,0 +1,143 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+@MissingNullInOutFunction
+public class QuarterOfYearDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ private static final FunctionIdentifier FID = BuiltinFunctions.QUARTER_OF_YEAR;
+ public static final IFunctionDescriptorFactory FACTORY = QuarterOfYearDescriptor::new;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new IScalarEvaluator() {
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+ private final IPointable argPtr = new VoidPointable();
+ private final IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
+
+ private final GregorianCalendarSystem calSystem = GregorianCalendarSystem.getInstance();
+
+ // for output: type integer
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt64> intSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ private final AMutableInt64 aMutableInt64 = new AMutableInt64(0);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, argPtr);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, argPtr)) {
+ return;
+ }
+
+ byte[] bytes = argPtr.getByteArray();
+ int startOffset = argPtr.getStartOffset();
+
+ resultStorage.reset();
+ try {
+ if (bytes[startOffset] == ATypeTag.SERIALIZED_DURATION_TYPE_TAG) {
+ int durationMonth = calSystem.getDurationMonth(
+ ADurationSerializerDeserializer.getYearMonth(bytes, startOffset + 1));
+ int durationQuarter = durationMonth / 3;
+ aMutableInt64.setValue(durationQuarter);
+ intSerde.serialize(aMutableInt64, out);
+ result.set(resultStorage);
+ return;
+ }
+ if (bytes[startOffset] == ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG) {
+ int durationMonth = calSystem.getDurationMonth(
+ AYearMonthDurationSerializerDeserializer.getYearMonth(bytes, startOffset + 1));
+ int durationQuarter = durationMonth / 3;
+ aMutableInt64.setValue(durationQuarter);
+ intSerde.serialize(aMutableInt64, out);
+ result.set(resultStorage);
+ return;
+ }
+
+ long chrononTimeInMs;
+ if (bytes[startOffset] == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ chrononTimeInMs = AInt32SerializerDeserializer.getInt(bytes, startOffset + 1)
+ * GregorianCalendarSystem.CHRONON_OF_DAY;
+ } else if (bytes[startOffset] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes, startOffset + 1);
+ } else {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[startOffset],
+ ATypeTag.SERIALIZED_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_YEAR_MONTH_DURATION_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATE_TYPE_TAG, ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
+ }
+
+ int year = calSystem.getYear(chrononTimeInMs);
+ int quarter = calSystem.getQuarterOfYear(chrononTimeInMs, year);
+
+ aMutableInt64.setValue(quarter);
+ intSerde.serialize(aMutableInt64, out);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
index b04944f..4771bc3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromDatetimeDescriptor.java
@@ -27,7 +27,6 @@
import org.apache.asterix.om.base.ATime;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -50,14 +49,7 @@
private static final long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.TIME_FROM_DATETIME;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TimeFromDatetimeDescriptor();
- }
-
- };
+ public final static IFunctionDescriptorFactory FACTORY = TimeFromDatetimeDescriptor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
@@ -82,6 +74,7 @@
private ISerializerDeserializer<ATime> timeSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
private AMutableTime aTime = new AMutableTime(0);
+ private final GregorianCalendarSystem cal = GregorianCalendarSystem.getInstance();
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
@@ -100,10 +93,7 @@
ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
}
long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- int timeChronon = (int) (datetimeChronon % GregorianCalendarSystem.CHRONON_OF_DAY);
- if (timeChronon < 0) {
- timeChronon += GregorianCalendarSystem.CHRONON_OF_DAY;
- }
+ int timeChronon = cal.getTimeChronon(datetimeChronon);
aTime.setValue(timeChronon);
timeSerde.serialize(aTime, out);
result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java
index f074d8c..b3cc2c0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimeFromUnixTimeInMsDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.om.base.AMutableTime;
import org.apache.asterix.om.base.ATime;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -46,13 +45,7 @@
public class TimeFromUnixTimeInMsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private final static long serialVersionUID = 1L;
public final static FunctionIdentifier FID = BuiltinFunctions.TIME_FROM_UNIX_TIME_IN_MS;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TimeFromUnixTimeInMsDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = TimeFromUnixTimeInMsDescriptor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimezoneHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimezoneHelper.java
new file mode 100644
index 0000000..7451570
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/TimezoneHelper.java
@@ -0,0 +1,64 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import java.time.zone.ZoneRules;
+import java.util.TimeZone;
+
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+
+final class TimezoneHelper {
+
+ private final SourceLocation srcLoc;
+
+ private final FunctionIdentifier funID;
+
+ private final ByteArrayAccessibleOutputStream lastTimezoneIdStorage = new ByteArrayAccessibleOutputStream();
+
+ private final UTF8StringPointable lastTimezoneIdPtr = new UTF8StringPointable();
+
+ private ZoneRules lastTimezoneRules;
+
+ TimezoneHelper(SourceLocation srcLoc, FunctionIdentifier funID) {
+ this.srcLoc = srcLoc;
+ this.funID = funID;
+ }
+
+ public ZoneRules parseTimeZone(UTF8StringPointable timezoneIdPtr) throws InvalidDataFormatException {
+ boolean newTimeZoneId = lastTimezoneRules == null || lastTimezoneIdPtr.compareTo(timezoneIdPtr) != 0;
+ if (newTimeZoneId) {
+ TimeZone tz = DateTimeFormatUtils.findTimeZone(timezoneIdPtr.getByteArray(),
+ timezoneIdPtr.getCharStartOffset(), timezoneIdPtr.getUTF8Length());
+ if (tz == null) {
+ throw new InvalidDataFormatException(srcLoc, funID, "timezone");
+ }
+ // ! object creation !
+ lastTimezoneRules = tz.toZoneId().getRules();
+ StringEvaluatorUtils.copyResetUTF8Pointable(timezoneIdPtr, lastTimezoneIdStorage, lastTimezoneIdPtr);
+ }
+ return lastTimezoneRules;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
index 2334438..930759a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDateInDaysDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -48,13 +47,7 @@
public class UnixTimeFromDateInDaysDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new UnixTimeFromDateInDaysDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = UnixTimeFromDateInDaysDescriptor::new;
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
index 5dd936b..5faaab6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsDescriptor.java
@@ -18,42 +18,21 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import java.io.DataOutput;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
-public class UnixTimeFromDatetimeInMsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public final class UnixTimeFromDatetimeInMsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public static final IFunctionDescriptorFactory FACTORY = UnixTimeFromDatetimeInMsDescriptor::new;
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new UnixTimeFromDatetimeInMsDescriptor();
- }
- };
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
@@ -62,41 +41,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable argPtr = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
-
- // possible returning types
- private AMutableInt64 aInt64 = new AMutableInt64(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AInt64> int64Serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- eval.evaluate(tuple, argPtr);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, argPtr)) {
- return;
- }
-
- byte[] bytes = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
-
- if (bytes[offset] != ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
- ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
- }
- long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- aInt64.setValue(datetimeChronon);
- int64Serde.serialize(aInt64, out);
- result.set(resultStorage);
- }
- };
+ IScalarEvaluator argEval = args[0].createScalarEvaluator(ctx);
+ return new UnixTimeFromDatetimeInMsEval(argEval, sourceLoc, getIdentifier());
}
};
}
@@ -105,5 +51,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.UNIX_TIME_FROM_DATETIME_IN_MS;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsEval.java
new file mode 100644
index 0000000..5bef2d9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsEval.java
@@ -0,0 +1,42 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+final class UnixTimeFromDatetimeInMsEval extends AbstractUnixTimeFromDatetimeEval {
+
+ UnixTimeFromDatetimeInMsEval(IScalarEvaluator arg0, SourceLocation sourceLoc,
+ FunctionIdentifier functionIdentifier) {
+ super(arg0, sourceLoc, functionIdentifier);
+ }
+
+ UnixTimeFromDatetimeInMsEval(IScalarEvaluator arg0, IScalarEvaluator arg1, SourceLocation sourceLoc,
+ FunctionIdentifier functionIdentifier) {
+ super(arg0, arg1, sourceLoc, functionIdentifier);
+ }
+
+ @Override
+ long chrononToUnixTime(long chronon) {
+ return chronon;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsWithTzDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsWithTzDescriptor.java
new file mode 100644
index 0000000..fd15179
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInMsWithTzDescriptor.java
@@ -0,0 +1,56 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public final class UnixTimeFromDatetimeInMsWithTzDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = UnixTimeFromDatetimeInMsWithTzDescriptor::new;
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ IScalarEvaluator arg0Eval = args[0].createScalarEvaluator(ctx);
+ IScalarEvaluator arg1Eval = args[1].createScalarEvaluator(ctx);
+ return new UnixTimeFromDatetimeInMsEval(arg0Eval, arg1Eval, sourceLoc, getIdentifier());
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.UNIX_TIME_FROM_DATETIME_IN_MS_WITH_TZ;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
index add409d..2b90b08 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsDescriptor.java
@@ -18,49 +18,22 @@
*/
package org.apache.asterix.runtime.evaluators.functions.temporal;
-import static org.apache.asterix.om.types.ATypeTag.SERIALIZED_DATETIME_TYPE_TAG;
-
-import java.io.DataOutput;
-
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@MissingNullInOutFunction
-public class UnixTimeFromDatetimeInSecsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public final class UnixTimeFromDatetimeInSecsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public static final IFunctionDescriptorFactory FACTORY = UnixTimeFromDatetimeInSecsDescriptor::new;
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new UnixTimeFromDatetimeInSecsDescriptor();
- }
- };
-
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
- */
@Override
public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
@@ -68,41 +41,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
- return new IScalarEvaluator() {
-
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable argPtr = new VoidPointable();
- private IScalarEvaluator eval = args[0].createScalarEvaluator(ctx);
-
- // possible returning types
- private AMutableInt64 aInt64 = new AMutableInt64(0);
- @SuppressWarnings("unchecked")
- private ISerializerDeserializer<AInt64> int64Serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- eval.evaluate(tuple, argPtr);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, argPtr)) {
- return;
- }
-
- byte[] bytes = argPtr.getByteArray();
- int offset = argPtr.getStartOffset();
-
- if (bytes[offset] != SERIALIZED_DATETIME_TYPE_TAG) {
- throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes[offset],
- ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
- }
- long datetimeChronon = ADateTimeSerializerDeserializer.getChronon(bytes, offset + 1);
- aInt64.setValue(datetimeChronon / 1000L);
- int64Serde.serialize(aInt64, out);
- result.set(resultStorage);
- }
- };
+ IScalarEvaluator argEval = args[0].createScalarEvaluator(ctx);
+ return new UnixTimeFromDatetimeInSecsEval(argEval, sourceLoc, getIdentifier());
}
};
}
@@ -111,5 +51,4 @@
public FunctionIdentifier getIdentifier() {
return BuiltinFunctions.UNIX_TIME_FROM_DATETIME_IN_SECS;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsEval.java
new file mode 100644
index 0000000..f5fa628
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsEval.java
@@ -0,0 +1,42 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+final class UnixTimeFromDatetimeInSecsEval extends AbstractUnixTimeFromDatetimeEval {
+
+ UnixTimeFromDatetimeInSecsEval(IScalarEvaluator arg0, SourceLocation sourceLoc,
+ FunctionIdentifier functionIdentifier) {
+ super(arg0, sourceLoc, functionIdentifier);
+ }
+
+ UnixTimeFromDatetimeInSecsEval(IScalarEvaluator arg0, IScalarEvaluator arg1, SourceLocation sourceLoc,
+ FunctionIdentifier functionIdentifier) {
+ super(arg0, arg1, sourceLoc, functionIdentifier);
+ }
+
+ @Override
+ long chrononToUnixTime(long chronon) {
+ return chronon / 1000L;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsWithTzDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsWithTzDescriptor.java
new file mode 100644
index 0000000..929e679
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromDatetimeInSecsWithTzDescriptor.java
@@ -0,0 +1,56 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public final class UnixTimeFromDatetimeInSecsWithTzDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = UnixTimeFromDatetimeInSecsWithTzDescriptor::new;
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+ IScalarEvaluator arg0Eval = args[0].createScalarEvaluator(ctx);
+ IScalarEvaluator arg1Eval = args[1].createScalarEvaluator(ctx);
+ return new UnixTimeFromDatetimeInSecsEval(arg0Eval, arg1Eval, sourceLoc, getIdentifier());
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.UNIX_TIME_FROM_DATETIME_IN_SECS_WITH_TZ;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
index 53071a1..184d0ad 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/UnixTimeFromTimeInMsDescriptor.java
@@ -26,7 +26,6 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
@@ -47,12 +46,7 @@
@MissingNullInOutFunction
public class UnixTimeFromTimeInMsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new UnixTimeFromTimeInMsDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = UnixTimeFromTimeInMsDescriptor::new;
/* (non-Javadoc)
* @see org.apache.asterix.runtime.base.IScalarFunctionDynamicDescriptor#createEvaluatorFactory(org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory[])
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYear2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYear2Descriptor.java
new file mode 100644
index 0000000..74c0172
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYear2Descriptor.java
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public class WeekOfYear2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public static final IFunctionDescriptorFactory FACTORY = WeekOfYear2Descriptor::new;
+ private static final long serialVersionUID = 1L;
+ private static final FunctionIdentifier FID = BuiltinFunctions.WEEK_OF_YEAR2;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new WeekOfYearEval(args[0].createScalarEvaluator(ctx), args[1].createScalarEvaluator(ctx), FID,
+ sourceLoc);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYearDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYearDescriptor.java
new file mode 100644
index 0000000..5b50320
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYearDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@MissingNullInOutFunction
+public class WeekOfYearDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public static final IFunctionDescriptorFactory FACTORY = WeekOfYearDescriptor::new;
+ private static final long serialVersionUID = 1L;
+ private static final FunctionIdentifier FID = BuiltinFunctions.WEEK_OF_YEAR;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new WeekOfYearEval(args[0].createScalarEvaluator(ctx), null, FID, sourceLoc);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYearEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYearEval.java
new file mode 100644
index 0000000..d1a89ed
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/WeekOfYearEval.java
@@ -0,0 +1,157 @@
+/*
+ * 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.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+import java.util.Calendar;
+import java.util.Objects;
+import java.util.TimeZone;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.temporal.DateTimeFormatUtils;
+import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+class WeekOfYearEval implements IScalarEvaluator {
+
+ private final IScalarEvaluator eval0;
+ private final IScalarEvaluator eval1;
+ private final IPointable arg0Ptr = new VoidPointable();
+ private final IPointable arg1Ptr;
+ private final UTF8StringPointable str1Ptr;
+
+ private final GregorianCalendarSystem calSystem = GregorianCalendarSystem.getInstance();
+ private final Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+
+ // possible returning types
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer<AInt64> int64Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ private final AMutableInt64 aInt64 = new AMutableInt64(0);
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+
+ private final FunctionIdentifier fid;
+ private final SourceLocation sourceLoc;
+
+ public WeekOfYearEval(IScalarEvaluator eval0, IScalarEvaluator eval1, FunctionIdentifier fid,
+ SourceLocation sourceLoc) {
+ this.eval0 = Objects.requireNonNull(eval0);
+ this.eval1 = eval1;
+ arg1Ptr = eval1 != null ? new VoidPointable() : null;
+ str1Ptr = eval1 != null ? new UTF8StringPointable() : null;
+ this.fid = Objects.requireNonNull(fid);
+ this.sourceLoc = sourceLoc;
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval0.evaluate(tuple, arg0Ptr);
+ if (eval1 != null) {
+ eval1.evaluate(tuple, arg1Ptr);
+ }
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, arg0Ptr, arg1Ptr)) {
+ return;
+ }
+
+ byte[] bytes0 = arg0Ptr.getByteArray();
+ int offset0 = arg0Ptr.getStartOffset();
+
+ long chrononTimeInMs;
+ if (bytes0[offset0] == ATypeTag.SERIALIZED_DATE_TYPE_TAG) {
+ chrononTimeInMs =
+ AInt32SerializerDeserializer.getInt(bytes0, offset0 + 1) * GregorianCalendarSystem.CHRONON_OF_DAY;
+ } else if (bytes0[offset0] == ATypeTag.SERIALIZED_DATETIME_TYPE_TAG) {
+ chrononTimeInMs = AInt64SerializerDeserializer.getLong(bytes0, offset0 + 1);
+ } else {
+ throw new TypeMismatchException(sourceLoc, fid, 0, bytes0[offset0], ATypeTag.SERIALIZED_DATE_TYPE_TAG,
+ ATypeTag.SERIALIZED_DATETIME_TYPE_TAG);
+ }
+
+ int weekStart = 0;
+
+ if (eval1 != null) {
+ byte[] bytes1 = arg1Ptr.getByteArray();
+ int offset1 = arg1Ptr.getStartOffset();
+ ATypeTag tt1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ switch (tt1) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ int v = ATypeHierarchy.getIntegerValue(fid.getName(), 1, bytes1, offset1);
+ weekStart = v - 1;
+ break;
+ case STRING:
+ int len1 = arg1Ptr.getLength();
+ str1Ptr.set(bytes1, offset1 + 1, len1 - 1);
+ int str1Len = str1Ptr.getStringLength();
+ weekStart = DateTimeFormatUtils.weekdayIDSearch(str1Ptr.getByteArray(),
+ str1Ptr.getCharStartOffset(), str1Len, str1Len == 3);
+ break;
+ default:
+ throw new TypeMismatchException(sourceLoc, fid, 1, bytes1[offset1],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ boolean weekStartValid = weekStart >= 0 && weekStart < GregorianCalendarSystem.DAYS_IN_A_WEEK;
+ if (!weekStartValid) {
+ throw new InvalidDataFormatException(sourceLoc, fid, "week_start_day");
+ }
+ }
+
+ int year = calSystem.getYear(chrononTimeInMs);
+ int month = calSystem.getMonthOfYear(chrononTimeInMs, year);
+ int day = calSystem.getDayOfMonthYear(chrononTimeInMs, year, month);
+
+ cal.setFirstDayOfWeek(weekStart + 1);
+ cal.setMinimalDaysInFirstWeek(1);
+ cal.set(year, month - 1, day);
+ int weekOfYear = cal.get(Calendar.WEEK_OF_YEAR);
+
+ resultStorage.reset();
+ aInt64.setValue(weekOfYear);
+ int64Serde.serialize(aInt64, out);
+ result.set(resultStorage);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationGreaterThanComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationGreaterThanComparatorDescriptor.java
index dbc3466..d865576 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationGreaterThanComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationGreaterThanComparatorDescriptor.java
@@ -20,20 +20,13 @@
package org.apache.asterix.runtime.evaluators.functions.temporal;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@MissingNullInOutFunction
public class YearMonthDurationGreaterThanComparatorDescriptor extends YearMonthDurationComparatorDescriptor {
private static final long serialVersionUID = 1L;
- public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new YearMonthDurationGreaterThanComparatorDescriptor();
- }
- };
+ public final static IFunctionDescriptorFactory FACTORY = YearMonthDurationGreaterThanComparatorDescriptor::new;
protected YearMonthDurationGreaterThanComparatorDescriptor() {
super(true);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationLessThanComparatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationLessThanComparatorDescriptor.java
index 2f32eed..6e03766 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationLessThanComparatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/YearMonthDurationLessThanComparatorDescriptor.java
@@ -20,20 +20,13 @@
package org.apache.asterix.runtime.evaluators.functions.temporal;
import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@MissingNullInOutFunction
public class YearMonthDurationLessThanComparatorDescriptor extends YearMonthDurationComparatorDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new YearMonthDurationLessThanComparatorDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = YearMonthDurationLessThanComparatorDescriptor::new;
protected YearMonthDurationLessThanComparatorDescriptor() {
super(false);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
index ba713e3..5b3f9a1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
@@ -30,8 +30,8 @@
public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, Throwable cause,
byte expectedTypeTag) {
- super(ErrorCode.INVALID_FORMAT, cause, sourceLoc, fid.getName(),
- EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(expectedTypeTag));
+ super(ErrorCode.INVALID_FORMAT, cause, sourceLoc,
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(expectedTypeTag), fid.getName());
}
public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, byte expectedTypeTag) {
@@ -39,6 +39,6 @@
}
public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, String expectedType) {
- super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(), expectedType);
+ super(ErrorCode.INVALID_FORMAT, sourceLoc, expectedType, fid.getName());
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index 31a93dc..8c8b7e6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -24,6 +24,7 @@
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
+import org.apache.asterix.dataflow.data.nontagged.NullWriterFactory;
import org.apache.asterix.formats.base.IDataFormat;
import org.apache.asterix.formats.nontagged.ADMPrinterFactoryProvider;
import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
@@ -33,6 +34,7 @@
import org.apache.asterix.formats.nontagged.BinaryIntegerInspector;
import org.apache.asterix.formats.nontagged.CSVPrinterFactoryProvider;
import org.apache.asterix.formats.nontagged.CleanJSONPrinterFactoryProvider;
+import org.apache.asterix.formats.nontagged.LosslessADMJSONPrinterFactoryProvider;
import org.apache.asterix.formats.nontagged.LosslessJSONPrinterFactoryProvider;
import org.apache.asterix.formats.nontagged.NormalizedKeyComputerFactoryProvider;
import org.apache.asterix.formats.nontagged.PredicateEvaluatorFactoryProvider;
@@ -41,6 +43,7 @@
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.IAObject;
@@ -299,6 +302,11 @@
}
@Override
+ public IPrinterFactoryProvider getLosslessADMJSONPrinterFactoryProvider() {
+ return LosslessADMJSONPrinterFactoryProvider.INSTANCE;
+ }
+
+ @Override
public IPrinterFactoryProvider getCleanJSONPrinterFactoryProvider() {
return CleanJSONPrinterFactoryProvider.INSTANCE;
}
@@ -311,9 +319,11 @@
@SuppressWarnings("unchecked")
@Override
public IScalarEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException {
- IAObject obj = null;
+ IAObject obj;
if (value.isMissing()) {
obj = AMissing.MISSING;
+ } else if (value.isNull()) {
+ obj = ANull.NULL;
} else if (value.isTrue()) {
obj = ABoolean.TRUE;
} else if (value.isFalse()) {
@@ -343,6 +353,11 @@
}
@Override
+ public IMissingWriterFactory getNullWriterFactory() {
+ return NullWriterFactory.INSTANCE;
+ }
+
+ @Override
public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory() {
return UnnestingPositionWriterFactory.INSTANCE;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
index 0179c34..c61cdd3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
@@ -22,6 +22,8 @@
import org.apache.asterix.common.metadata.DataverseName;
public abstract class AbstractFullTextFilterDescriptor implements IFullTextFilterDescriptor {
+ private static final long serialVersionUID = 6884215200256734408L;
+
protected final DataverseName dataverseName;
protected final String name;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 18d55aa..35993cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -37,6 +37,7 @@
import org.apache.asterix.runtime.aggregates.scalar.ScalarAvgDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarCountAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarCountDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarFirstElementAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarKurtosisAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarKurtosisDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarMaxAggregateDescriptor;
@@ -63,6 +64,7 @@
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlStddevPopDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlSumAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlSumDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlVarDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlVarPopAggregateDescriptor;
@@ -73,6 +75,7 @@
import org.apache.asterix.runtime.aggregates.scalar.ScalarStddevPopDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSumAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarSumDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarVarDistinctAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.scalar.ScalarVarPopAggregateDescriptor;
@@ -158,11 +161,13 @@
import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalSqlUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSqlVarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalStddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalStddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.GlobalVarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateAvgAggregateDescriptor;
@@ -178,11 +183,13 @@
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlStddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlStddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateSqlUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSqlVarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateStddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateStddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.IntermediateVarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.KurtosisAggregateDescriptor;
@@ -200,11 +207,13 @@
import org.apache.asterix.runtime.aggregates.std.LocalSqlStddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlStddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSqlUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSqlVarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalStddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalStddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.LocalVarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.MaxAggregateDescriptor;
@@ -220,11 +229,13 @@
import org.apache.asterix.runtime.aggregates.std.SqlStddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlStddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlUnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlVarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SqlVarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.StddevAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.StddevPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.SumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.UnionMbrAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.VarAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.std.VarPopAggregateDescriptor;
import org.apache.asterix.runtime.aggregates.stream.EmptyStreamAggregateDescriptor;
@@ -261,19 +272,35 @@
import org.apache.asterix.runtime.evaluators.comparisons.NullIfEqualsDescriptor;
import org.apache.asterix.runtime.evaluators.comparisons.PosInfIfEqualsDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ABinaryBase64StringConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ABinaryBase64StringDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ABinaryHexStringConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ABooleanDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ACircleConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateConstructorWithFormatDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateDefaultNullConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateDefaultNullConstructorWithFormatDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateTimeConstructorWithFormatDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateTimeDefaultNullConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADateTimeDefaultNullConstructorWithFormatDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ADayTimeDurationConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADayTimeDurationDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ADoubleConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADoubleDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ADurationConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ADurationDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AFloatConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AFloatDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AInt16ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt16DefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AInt32ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt32DefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AInt64ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt64DefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AInt8ConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AInt8DefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AIntervalConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromDateConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AIntervalStartFromDateTimeConstructorDescriptor;
@@ -284,9 +311,15 @@
import org.apache.asterix.runtime.evaluators.constructors.APolygonConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ARectangleConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AStringConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AStringDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ATimeConstructorWithFormatDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ATimeDefaultNullConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.ATimeDefaultNullConstructorWithFormatDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AUUIDFromStringConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AUUIDFromStringDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
+import org.apache.asterix.runtime.evaluators.constructors.AYearMonthDurationDefaultNullConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ClosedRecordConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.OpenRecordConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.OrderedListConstructorDescriptor;
@@ -335,6 +368,7 @@
import org.apache.asterix.runtime.evaluators.functions.DeepEqualityDescriptor;
import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
import org.apache.asterix.runtime.evaluators.functions.FullTextContainsWithoutOptionFunctionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.GetIntersectionDescriptor;
import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
import org.apache.asterix.runtime.evaluators.functions.GetJobParameterByNameDescriptor;
import org.apache.asterix.runtime.evaluators.functions.GetTypeDescriptor;
@@ -397,6 +431,7 @@
import org.apache.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfUp2Descriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericRoundWithRoundDigitDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericSignDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericSinDescriptor;
@@ -410,6 +445,7 @@
import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
import org.apache.asterix.runtime.evaluators.functions.RandomDescriptor;
import org.apache.asterix.runtime.evaluators.functions.RandomWithSeedDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ReferenceTileDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SleepDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
@@ -516,14 +552,21 @@
import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDurationFromDateDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.CalendarDurationFromDateTimeDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateImmediateDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentDateTimeImmediateDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.CurrentTimeImmediateDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DateFromDatetimeDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DateFromUnixTimeInDaysDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromDateAndTimeDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInMsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInMsWithTzDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInSecsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DatetimeFromUnixTimeInSecsWithTzDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DayOfWeek2Descriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DayOfWeekDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.DayOfYearDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DayTimeDurationGreaterThanComparatorDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DayTimeDurationLessThanComparatorDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.DurationEqualDescriptor;
@@ -556,12 +599,17 @@
import org.apache.asterix.runtime.evaluators.functions.temporal.PrintDateDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.PrintDateTimeDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.PrintTimeDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.QuarterOfYearDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromDatetimeDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.TimeFromUnixTimeInMsDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromDateInDaysDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromDatetimeInMsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromDatetimeInMsWithTzDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromDatetimeInSecsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromDatetimeInSecsWithTzDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.UnixTimeFromTimeInMsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.WeekOfYear2Descriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.WeekOfYearDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.YearMonthDurationGreaterThanComparatorDescriptor;
import org.apache.asterix.runtime.evaluators.functions.temporal.YearMonthDurationLessThanComparatorDescriptor;
import org.apache.asterix.runtime.runningaggregates.std.DenseRankRunningAggregateDescriptor;
@@ -571,9 +619,11 @@
import org.apache.asterix.runtime.runningaggregates.std.RowNumberRunningAggregateDescriptor;
import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
import org.apache.asterix.runtime.runningaggregates.std.WinMarkFirstMissingRunningAggregateDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.WinMarkFirstNullRunningAggregateDescriptor;
import org.apache.asterix.runtime.runningaggregates.std.WinPartitionLenRunningAggregateDescriptor;
import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
+import org.apache.asterix.runtime.unnestingfunctions.std.SpatialTileDescriptor;
import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
/**
@@ -625,6 +675,7 @@
fc.add(ScanCollectionDescriptor.FACTORY);
fc.add(RangeDescriptor.FACTORY);
fc.add(SubsetCollectionDescriptor.FACTORY);
+ fc.add(SpatialTileDescriptor.FACTORY);
// aggregate functions
fc.add(ListifyAggregateDescriptor.FACTORY);
@@ -677,6 +728,10 @@
fc.add(EmptyStreamAggregateDescriptor.FACTORY);
fc.add(NonEmptyStreamAggregateDescriptor.FACTORY);
fc.add(NullWriterAggregateDescriptor.FACTORY);
+ fc.add(UnionMbrAggregateDescriptor.FACTORY);
+ fc.add(LocalUnionMbrAggregateDescriptor.FACTORY);
+ fc.add(IntermediateUnionMbrAggregateDescriptor.FACTORY);
+ fc.add(GlobalUnionMbrAggregateDescriptor.FACTORY);
// serializable aggregates
fc.add(SerializableCountAggregateDescriptor.FACTORY);
@@ -738,6 +793,8 @@
fc.add(ScalarKurtosisDistinctAggregateDescriptor.FACTORY);
fc.add(ScalarSkewnessAggregateDescriptor.FACTORY);
fc.add(ScalarSkewnessDistinctAggregateDescriptor.FACTORY);
+ fc.add(ScalarUnionMbrAggregateDescriptor.FACTORY);
+ fc.add(ScalarFirstElementAggregateDescriptor.FACTORY);
// SQL aggregates
fc.add(SqlCountAggregateDescriptor.FACTORY);
@@ -781,6 +838,10 @@
fc.add(LocalSqlSkewnessAggregateDescriptor.FACTORY);
fc.add(IntermediateSqlSkewnessAggregateDescriptor.FACTORY);
fc.add(GlobalSqlSkewnessAggregateDescriptor.FACTORY);
+ fc.add(SqlUnionMbrAggregateDescriptor.FACTORY);
+ fc.add(LocalSqlUnionMbrAggregateDescriptor.FACTORY);
+ fc.add(IntermediateSqlUnionMbrAggregateDescriptor.FACTORY);
+ fc.add(GlobalSqlUnionMbrAggregateDescriptor.FACTORY);
// SQL serializable aggregates
fc.add(SerializableSqlCountAggregateDescriptor.FACTORY);
@@ -840,6 +901,7 @@
fc.add(ScalarSqlKurtosisDistinctAggregateDescriptor.FACTORY);
fc.add(ScalarSqlSkewnessAggregateDescriptor.FACTORY);
fc.add(ScalarSqlSkewnessDistinctAggregateDescriptor.FACTORY);
+ fc.add(ScalarSqlUnionMbrAggregateDescriptor.FACTORY);
// window functions
fc.add(DenseRankRunningAggregateDescriptor.FACTORY);
@@ -848,6 +910,7 @@
fc.add(RowNumberRunningAggregateDescriptor.FACTORY);
fc.add(PercentRankRunningAggregateDescriptor.FACTORY);
fc.add(WinMarkFirstMissingRunningAggregateDescriptor.FACTORY);
+ fc.add(WinMarkFirstNullRunningAggregateDescriptor.FACTORY);
fc.add(WinPartitionLenRunningAggregateDescriptor.FACTORY);
// boolean functions
@@ -893,8 +956,11 @@
fc.add(CreateQueryUIDDescriptor.FACTORY);
fc.add(RandomDescriptor.FACTORY);
fc.add(CurrentDateDescriptor.FACTORY);
+ fc.add(CurrentDateImmediateDescriptor.FACTORY);
fc.add(CurrentTimeDescriptor.FACTORY);
+ fc.add(CurrentTimeImmediateDescriptor.FACTORY);
fc.add(CurrentDateTimeDescriptor.FACTORY);
+ fc.add(CurrentDateTimeImmediateDescriptor.FACTORY);
fc.add(IsNumericAddCompatibleDescriptor.FACTORY);
@@ -929,6 +995,7 @@
fc.add(NumericRoundWithRoundDigitDescriptor.FACTORY);
fc.add(NumericRoundHalfToEvenDescriptor.FACTORY);
fc.add(NumericRoundHalfToEven2Descriptor.FACTORY);
+ fc.add(NumericRoundHalfUp2Descriptor.FACTORY);
fc.add(NumericACosDescriptor.FACTORY);
fc.add(NumericASinDescriptor.FACTORY);
fc.add(NumericATanDescriptor.FACTORY);
@@ -1036,15 +1103,24 @@
// Constructors
fc.add(ABooleanConstructorDescriptor.FACTORY);
+ fc.add(ABooleanDefaultNullConstructorDescriptor.FACTORY);
fc.add(ABinaryHexStringConstructorDescriptor.FACTORY);
fc.add(ABinaryBase64StringConstructorDescriptor.FACTORY);
+ fc.add(ABinaryBase64StringDefaultNullConstructorDescriptor.FACTORY);
fc.add(AStringConstructorDescriptor.FACTORY);
+ fc.add(AStringDefaultNullConstructorDescriptor.FACTORY);
fc.add(AInt8ConstructorDescriptor.FACTORY);
+ fc.add(AInt8DefaultNullConstructorDescriptor.FACTORY);
fc.add(AInt16ConstructorDescriptor.FACTORY);
+ fc.add(AInt16DefaultNullConstructorDescriptor.FACTORY);
fc.add(AInt32ConstructorDescriptor.FACTORY);
+ fc.add(AInt32DefaultNullConstructorDescriptor.FACTORY);
fc.add(AInt64ConstructorDescriptor.FACTORY);
+ fc.add(AInt64DefaultNullConstructorDescriptor.FACTORY);
fc.add(AFloatConstructorDescriptor.FACTORY);
+ fc.add(AFloatDefaultNullConstructorDescriptor.FACTORY);
fc.add(ADoubleConstructorDescriptor.FACTORY);
+ fc.add(ADoubleDefaultNullConstructorDescriptor.FACTORY);
fc.add(APointConstructorDescriptor.FACTORY);
fc.add(APoint3DConstructorDescriptor.FACTORY);
fc.add(ALineConstructorDescriptor.FACTORY);
@@ -1052,12 +1128,25 @@
fc.add(ACircleConstructorDescriptor.FACTORY);
fc.add(ARectangleConstructorDescriptor.FACTORY);
fc.add(ATimeConstructorDescriptor.FACTORY);
+ fc.add(ATimeDefaultNullConstructorDescriptor.FACTORY);
+ fc.add(ATimeConstructorWithFormatDescriptor.FACTORY);
+ fc.add(ATimeDefaultNullConstructorWithFormatDescriptor.FACTORY);
fc.add(ADateConstructorDescriptor.FACTORY);
+ fc.add(ADateDefaultNullConstructorDescriptor.FACTORY);
+ fc.add(ADateConstructorWithFormatDescriptor.FACTORY);
+ fc.add(ADateDefaultNullConstructorWithFormatDescriptor.FACTORY);
fc.add(ADateTimeConstructorDescriptor.FACTORY);
+ fc.add(ADateTimeDefaultNullConstructorDescriptor.FACTORY);
+ fc.add(ADateTimeConstructorWithFormatDescriptor.FACTORY);
+ fc.add(ADateTimeDefaultNullConstructorWithFormatDescriptor.FACTORY);
fc.add(ADurationConstructorDescriptor.FACTORY);
+ fc.add(ADurationDefaultNullConstructorDescriptor.FACTORY);
fc.add(AYearMonthDurationConstructorDescriptor.FACTORY);
+ fc.add(AYearMonthDurationDefaultNullConstructorDescriptor.FACTORY);
fc.add(ADayTimeDurationConstructorDescriptor.FACTORY);
+ fc.add(ADayTimeDurationDefaultNullConstructorDescriptor.FACTORY);
fc.add(AUUIDFromStringConstructorDescriptor.FACTORY);
+ fc.add(AUUIDFromStringDefaultNullConstructorDescriptor.FACTORY);
fc.add(AIntervalConstructorDescriptor.FACTORY);
fc.add(AIntervalStartFromDateConstructorDescriptor.FACTORY);
fc.add(AIntervalStartFromDateTimeConstructorDescriptor.FACTORY);
@@ -1078,6 +1167,8 @@
fc.add(CircleRadiusAccessor.FACTORY);
fc.add(CircleCenterAccessor.FACTORY);
fc.add(LineRectanglePolygonAccessor.FACTORY);
+ fc.add(ReferenceTileDescriptor.FACTORY);
+ fc.add(GetIntersectionDescriptor.FACTORY);
// full-text function
fc.add(FullTextContainsFunctionDescriptor.FACTORY);
@@ -1123,13 +1214,17 @@
fc.add(UnixTimeFromDateInDaysDescriptor.FACTORY);
fc.add(UnixTimeFromTimeInMsDescriptor.FACTORY);
fc.add(UnixTimeFromDatetimeInMsDescriptor.FACTORY);
+ fc.add(UnixTimeFromDatetimeInMsWithTzDescriptor.FACTORY);
fc.add(UnixTimeFromDatetimeInSecsDescriptor.FACTORY);
+ fc.add(UnixTimeFromDatetimeInSecsWithTzDescriptor.FACTORY);
fc.add(DateFromUnixTimeInDaysDescriptor.FACTORY);
fc.add(DateFromDatetimeDescriptor.FACTORY);
fc.add(TimeFromUnixTimeInMsDescriptor.FACTORY);
fc.add(TimeFromDatetimeDescriptor.FACTORY);
fc.add(DatetimeFromUnixTimeInMsDescriptor.FACTORY);
+ fc.add(DatetimeFromUnixTimeInMsWithTzDescriptor.FACTORY);
fc.add(DatetimeFromUnixTimeInSecsDescriptor.FACTORY);
+ fc.add(DatetimeFromUnixTimeInSecsWithTzDescriptor.FACTORY);
fc.add(DatetimeFromDateAndTimeDescriptor.FACTORY);
fc.add(CalendarDurationFromDateTimeDescriptor.FACTORY);
fc.add(CalendarDurationFromDateDescriptor.FACTORY);
@@ -1162,6 +1257,11 @@
fc.add(IntervalBinDescriptor.FACTORY);
fc.add(OverlapBinsDescriptor.FACTORY);
fc.add(DayOfWeekDescriptor.FACTORY);
+ fc.add(DayOfWeek2Descriptor.FACTORY);
+ fc.add(DayOfYearDescriptor.FACTORY);
+ fc.add(QuarterOfYearDescriptor.FACTORY);
+ fc.add(WeekOfYearDescriptor.FACTORY);
+ fc.add(WeekOfYear2Descriptor.FACTORY);
fc.add(ParseDateDescriptor.FACTORY);
fc.add(ParseTimeDescriptor.FACTORY);
fc.add(ParseDateTimeDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index 3882222..bc763bd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -23,6 +23,8 @@
import java.util.List;
import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -34,6 +36,7 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.ConstantExpressionUtil;
import org.apache.asterix.om.utils.RecordUtil;
@@ -118,8 +121,15 @@
public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
CompilerProperties compilerProps) throws AlgebricksException {
AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
- IAType rt = TypeCastUtils.getRequiredType(funcExpr);
- IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
+ IAType reqType = TypeCastUtils.getRequiredType(funcExpr);
+ IAType inputType = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
+ // If reqType or inputType is null it indicates there is a bug in the compiler.
+ if (reqType == null || inputType == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, expr.getSourceLocation(),
+ "Invalid types for casting, required type " + reqType + ", input type " + inputType);
+ }
+ IAType rt = TypeComputeUtils.getActualType(reqType);
+ IAType it = TypeComputeUtils.getActualType(inputType);
fd.setImmutableStates(rt, it);
}
}
@@ -306,14 +316,34 @@
List<Mutable<ILogicalExpression>> args = f.getArguments();
int n = args.size();
ARecordType[] argRecordTypes = new ARecordType[n];
- for (int i = 0; i < n; i++) {
- IAType argType = (IAType) context.getType(args.get(i).getValue());
- IAType t = TypeComputeUtils.getActualType(argType);
- if (t.getTypeTag() == ATypeTag.OBJECT) {
- argRecordTypes[i] = (ARecordType) t;
+ ARecordType listItemRecordType = null;
+ if (n == 1) {
+ // check and handle if it's the single argument list case
+ IAType t = getExprActualType(args.get(0).getValue(), context);
+ if (t.getTypeTag().isListType()) {
+ listItemRecordType = getListItemRecordType(t);
+ } else if (t.getTypeTag() == ATypeTag.OBJECT) {
+ argRecordTypes[0] = (ARecordType) t;
+ }
+ } else {
+ for (int i = 0; i < n; i++) {
+ IAType t = getExprActualType(args.get(i).getValue(), context);
+ if (t.getTypeTag() == ATypeTag.OBJECT) {
+ argRecordTypes[i] = (ARecordType) t;
+ }
}
}
- fd.setImmutableStates((Object[]) argRecordTypes);
+ fd.setImmutableStates(argRecordTypes, listItemRecordType);
+ }
+
+ private static IAType getExprActualType(ILogicalExpression expr, IVariableTypeEnvironment ctx)
+ throws AlgebricksException {
+ return TypeComputeUtils.getActualType((IAType) ctx.getType(expr));
+ }
+
+ private static ARecordType getListItemRecordType(IAType listType) {
+ IAType itemType = ((AbstractCollectionType) listType).getItemType();
+ return itemType.getTypeTag() == ATypeTag.OBJECT ? (ARecordType) itemType : null;
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
index 8e0de5f..1e7f4b7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
@@ -30,7 +30,7 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.ILogMarkerCallback;
import org.apache.asterix.common.transactions.PrimaryIndexLogMarkerCallback;
-import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AInt8;
import org.apache.asterix.om.pointables.nonvisitor.ARecordPointable;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -80,6 +80,9 @@
public class LSMPrimaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
+ public static final AInt8 UPSERT_NEW = new AInt8((byte) 0);
+ public static final AInt8 UPSERT_EXISTING = new AInt8((byte) 1);
+ public static final AInt8 DELETE_EXISTING = new AInt8((byte) 2);
private static final Logger LOGGER = LogManager.getLogger();
private static final ThreadLocal<DateFormat> DATE_FORMAT =
ThreadLocal.withInitial(() -> new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS"));
@@ -172,21 +175,22 @@
if (cursor.hasNext()) {
cursor.next();
prevTuple = cursor.getTuple();
- appendUpsertIndicator(!isDelete);
+ appendOperationIndicator(!isDelete, true);
appendFilterToPrevTuple();
appendPrevRecord();
appendPreviousMeta();
appendFilterToOutput();
} else {
- appendUpsertIndicator(!isDelete);
+ appendOperationIndicator(!isDelete, false);
appendPreviousTupleAsMissing();
}
} finally {
cursor.close(); // end the search
}
} else {
+ // simple upsert into a non-filtered dataset having no secondary indexes
searchCallback.before(key); // lock
- appendUpsertIndicator(!isDelete);
+ appendOperationIndicator(true, false);
appendPreviousTupleAsMissing();
}
beforeModification(tuple);
@@ -353,8 +357,17 @@
}
}
- protected void appendUpsertIndicator(boolean isUpsert) throws IOException {
- recordDesc.getFields()[0].serialize(isUpsert ? ABoolean.TRUE : ABoolean.FALSE, dos);
+ @SuppressWarnings("unchecked") // using serializer
+ protected void appendOperationIndicator(boolean isUpsert, boolean prevTupleExists) throws IOException {
+ if (isUpsert) {
+ if (prevTupleExists) {
+ recordDesc.getFields()[0].serialize(UPSERT_EXISTING, dos);
+ } else {
+ recordDesc.getFields()[0].serialize(UPSERT_NEW, dos);
+ }
+ } else {
+ recordDesc.getFields()[0].serialize(DELETE_EXISTING, dos);
+ }
tb.addFieldEndOffset();
}
@@ -446,7 +459,7 @@
if (tracer.isEnabled(traceCategory) && lastRecordInTimeStamp > 0 && indexHelper != null
&& indexHelper.getIndexInstance() != null) {
tracer.instant("UpsertClose", traceCategory, Scope.t,
- "{\"last-record-in\":\"" + DATE_FORMAT.get().format(new Date(lastRecordInTimeStamp))
+ () -> "{\"last-record-in\":\"" + DATE_FORMAT.get().format(new Date(lastRecordInTimeStamp))
+ "\", \"index\":" + indexHelper.getIndexInstance().toString() + "}");
}
} catch (Throwable traceFailure) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
index c5e8a53..f09df2e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
@@ -82,6 +82,8 @@
private DataOutput dos;
private final IMissingWriter missingWriter;
private DeletedTupleCounter deletedTupleCounter;
+ private final boolean excludeUnknownKeys;
+ private final boolean forAnyUnknownKey;
public static class DeletedTupleCounter extends AbstractStateObject {
private final Map<Integer, Integer> map = new HashMap<>();
@@ -102,12 +104,14 @@
public LSMSecondaryIndexCreationTupleProcessorNodePushable(IHyracksTaskContext ctx, int partition,
RecordDescriptor inputRecDesc, IMissingWriterFactory missingWriterFactory, int numTagFields,
- int numSecondaryKeys, int numPrimaryKeys, boolean hasBuddyBTree) throws HyracksDataException {
+ int numSecondaryKeys, int numPrimaryKeys, boolean hasBuddyBTree, boolean excludeUnknownKeys,
+ boolean forAnyUnknownKey) throws HyracksDataException {
super(ctx, partition, inputRecDesc, numTagFields, numSecondaryKeys, numPrimaryKeys, hasBuddyBTree);
this.prevMatterTupleBuilder = new ArrayTupleBuilder(inputRecDesc.getFieldCount());
-
+ this.excludeUnknownKeys = excludeUnknownKeys;
+ this.forAnyUnknownKey = forAnyUnknownKey;
if (this.hasBuddyBTree) {
missingWriter = missingWriterFactory.createMissingWriter();
} else {
@@ -192,16 +196,16 @@
private void processMatterTuple(ITupleReference tuple) throws HyracksDataException {
- boolean isNewValueMissing = isSecondaryKeyMissing(tuple);
- boolean isOldValueMissing = !hasPrevMatterTuple || !equalPrimaryKeys(tuple, prevMatterTuple)
- || isSecondaryKeyMissing(prevMatterTuple);
+ boolean skipNewValue = skipTuple(tuple);
+ boolean skipOldValue =
+ !hasPrevMatterTuple || !equalPrimaryKeys(tuple, prevMatterTuple) || skipTuple(prevMatterTuple);
- if (isNewValueMissing && isOldValueMissing) {
+ if (skipNewValue && skipOldValue) {
// if both values are missing, then do nothing
return;
}
// At least one is not null
- if (!isOldValueMissing && equalSecondaryKeys(prevMatterTuple, tuple)) {
+ if (!skipOldValue && equalSecondaryKeys(prevMatterTuple, tuple)) {
if (hasBuddyBTree) {
// if the index has buddy btree, then we have to delete the index entry
// from the older disk components
@@ -211,11 +215,11 @@
writeMatterTuple(tuple);
return;
}
- if (!isOldValueMissing) {
+ if (!skipOldValue) {
// we need to delete the previous entry
writeAntiMatterTuple(prevMatterTuple, getComponentPos(tuple));
}
- if (!isNewValueMissing) {
+ if (!skipNewValue) {
// we need to insert the new entry
writeMatterTuple(tuple);
}
@@ -223,10 +227,10 @@
}
private void processAntiMatterTuple(ITupleReference tuple) throws HyracksDataException {
- boolean isNewValueMissing = isSecondaryKeyMissing(tuple);
+ boolean skipNewValue = skipTuple(tuple);
// if the secondary value is missing (which means the secondary value of the previous matter tuple
// is also missing), we then simply ignore this tuple since there is nothing to delete
- if (!isNewValueMissing) {
+ if (!skipNewValue) {
writeAntiMatterTuple(tuple, getComponentPos(tuple));
}
}
@@ -264,7 +268,12 @@
FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
}
- private boolean isSecondaryKeyMissing(ITupleReference tuple) {
+ private boolean skipTuple(ITupleReference tuple) {
+ return excludeUnknownKeys
+ && (forAnyUnknownKey ? isAnySecondaryKeyMissing(tuple) : isAllSecondaryKeysMissing(tuple));
+ }
+
+ private boolean isAnySecondaryKeyMissing(ITupleReference tuple) {
for (int i = numTagFields; i < numTagFields + numSecondaryKeys; i++) {
if (TypeTagUtil.isType(tuple, i, ATypeTag.SERIALIZED_MISSING_TYPE_TAG)
|| TypeTagUtil.isType(tuple, i, ATypeTag.SERIALIZED_NULL_TYPE_TAG)) {
@@ -274,6 +283,16 @@
return false;
}
+ private boolean isAllSecondaryKeysMissing(ITupleReference tuple) {
+ for (int i = numTagFields; i < numTagFields + numSecondaryKeys; i++) {
+ if (!TypeTagUtil.isType(tuple, i, ATypeTag.SERIALIZED_MISSING_TYPE_TAG)
+ && !TypeTagUtil.isType(tuple, i, ATypeTag.SERIALIZED_NULL_TYPE_TAG)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
private boolean equalPrimaryKeys(ITupleReference tuple1, ITupleReference tuple2) {
for (int i = numTagFields + numSecondaryKeys; i < numTagFields + numPrimaryKeys + numSecondaryKeys; i++) {
if (!TupleUtils.equalFields(tuple1, tuple2, i)) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor.java
index 68e7603..f788d23 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor.java
@@ -31,7 +31,7 @@
public class LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor
extends AbstractSingleActivityOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private final IMissingWriterFactory missingWriterFactory;
@@ -40,10 +40,13 @@
private final int numPrimaryKeys;
private final boolean hasBuddyBTree;
+ private final boolean excludeUnknownKeys;
+ private final boolean forAnyUnknownKey;
public LSMSecondaryIndexCreationTupleProcessorOperatorDescriptor(IOperatorDescriptorRegistry spec,
RecordDescriptor outRecDesc, IMissingWriterFactory missingWriterFactory, int numTagFields,
- int numSecondaryKeys, int numPrimaryKeys, boolean hasBuddyBTree) {
+ int numSecondaryKeys, int numPrimaryKeys, boolean hasBuddyBTree, boolean excludeUnknownKeys,
+ boolean forAnyUnknownKey) {
super(spec, 1, 1);
this.outRecDescs[0] = outRecDesc;
this.missingWriterFactory = missingWriterFactory;
@@ -51,6 +54,8 @@
this.numSecondaryKeys = numSecondaryKeys;
this.numPrimaryKeys = numPrimaryKeys;
this.hasBuddyBTree = hasBuddyBTree;
+ this.excludeUnknownKeys = excludeUnknownKeys;
+ this.forAnyUnknownKey = forAnyUnknownKey;
}
@Override
@@ -58,6 +63,6 @@
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
return new LSMSecondaryIndexCreationTupleProcessorNodePushable(ctx, partition,
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), missingWriterFactory, numTagFields,
- numSecondaryKeys, numPrimaryKeys, hasBuddyBTree);
+ numSecondaryKeys, numPrimaryKeys, hasBuddyBTree, excludeUnknownKeys, forAnyUnknownKey);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
index 0b36774..5712991 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
@@ -31,13 +31,13 @@
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
@@ -122,10 +122,8 @@
private class IndexTupleInsertDelete implements IFrameWriter {
private final RecordDescriptor inputRecordDescriptor;
private FrameTupleAccessor endOfPipelineTupleAccessor;
-
- // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
- private ArrayTupleBuilder arrayTupleBuilder;
- private ArrayTupleReference arrayTupleReference;
+ private FrameTupleReference endOfPipelineTupleReference;
+ private ConcatenatingTupleReference endTupleReference;
private IndexTupleInsertDelete(RecordDescriptor recordDescriptor) {
this.inputRecordDescriptor = recordDescriptor;
@@ -133,11 +131,9 @@
@Override
public void open() throws HyracksDataException {
- int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
-
+ endTupleReference = new ConcatenatingTupleReference(2);
endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
- arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
- arrayTupleReference = new ArrayTupleReference();
+ endOfPipelineTupleReference = new FrameTupleReference();
}
@Override
@@ -147,25 +143,33 @@
endOfPipelineTupleAccessor.reset(buffer);
int nTuple = endOfPipelineTupleAccessor.getTupleCount();
for (int t = 0; t < nTuple; t++) {
+ endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+ endTupleReference.reset();
- // First, add the secondary keys.
- arrayTupleBuilder.reset();
- int nFields = endOfPipelineTupleAccessor.getFieldCount();
- for (int f = 0; f < nFields; f++) {
- arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
- }
+ // Add the secondary keys.
+ endTupleReference.addTuple(endOfPipelineTupleReference);
- // Next, add the primary keys and filter fields.
- for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
- arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
- }
+ // Add the primary keys and filter fields.
+ endTupleReference.addTuple(tuple);
- // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
- arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
+ // Pass the tuple to our accessor. There are only two operations: insert or delete.
if (op.equals(IndexOperation.INSERT)) {
- workingLSMAccessor.forceInsert(arrayTupleReference);
+ try {
+ workingLSMAccessor.forceInsert(endTupleReference);
+ } catch (HyracksDataException e) {
+ if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.DUPLICATE_KEY)) {
+ throw e;
+ }
+ }
+
} else {
- workingLSMAccessor.forceDelete(arrayTupleReference);
+ try {
+ workingLSMAccessor.forceDelete(endTupleReference);
+ } catch (HyracksDataException e) {
+ if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
+ throw e;
+ }
+ }
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
index a4b4012..9de0827 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
@@ -19,7 +19,7 @@
package org.apache.asterix.runtime.operators;
import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
-import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -33,21 +33,23 @@
public class LSMSecondaryUpsertOperatorDescriptor extends LSMTreeInsertDeleteOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 3L;
private final int[] prevValuePermutation;
- protected final int upsertIndicatorFieldIndex;
- protected final IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory;
+ protected final int operationFieldIndex;
+ protected final IBinaryIntegerInspectorFactory operationInspectorFactory;
+ private final ITupleFilterFactory prevTupleFilterFactory;
public LSMSecondaryUpsertOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
int[] fieldPermutation, IIndexDataflowHelperFactory indexHelperFactory,
- ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackFactory,
- int upsertIndicatorFieldIndex, IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
- int[] prevValuePermutation) {
+ ITupleFilterFactory tupleFilterFactory, ITupleFilterFactory prevTupleFilterFactory,
+ IModificationOperationCallbackFactory modificationOpCallbackFactory, int operationFieldIndex,
+ IBinaryIntegerInspectorFactory operationInspectorFactory, int[] prevValuePermutation) {
super(spec, outRecDesc, fieldPermutation, IndexOperation.UPSERT, indexHelperFactory, tupleFilterFactory, false,
modificationOpCallbackFactory);
this.prevValuePermutation = prevValuePermutation;
- this.upsertIndicatorFieldIndex = upsertIndicatorFieldIndex;
- this.upsertIndicatorInspectorFactory = upsertIndicatorInspectorFactory;
+ this.operationFieldIndex = operationFieldIndex;
+ this.operationInspectorFactory = operationInspectorFactory;
+ this.prevTupleFilterFactory = prevTupleFilterFactory;
}
@Override
@@ -55,7 +57,7 @@
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
RecordDescriptor intputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
return new LSMSecondaryUpsertOperatorNodePushable(ctx, partition, indexHelperFactory, modCallbackFactory,
- tupleFilterFactory, fieldPermutation, intputRecDesc, upsertIndicatorFieldIndex,
- upsertIndicatorInspectorFactory, prevValuePermutation);
+ tupleFilterFactory, prevTupleFilterFactory, fieldPermutation, intputRecDesc, operationFieldIndex,
+ operationInspectorFactory, prevValuePermutation);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
index b588323..955d5aa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
@@ -19,14 +19,13 @@
package org.apache.asterix.runtime.operators;
import java.nio.ByteBuffer;
-import java.util.Arrays;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.TypeTagUtil;
import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback;
import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
-import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
-import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspector;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -35,6 +34,7 @@
import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
import org.apache.hyracks.dataflow.common.utils.TupleUtils;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilter;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -48,38 +48,37 @@
* -If old secondary index tuple == new secondary index tuple
* --do nothing
* -else
- * --If any old field is null/missing?
- * ---do nothing
- * --else
- * ---delete old secondary index tuple
- * --If any new field is null/missing?
- * ---do nothing
- * --else
- * ---insert new secondary index tuple
+ * --perform the operation based on the operation kind
*/
public class LSMSecondaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
- private final PermutingFrameTupleReference prevValueTuple = new PermutingFrameTupleReference();
- private final int numberOfFields;
- private final boolean isPrimaryKeyIndex;
+ protected static final int UPSERT_NEW = LSMPrimaryUpsertOperatorNodePushable.UPSERT_NEW.getByteValue();
+ protected static final int UPSERT_EXISTING = LSMPrimaryUpsertOperatorNodePushable.UPSERT_EXISTING.getByteValue();
+ protected static final int DELETE_EXISTING = LSMPrimaryUpsertOperatorNodePushable.DELETE_EXISTING.getByteValue();
- protected final int upsertIndicatorFieldIndex;
- protected final IBinaryBooleanInspector upsertIndicatorInspector;
+ private final PermutingFrameTupleReference prevTuple = new PermutingFrameTupleReference();
+ private final int numberOfFields;
+ private final ITupleFilterFactory prevTupleFilterFactory;
+ private ITupleFilter prevTupleFilter;
+
+ protected final int operationFieldIndex;
+ protected final IBinaryIntegerInspector operationInspector;
protected AbstractIndexModificationOperationCallback abstractModCallback;
public LSMSecondaryUpsertOperatorNodePushable(IHyracksTaskContext ctx, int partition,
IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
- ITupleFilterFactory tupleFilterFactory, int[] fieldPermutation, RecordDescriptor inputRecDesc,
- int upsertIndicatorFieldIndex, IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
- int[] prevValuePermutation) throws HyracksDataException {
+ ITupleFilterFactory tupleFilterFactory, ITupleFilterFactory prevTupleFilterFactory, int[] fieldPermutation,
+ RecordDescriptor inputRecDesc, int operationFieldIndex,
+ IBinaryIntegerInspectorFactory operationInspectorFactory, int[] prevTuplePermutation)
+ throws HyracksDataException {
super(ctx, partition, indexHelperFactory, fieldPermutation, inputRecDesc, IndexOperation.UPSERT,
modCallbackFactory, tupleFilterFactory);
- this.prevValueTuple.setFieldPermutation(prevValuePermutation);
- this.upsertIndicatorFieldIndex = upsertIndicatorFieldIndex;
- this.upsertIndicatorInspector = upsertIndicatorInspectorFactory.createBinaryBooleanInspector(ctx);
+ this.prevTuple.setFieldPermutation(prevTuplePermutation);
+ this.operationFieldIndex = operationFieldIndex;
+ this.operationInspector = operationInspectorFactory.createBinaryIntegerInspector(ctx);
this.numberOfFields = fieldPermutation.length;
- // a primary key index only has primary keys, and thus these two permutations are the same
- this.isPrimaryKeyIndex = Arrays.equals(fieldPermutation, prevValuePermutation);
+ this.prevTupleFilterFactory = prevTupleFilterFactory;
+
}
@Override
@@ -87,6 +86,9 @@
super.open();
frameTuple = new FrameTupleReference();
abstractModCallback = (AbstractIndexModificationOperationCallback) modCallback;
+ if (prevTupleFilterFactory != null) {
+ prevTupleFilter = prevTupleFilterFactory.createTupleFilter(ctx);
+ }
}
@Override
@@ -94,39 +96,37 @@
accessor.reset(buffer);
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
int tupleCount = accessor.getTupleCount();
+ boolean tupleFilterIsNull = tupleFilter == null;
+ boolean prevTupleFilterIsNull = prevTupleFilter == null;
for (int i = 0; i < tupleCount; i++) {
try {
frameTuple.reset(accessor, i);
- boolean isUpsert =
- upsertIndicatorInspector.getBooleanValue(frameTuple.getFieldData(upsertIndicatorFieldIndex),
- frameTuple.getFieldStart(upsertIndicatorFieldIndex),
- frameTuple.getFieldLength(upsertIndicatorFieldIndex));
- // if both previous value and new value are null, then we skip
+ int operation = operationInspector.getIntegerValue(frameTuple.getFieldData(operationFieldIndex),
+ frameTuple.getFieldStart(operationFieldIndex), frameTuple.getFieldLength(operationFieldIndex));
tuple.reset(accessor, i);
- prevValueTuple.reset(accessor, i);
+ prevTuple.reset(accessor, i);
- boolean newTupleHasNullOrMissing = hasNullOrMissing(tuple);
- boolean oldTupleHasNullOrMissing = hasNullOrMissing(prevValueTuple);
- if (newTupleHasNullOrMissing && oldTupleHasNullOrMissing) {
- // No op
- continue;
- }
- // At least, one is not null
- if (!isPrimaryKeyIndex && TupleUtils.equalTuples(tuple, prevValueTuple, numberOfFields)) {
- // For a secondary index, if the secondary key values do not change, we can skip upserting it.
- // However, for a primary key index, we cannot do this because it only contains primary keys
- // which are always the same
- continue;
- }
- // if all old fields are known values, then delete. skip deleting if any is null or missing
- if (!oldTupleHasNullOrMissing) {
- abstractModCallback.setOp(Operation.DELETE);
- lsmAccessor.forceDelete(prevValueTuple);
- }
- // if all new fields are known values, then insert. skip inserting if any is null or missing
- if (isUpsert && !newTupleHasNullOrMissing) {
- abstractModCallback.setOp(Operation.INSERT);
- lsmAccessor.forceInsert(tuple);
+ if (operation == UPSERT_NEW) {
+ if (tupleFilterIsNull || tupleFilter.accept(frameTuple)) {
+ abstractModCallback.setOp(Operation.INSERT);
+ lsmAccessor.forceInsert(tuple);
+ }
+ } else if (operation == UPSERT_EXISTING) {
+ if (!TupleUtils.equalTuples(tuple, prevTuple, numberOfFields)) {
+ if (prevTupleFilterIsNull || prevTupleFilter.accept(frameTuple)) {
+ abstractModCallback.setOp(Operation.DELETE);
+ lsmAccessor.forceDelete(prevTuple);
+ }
+ if (tupleFilterIsNull || tupleFilter.accept(frameTuple)) {
+ abstractModCallback.setOp(Operation.INSERT);
+ lsmAccessor.forceInsert(tuple);
+ }
+ }
+ } else if (operation == DELETE_EXISTING) {
+ if (prevTupleFilterIsNull || prevTupleFilter.accept(frameTuple)) {
+ abstractModCallback.setOp(Operation.DELETE);
+ lsmAccessor.forceDelete(prevTuple);
+ }
}
} catch (Exception e) {
throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java
index d077987..41bd0fb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java
@@ -20,7 +20,7 @@
import java.util.List;
-import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -39,11 +39,11 @@
public LSMSecondaryUpsertWithNestedPlanOperatorDescriptor(JobSpecification spec, RecordDescriptor outRecDesc,
int[] fieldPermutation, IIndexDataflowHelperFactory indexHelperFactory,
- IModificationOperationCallbackFactory modCallbackFactory, int upsertIndicatorFieldIndex,
- IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
- List<AlgebricksPipeline> secondaryKeysPipeline, List<AlgebricksPipeline> prevSecondaryKeysPipeline) {
- super(spec, outRecDesc, fieldPermutation, indexHelperFactory, null, modCallbackFactory,
- upsertIndicatorFieldIndex, upsertIndicatorInspectorFactory, null);
+ IModificationOperationCallbackFactory modCallbackFactory, int operationFieldIndex,
+ IBinaryIntegerInspectorFactory operationInspectorFactory, List<AlgebricksPipeline> secondaryKeysPipeline,
+ List<AlgebricksPipeline> prevSecondaryKeysPipeline) {
+ super(spec, outRecDesc, fieldPermutation, indexHelperFactory, null, null, modCallbackFactory,
+ operationFieldIndex, operationInspectorFactory, null);
this.secondaryKeysPipeline = secondaryKeysPipeline;
this.prevSecondaryKeysPipeline = prevSecondaryKeysPipeline;
}
@@ -53,7 +53,7 @@
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
return new LSMSecondaryUpsertWithNestedPlanOperatorNodePushable(ctx, partition, indexHelperFactory,
- modCallbackFactory, fieldPermutation, inputRecDesc, upsertIndicatorFieldIndex,
- upsertIndicatorInspectorFactory, secondaryKeysPipeline, prevSecondaryKeysPipeline);
+ modCallbackFactory, fieldPermutation, inputRecDesc, operationFieldIndex, operationInspectorFactory,
+ secondaryKeysPipeline, prevSecondaryKeysPipeline);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
index f1af496..08fd566 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
@@ -24,7 +24,7 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback;
-import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.data.IBinaryIntegerInspectorFactory;
import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.operators.meta.PipelineAssembler;
@@ -33,13 +33,12 @@
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
public class LSMSecondaryUpsertWithNestedPlanOperatorNodePushable extends LSMSecondaryUpsertOperatorNodePushable {
@@ -49,12 +48,11 @@
public LSMSecondaryUpsertWithNestedPlanOperatorNodePushable(IHyracksTaskContext ctx, int partition,
IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
- int[] fieldPermutation, RecordDescriptor inputRecDesc, int upsertIndicatorFieldIndex,
- IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
- List<AlgebricksPipeline> secondaryKeysPipeline, List<AlgebricksPipeline> prevSecondaryKeysPipeline)
- throws HyracksDataException {
- super(ctx, partition, indexHelperFactory, modCallbackFactory, null, fieldPermutation, inputRecDesc,
- upsertIndicatorFieldIndex, upsertIndicatorInspectorFactory, null);
+ int[] fieldPermutation, RecordDescriptor inputRecDesc, int operationFieldIndex,
+ IBinaryIntegerInspectorFactory operationInspectorFactory, List<AlgebricksPipeline> secondaryKeysPipeline,
+ List<AlgebricksPipeline> prevSecondaryKeysPipeline) throws HyracksDataException {
+ super(ctx, partition, indexHelperFactory, modCallbackFactory, null, null, fieldPermutation, inputRecDesc,
+ operationFieldIndex, operationInspectorFactory, null);
this.numberOfPrimaryKeyAndFilterFields = fieldPermutation.length;
this.startOfNewKeyPipelines = buildStartOfPipelines(secondaryKeysPipeline, inputRecDesc, false);
this.startOfPrevKeyPipelines = buildStartOfPipelines(prevSecondaryKeysPipeline, inputRecDesc, true);
@@ -111,9 +109,9 @@
// Insert all of our new keys, if the PIDX operation was also an UPSERT (and not just a DELETE).
frameTuple.reset(accessor, i);
- if (upsertIndicatorInspector.getBooleanValue(frameTuple.getFieldData(upsertIndicatorFieldIndex),
- frameTuple.getFieldStart(upsertIndicatorFieldIndex),
- frameTuple.getFieldLength(upsertIndicatorFieldIndex))) {
+ int operation = operationInspector.getIntegerValue(frameTuple.getFieldData(operationFieldIndex),
+ frameTuple.getFieldStart(operationFieldIndex), frameTuple.getFieldLength(operationFieldIndex));
+ if (operation == UPSERT_NEW || operation == UPSERT_EXISTING) {
writeTupleToPipelineStarts(buffer, i, startOfNewKeyPipelines);
}
}
@@ -154,10 +152,7 @@
private FrameTupleAccessor endOfPipelineTupleAccessor;
private FrameTupleReference endOfPipelineTupleReference;
-
- // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
- private ArrayTupleBuilder arrayTupleBuilder;
- private ArrayTupleReference arrayTupleReference;
+ private ConcatenatingTupleReference endTupleReference;
private IndexTupleUnconditionalOperation(RecordDescriptor recordDescriptor, boolean isInsert) {
this.inputRecordDescriptor = recordDescriptor;
@@ -166,11 +161,9 @@
@Override
public void open() throws HyracksDataException {
- int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+ endTupleReference = new ConcatenatingTupleReference(2);
endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
endOfPipelineTupleReference = new FrameTupleReference();
- arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
- arrayTupleReference = new ArrayTupleReference();
}
@Override
@@ -180,33 +173,33 @@
endOfPipelineTupleAccessor.reset(buffer);
int nTuple = endOfPipelineTupleAccessor.getTupleCount();
for (int t = 0; t < nTuple; t++) {
-
endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+
+ // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
if (hasNullOrMissing(endOfPipelineTupleReference)) {
- // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
continue;
}
- // First, add the secondary keys.
- arrayTupleBuilder.reset();
- int nFields = endOfPipelineTupleAccessor.getFieldCount();
- for (int f = 0; f < nFields; f++) {
- arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
- }
+ // Add the secondary keys.
+ endTupleReference.reset();
+ endTupleReference.addTuple(endOfPipelineTupleReference);
- // Next, add the primary keys and filter fields.
- for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
- arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
- }
+ // Add the primary keys and filter fields.
+ endTupleReference.addTuple(tuple);
// Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
- arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
if (this.isInsert) {
abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.INSERT);
- workingLSMAccessor.forceInsert(arrayTupleReference);
+ try {
+ workingLSMAccessor.forceInsert(endTupleReference);
+ } catch (HyracksDataException e) {
+ if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.DUPLICATE_KEY)) {
+ throw e;
+ }
+ }
} else {
abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.DELETE);
- workingLSMAccessor.forceDelete(arrayTupleReference);
+ workingLSMAccessor.forceDelete(endTupleReference);
}
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoinOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoinOperatorDescriptor.java
index 0417ee0..fdcee13 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoinOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoinOperatorDescriptor.java
@@ -99,8 +99,8 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
- final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
+ final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
return new AbstractUnaryInputSinkOperatorNodePushable() {
private JoinCacheTaskState state;
@@ -112,7 +112,8 @@
IIntervalJoinUtil imjc = imjcf.createIntervalMergeJoinUtil(buildKey, probeKey, ctx, nPartitions);
- state.joiner = new IntervalMergeJoiner(ctx, memoryForJoin, imjc, buildKey, probeKey, rd0, rd1);
+ state.joiner =
+ new IntervalMergeJoiner(ctx, memoryForJoin, imjc, buildKey, probeKey, buildRd, probeRd);
}
@Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/PlaneSweepJoinOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/PlaneSweepJoinOperatorDescriptor.java
new file mode 100644
index 0000000..cf44aa9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/PlaneSweepJoinOperatorDescriptor.java
@@ -0,0 +1,179 @@
+/*
+ * 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.runtime.operators.joins.spatial;
+
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtil;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtilFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivity;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class PlaneSweepJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private static final int JOIN_BUILD_ACTIVITY_ID = 0;
+ private static final int JOIN_PROBE_ACTIVITY_ID = 1;
+ private final int[] buildKeys;
+ private final int[] probeKeys;
+ private final int memoryForJoin;
+ private final ISpatialJoinUtilFactory imjcf;
+
+ public PlaneSweepJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memoryForJoin, int[] buildKeys,
+ int[] probeKeys, RecordDescriptor recordDescriptor, ISpatialJoinUtilFactory imjcf) {
+ super(spec, 2, 1);
+ outRecDescs[0] = recordDescriptor;
+ this.buildKeys = buildKeys;
+ this.probeKeys = probeKeys;
+ this.memoryForJoin = memoryForJoin;
+ this.imjcf = imjcf;
+ }
+
+ @Override
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ ActivityId buildAid = new ActivityId(odId, JOIN_BUILD_ACTIVITY_ID);
+ ActivityId probeAid = new ActivityId(odId, JOIN_PROBE_ACTIVITY_ID);
+
+ IActivity probeAN = new JoinProbeActivityNode(probeAid);
+ IActivity buildAN = new JoinBuildActivityNode(buildAid, probeAid);
+
+ builder.addActivity(this, buildAN);
+ builder.addSourceEdge(0, buildAN, 0);
+
+ builder.addActivity(this, probeAN);
+ builder.addSourceEdge(1, probeAN, 0);
+ builder.addTargetEdge(0, probeAN, 0);
+ builder.addBlockingEdge(buildAN, probeAN);
+ }
+
+ public static class JoinCacheTaskState extends AbstractStateObject {
+ private SpatialJoiner joiner;
+
+ private JoinCacheTaskState(JobId jobId, TaskId taskId) {
+ super(jobId, taskId);
+ }
+ }
+
+ private class JoinBuildActivityNode extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ private final ActivityId nljAid;
+
+ public JoinBuildActivityNode(ActivityId id, ActivityId nljAid) {
+ super(id);
+ this.nljAid = nljAid;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
+ final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+
+ return new AbstractUnaryInputSinkOperatorNodePushable() {
+ private JoinCacheTaskState state;
+
+ @Override
+ public void open() throws HyracksDataException {
+ state = new JoinCacheTaskState(ctx.getJobletContext().getJobId(),
+ new TaskId(getActivityId(), partition));
+
+ ISpatialJoinUtil imjc = imjcf.createSpatialJoinUtil(buildKeys, probeKeys, ctx, nPartitions);
+
+ state.joiner = new SpatialJoiner(ctx, memoryForJoin, imjc, buildKeys, probeKeys, buildRd, probeRd);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ ByteBuffer copyBuffer = ctx.allocateFrame(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, copyBuffer);
+ state.joiner.processBuildFrame(copyBuffer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ state.joiner.processBuildClose();
+ ctx.setStateObject(state);
+ }
+
+ @Override
+ public void fail() {
+ // No variables to update.
+ }
+ };
+ }
+ }
+
+ private class JoinProbeActivityNode extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ public JoinProbeActivityNode(ActivityId id) {
+ super(id);
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+ private JoinCacheTaskState state;
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ state = (JoinCacheTaskState) ctx.getStateObject(
+ new TaskId(new ActivityId(getOperatorId(), JOIN_BUILD_ACTIVITY_ID), partition));
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ state.joiner.processProbeFrame(buffer, writer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ state.joiner.processProbeClose(writer);
+ } finally {
+ writer.close();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+ };
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/SpatialJoiner.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/SpatialJoiner.java
new file mode 100644
index 0000000..aaac216
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/SpatialJoiner.java
@@ -0,0 +1,228 @@
+/*
+ * 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.runtime.operators.joins.spatial;
+
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.FrameTupleCursor;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.RunFilePointer;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.RunFileStream;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.TuplePointerCursor;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtil;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.memory.SpatialSideTuple;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.buffermanager.DeallocatableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeallocatableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableDeletableTupleMemoryManager;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class SpatialJoiner {
+ private final IDeallocatableFramePool framePool;
+ private final IDeletableTupleBufferManager bufferManager;
+ private final TuplePointerCursor memoryCursor;
+ private final LinkedList<TuplePointer> memoryBuffer = new LinkedList<>();
+
+ private final RunFileStream runFileStream;
+ private final RunFilePointer runFilePointer;
+
+ private SpatialSideTuple memoryTuple;
+ private SpatialSideTuple[] inputTuple;
+ private TuplePointer tp;
+
+ private final ISpatialJoinUtil mjc;
+
+ protected static final int JOIN_PARTITIONS = 2;
+ protected static final int BUILD_PARTITION = 0;
+ protected static final int PROBE_PARTITION = 1;
+
+ protected final IFrame[] inputBuffer;
+ protected final FrameTupleAppender resultAppender;
+ protected final FrameTupleCursor[] inputCursor;
+
+ public SpatialJoiner(IHyracksTaskContext ctx, int memorySize, ISpatialJoinUtil mjc, int[] buildKeys,
+ int[] probeKeys, RecordDescriptor buildRd, RecordDescriptor probeRd) throws HyracksDataException {
+ this.mjc = mjc;
+
+ // Memory (probe buffer)
+ if (memorySize < 5) {
+ throw new RuntimeException(
+ "SpatialJoiner does not have enough memory (needs > 4, got " + memorySize + ").");
+ }
+
+ inputCursor = new FrameTupleCursor[JOIN_PARTITIONS];
+ inputCursor[BUILD_PARTITION] = new FrameTupleCursor(buildRd);
+ inputCursor[PROBE_PARTITION] = new FrameTupleCursor(probeRd);
+
+ inputBuffer = new IFrame[JOIN_PARTITIONS];
+ inputBuffer[BUILD_PARTITION] = new VSizeFrame(ctx);
+ inputBuffer[PROBE_PARTITION] = new VSizeFrame(ctx);
+
+ //Two frames are used for the runfile stream, and one frame for each input (2 outputs).
+ framePool = new DeallocatableFramePool(ctx, (memorySize - 4) * ctx.getInitialFrameSize());
+ bufferManager = new VariableDeletableTupleMemoryManager(framePool, probeRd);
+ memoryCursor = new TuplePointerCursor(bufferManager.createTuplePointerAccessor());
+
+ // Run File and frame cache (build buffer)
+ runFileStream = new RunFileStream(ctx, "sj-build");
+ runFilePointer = new RunFilePointer();
+ runFileStream.createRunFileWriting();
+ runFileStream.startRunFileWriting();
+
+ memoryTuple = new SpatialSideTuple(mjc, memoryCursor, probeKeys);
+
+ inputTuple = new SpatialSideTuple[JOIN_PARTITIONS];
+ inputTuple[PROBE_PARTITION] = new SpatialSideTuple(mjc, inputCursor[PROBE_PARTITION], probeKeys);
+ inputTuple[BUILD_PARTITION] = new SpatialSideTuple(mjc, inputCursor[BUILD_PARTITION], buildKeys);
+
+ // Result
+ this.resultAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+ }
+
+ public void processBuildFrame(ByteBuffer buffer) throws HyracksDataException {
+ inputCursor[BUILD_PARTITION].reset(buffer);
+ for (int x = 0; x < inputCursor[BUILD_PARTITION].getAccessor().getTupleCount(); x++) {
+ runFileStream.addToRunFile(inputCursor[BUILD_PARTITION].getAccessor(), x);
+ }
+ }
+
+ public void processBuildClose() throws HyracksDataException {
+ runFileStream.flushRunFile();
+ runFileStream.startReadingRunFile(inputCursor[BUILD_PARTITION]);
+ }
+
+ public void processProbeFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+ inputCursor[PROBE_PARTITION].reset(buffer);
+ while (buildHasNext() && inputCursor[PROBE_PARTITION].hasNext()) {
+ if (inputCursor[PROBE_PARTITION].hasNext() && mjc.checkToLoadNextProbeTuple(
+ inputCursor[BUILD_PARTITION].getAccessor(), inputCursor[BUILD_PARTITION].getTupleId() + 1,
+ inputCursor[PROBE_PARTITION].getAccessor(), inputCursor[PROBE_PARTITION].getTupleId() + 1)) {
+ // Process probe side from stream
+ inputCursor[PROBE_PARTITION].next();
+ processProbeTuple(writer);
+ } else {
+ // Process build side from runfile
+ inputCursor[BUILD_PARTITION].next();
+ processBuildTuple(writer);
+ }
+ }
+ }
+
+ public void processProbeClose(IFrameWriter writer) throws HyracksDataException {
+ while (buildHasNext() && memoryHasTuples()) {
+ // Process build side from runfile
+ inputCursor[BUILD_PARTITION].next();
+ processBuildTuple(writer);
+ }
+ resultAppender.write(writer, true);
+ runFileStream.close();
+ runFileStream.removeRunFile();
+ }
+
+ private boolean buildHasNext() throws HyracksDataException {
+ if (!inputCursor[BUILD_PARTITION].hasNext()) {
+ // Must keep condition in a separate `if` due to actions applied in loadNextBuffer.
+ return runFileStream.loadNextBuffer(inputCursor[BUILD_PARTITION]);
+ } else {
+ return true;
+ }
+ }
+
+ private void processBuildTuple(IFrameWriter writer) throws HyracksDataException {
+ // Check against memory
+ if (memoryHasTuples()) {
+ memoryCursor.reset(memoryBuffer.iterator());
+ while (memoryCursor.hasNext()) {
+ memoryCursor.next();
+ if (inputTuple[BUILD_PARTITION].removeFromMemory(memoryTuple)) {
+ // remove from memory
+ bufferManager.deleteTuple(memoryCursor.getTuplePointer());
+ memoryCursor.remove();
+ continue;
+ } else if (inputTuple[BUILD_PARTITION].checkForEarlyExit(memoryTuple)) {
+ // No more possible comparisons
+ break;
+ } else if (inputTuple[BUILD_PARTITION].compareJoin(memoryTuple)) {
+ // add to result
+ addToResult(inputCursor[BUILD_PARTITION].getAccessor(), inputCursor[BUILD_PARTITION].getTupleId(),
+ memoryCursor.getAccessor(), memoryCursor.getTupleId(), writer);
+ }
+ }
+ }
+ }
+
+ private void processProbeTuple(IFrameWriter writer) throws HyracksDataException {
+ // append to memory
+ // BUILD Cursor is guaranteed to have next
+ if (mjc.checkToSaveInMemory(inputCursor[BUILD_PARTITION].getAccessor(),
+ inputCursor[BUILD_PARTITION].getTupleId() + 1, inputCursor[PROBE_PARTITION].getAccessor(),
+ inputCursor[PROBE_PARTITION].getTupleId())) {
+ if (!addToMemory(inputCursor[PROBE_PARTITION].getAccessor(), inputCursor[PROBE_PARTITION].getTupleId())) {
+ unfreezeAndClearMemory(writer);
+ if (!addToMemory(inputCursor[PROBE_PARTITION].getAccessor(),
+ inputCursor[PROBE_PARTITION].getTupleId())) {
+ throw new RuntimeException("Should Never get called.");
+ }
+ }
+ }
+ }
+
+ private void unfreezeAndClearMemory(IFrameWriter writer) throws HyracksDataException {
+ runFilePointer.reset(runFileStream.getReadPointer(), inputCursor[BUILD_PARTITION].getTupleId());
+ while (buildHasNext() && memoryHasTuples()) {
+ // Process build side from runfile
+ inputCursor[BUILD_PARTITION].next();
+ processBuildTuple(writer);
+ }
+ // Clear memory
+ memoryBuffer.clear();
+ bufferManager.reset();
+ // Start reading
+ runFileStream.startReadingRunFile(inputCursor[BUILD_PARTITION], runFilePointer.getFileOffset());
+ inputCursor[BUILD_PARTITION].resetPosition(runFilePointer.getTupleIndex());
+ }
+
+ private boolean addToMemory(IFrameTupleAccessor accessor, int tupleId) throws HyracksDataException {
+ tp = new TuplePointer();
+ if (bufferManager.insertTuple(accessor, tupleId, tp)) {
+ memoryBuffer.add(tp);
+ return true;
+ }
+ return false;
+ }
+
+ private void addToResult(IFrameTupleAccessor buildAccessor, int buildTupleId, IFrameTupleAccessor probeAccessor,
+ int probeTupleId, IFrameWriter writer) throws HyracksDataException {
+ FrameUtils.appendConcatToWriter(writer, resultAppender, buildAccessor, buildTupleId, probeAccessor,
+ probeTupleId);
+ }
+
+ private boolean memoryHasTuples() {
+ return bufferManager.getNumTuples() > 0;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtil.java
new file mode 100644
index 0000000..7c18480
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtil.java
@@ -0,0 +1,100 @@
+/*
+ * 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.runtime.operators.joins.spatial.utils;
+
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISpatialJoinUtil {
+
+ /**
+ * Check to see if the right tuple should be added to memory during the merge join.
+ * The memory is used to check the right tuple with the remaining left tuples.
+ * The check is true if the next left tuple could still match with this right tuple.
+ *
+ * @param buildAccessor
+ * @param probeAccessor
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkToSaveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+
+ /**
+ * Check to see if the right tuple should be removed from memory during the merge join.
+ * The memory is used to check the right tuple with the remaining left tuples.
+ * The check is true if the next left tuple is NOT able match with this right tuple.
+ *
+ * @param buildAccessor
+ * @param probeAccessor
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkToRemoveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+
+ /**
+ * Check to see if tuples match join condition
+ * The check is true if they match.
+ *
+ * @param buildAccessor
+ * @param buildTupleIndex
+ * @param probeAccessor
+ * @param probeTupleIndex
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkToSaveInResult(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+
+ /**
+ * Check to see if the spatial object matches the join condition.
+ * The check is true if it matches.
+ *
+ * @param rectBuild
+ * @param rectProbe
+ * @return boolean
+ */
+ boolean compareRectangle(ARectangle rectBuild, ARectangle rectProbe);
+
+ /**
+ * Check to see if the left tuple should stop checking for matches.
+ * The check is true if there can be no more matches
+ *
+ * @param buildAccessor
+ * @param probeAccessor
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkForEarlyExit(IFrameTupleAccessor buildAccessor, int buildTupleIndex, IFrameTupleAccessor probeAccessor,
+ int probeTupleIndex) throws HyracksDataException;
+
+ /**
+ * Check if next tuple should be loaded into memory.
+ * The check is true if there are more tuples
+ *
+ * @param buildAccessor
+ * @param probeAccessor
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkToLoadNextProbeTuple(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtilFactory.java
new file mode 100644
index 0000000..0ad911a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtilFactory.java
@@ -0,0 +1,30 @@
+/*
+ * 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.runtime.operators.joins.spatial.utils;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISpatialJoinUtilFactory extends Serializable {
+
+ ISpatialJoinUtil createSpatialJoinUtil(int[] buildKey, int[] probeKey, IHyracksTaskContext ctx, int nPartitions)
+ throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtil.java
new file mode 100644
index 0000000..8ef8a85
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtil.java
@@ -0,0 +1,124 @@
+/*
+ * 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.runtime.operators.joins.spatial.utils;
+
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.memory.SpatialJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntersectSpatialJoinUtil implements ISpatialJoinUtil {
+
+ protected final int[] idBuild;
+ protected final int[] idProbe;
+
+ public IntersectSpatialJoinUtil(int[] idBuild, int[] idProbe) {
+ this.idBuild = idBuild;
+ this.idProbe = idProbe;
+ }
+
+ /**
+ * Right (second argument) interval starts before left (first argument) interval ends.
+ */
+ @Override
+ public boolean checkToSaveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+ int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+ int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+ double buildXmin = SpatialJoinUtil.getRectangleXmin(buildAccessor, buildTupleIndex, idBuild[1]);
+ double probeXmax = SpatialJoinUtil.getRectangleXmax(probeAccessor, probeTupleIndex, idProbe[1]);
+
+ if (buildTileId == probeTileId) {
+ return buildXmin < probeXmax;
+ } else {
+ return false;
+ }
+ }
+
+ /**
+ * Left (first argument) interval starts after the Right (second argument) interval ends.
+ */
+ @Override
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+ int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+ int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+ double buildXmin = SpatialJoinUtil.getRectangleXmin(buildAccessor, buildTupleIndex, idBuild[1]);
+ double probeXmax = SpatialJoinUtil.getRectangleXmax(probeAccessor, probeTupleIndex, idProbe[1]);
+
+ if (buildTileId != probeTileId) {
+ return true;
+ } else {
+ return buildXmin >= probeXmax;
+ }
+ }
+
+ @Override
+ public boolean checkToSaveInResult(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+ int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+ int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+ ARectangle rectBuild = SpatialJoinUtil.getRectangle(buildAccessor, buildTupleIndex, idBuild[1]);
+ ARectangle rectProbe = SpatialJoinUtil.getRectangle(probeAccessor, probeTupleIndex, idProbe[1]);
+
+ if (buildTileId == probeTileId) {
+ return compareRectangle(rectBuild, rectProbe);
+ } else {
+ return false;
+ }
+ }
+
+ /**
+ * Right (second argument) rectangle starts before left (first argument) rectangle ends.
+ */
+ @Override
+ public boolean checkForEarlyExit(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+ int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+ int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+ double probeXmin = SpatialJoinUtil.getRectangleXmin(probeAccessor, probeTupleIndex, idProbe[1]);
+ double buildXmax = SpatialJoinUtil.getRectangleXmax(buildAccessor, buildTupleIndex, idBuild[1]);
+
+ if (buildTileId != probeTileId) {
+ return true;
+ } else {
+ return buildXmax < probeXmin;
+ }
+ }
+
+ @Override
+ public boolean compareRectangle(ARectangle rectBuild, ARectangle rectProbe) {
+ return SpatialUtils.intersects(rectBuild, rectProbe);
+ }
+
+ @Override
+ public boolean checkToLoadNextProbeTuple(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+ IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+ int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+ int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+ double probeXmin = SpatialJoinUtil.getRectangleXmin(probeAccessor, probeTupleIndex, idProbe[1]);
+ double buildXmax = SpatialJoinUtil.getRectangleXmax(buildAccessor, buildTupleIndex, idBuild[1]);
+
+ if (buildTileId == probeTileId) {
+ return buildXmax > probeXmin;
+ }
+ return buildTileId >= probeTileId;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtilFactory.java
new file mode 100644
index 0000000..02a79d4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtilFactory.java
@@ -0,0 +1,33 @@
+/*
+ * 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.runtime.operators.joins.spatial.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntersectSpatialJoinUtilFactory implements ISpatialJoinUtilFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ISpatialJoinUtil createSpatialJoinUtil(int[] buildKeys, int[] probeKeys, IHyracksTaskContext ctx,
+ int nPartitions) throws HyracksDataException {
+ return new IntersectSpatialJoinUtil(buildKeys, probeKeys);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialJoinUtil.java
new file mode 100644
index 0000000..f8c2619
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialJoinUtil.java
@@ -0,0 +1,80 @@
+/*
+ * 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.runtime.operators.joins.spatial.utils.memory;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SpatialJoinUtil {
+
+ private SpatialJoinUtil() {
+ }
+
+ public static ARectangle getRectangle(IFrameTupleAccessor accessor, int tupleId, int fieldId)
+ throws HyracksDataException {
+ int start = getFieldOffset(accessor, tupleId, fieldId);
+ double xmin = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+ start + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double ymin = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+ start + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+ double xmax = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+ start + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double ymax = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+ start + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+ return new ARectangle(new APoint(xmin, ymin), new APoint(xmax, ymax));
+ }
+
+ public static int getTileId(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+ int start = getFieldOffset(accessor, tupleId, fieldId);
+ int tileId = AInt32SerializerDeserializer.getInt(accessor.getBuffer().array(), start);
+ return tileId;
+ }
+
+ public static double getRectangleXmin(IFrameTupleAccessor accessor, int tupleId, int fieldId)
+ throws HyracksDataException {
+ int start = getFieldOffset(accessor, tupleId, fieldId);
+ double xmin = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+ start + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ return xmin;
+ }
+
+ public static double getRectangleXmax(IFrameTupleAccessor accessor, int tupleId, int fieldId)
+ throws HyracksDataException {
+ int start = getFieldOffset(accessor, tupleId, fieldId);
+ double xmax = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+ start + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ return xmax;
+ }
+
+ public static int getFieldOffset(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+ return getFieldOffsetWithTag(accessor, tupleId, fieldId) + 1;
+ }
+
+ public static int getFieldOffsetWithTag(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+ int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+ + accessor.getFieldStartOffset(tupleId, fieldId);
+ return start;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialSideTuple.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialSideTuple.java
new file mode 100644
index 0000000..db84749
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialSideTuple.java
@@ -0,0 +1,61 @@
+/*
+ * 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.runtime.operators.joins.spatial.utils.memory;
+
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.ITupleCursor;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SpatialSideTuple {
+ // Tuple access
+ int[] fieldId;
+ ITupleCursor cursor;
+
+ // Join details
+ final ISpatialJoinUtil imjc;
+
+ public SpatialSideTuple(ISpatialJoinUtil imjc, ITupleCursor cursor, int[] fieldId) {
+ this.imjc = imjc;
+ this.cursor = cursor;
+ this.fieldId = fieldId;
+ }
+
+ public int getTupleIndex() {
+ return cursor.getTupleId();
+ }
+
+ public ITupleCursor getCursor() {
+ return cursor;
+ }
+
+ public boolean compareJoin(SpatialSideTuple ist) throws HyracksDataException {
+ return imjc.checkToSaveInResult(cursor.getAccessor(), cursor.getTupleId(), ist.cursor.getAccessor(),
+ ist.cursor.getTupleId());
+ }
+
+ public boolean removeFromMemory(SpatialSideTuple ist) throws HyracksDataException {
+ return imjc.checkToRemoveInMemory(cursor.getAccessor(), cursor.getTupleId(), ist.cursor.getAccessor(),
+ ist.cursor.getTupleId());
+ }
+
+ public boolean checkForEarlyExit(SpatialSideTuple ist) throws HyracksDataException {
+ return imjc.checkForEarlyExit(cursor.getAccessor(), cursor.getTupleId(), ist.cursor.getAccessor(),
+ ist.cursor.getTupleId());
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionInfo.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionInfo.java
new file mode 100644
index 0000000..de402ec
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/DataProjectionInfo.java
@@ -0,0 +1,159 @@
+/*
+ * 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.runtime.projection;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.visitor.SimpleStringBuilderForIATypeVisitor;
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+
+public class DataProjectionInfo implements IProjectionInfo<ARecordType> {
+ //Default open record type when requesting the entire fields
+ public static final ARecordType ALL_FIELDS_TYPE = createType("");
+ //Default open record type when requesting none of the fields
+ public static final ARecordType EMPTY_TYPE = createType("{}");
+
+ private final ARecordType root;
+ private final Map<String, FunctionCallInformation> functionCallInfoMap;
+
+ public DataProjectionInfo(ARecordType root, Map<String, FunctionCallInformation> sourceInformationMap) {
+ this.root = root;
+ this.functionCallInfoMap = sourceInformationMap;
+ }
+
+ private DataProjectionInfo(DataProjectionInfo other) {
+ if (other.root == ALL_FIELDS_TYPE) {
+ root = ALL_FIELDS_TYPE;
+ } else if (other.root == EMPTY_TYPE) {
+ root = EMPTY_TYPE;
+ } else {
+ root = other.root.deepCopy(other.root);
+ }
+ functionCallInfoMap = new HashMap<>(other.functionCallInfoMap);
+ }
+
+ @Override
+ public ARecordType getProjectionInfo() {
+ return root;
+ }
+
+ @Override
+ public DataProjectionInfo createCopy() {
+ return new DataProjectionInfo(this);
+ }
+
+ public Map<String, FunctionCallInformation> getFunctionCallInfoMap() {
+ return functionCallInfoMap;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ DataProjectionInfo otherInfo = (DataProjectionInfo) o;
+ return root.deepEqual(otherInfo.root) && Objects.equals(functionCallInfoMap, otherInfo.functionCallInfoMap);
+ }
+
+ @Override
+ public String toString() {
+ if (root == ALL_FIELDS_TYPE || root == EMPTY_TYPE) {
+ //Return the type name if all fields or empty types
+ return root.getTypeName();
+ }
+ //Return a oneliner JSON like representation for the requested fields
+ StringBuilder builder = new StringBuilder();
+ SimpleStringBuilderForIATypeVisitor visitor = new SimpleStringBuilderForIATypeVisitor();
+ root.accept(visitor, builder);
+ return builder.toString();
+ }
+
+ /**
+ * Serialize expected record type
+ *
+ * @param expectedRecordType expected record type
+ * @param output data output
+ */
+ public static void writeTypeField(ARecordType expectedRecordType, DataOutput output) throws IOException {
+ byte[] recordTypeBytes = SerializationUtils.serialize(expectedRecordType);
+ output.writeInt(recordTypeBytes.length);
+ output.write(recordTypeBytes);
+ }
+
+ /**
+ * Deserialize expected record type
+ *
+ * @param input data input
+ * @return deserialized expected record type
+ */
+ public static ARecordType createTypeField(DataInput input) throws IOException {
+ int length = input.readInt();
+ byte[] recordTypeBytes = new byte[length];
+ input.readFully(recordTypeBytes, 0, length);
+ return SerializationUtils.deserialize(recordTypeBytes);
+ }
+
+ /**
+ * Serialize function call information map
+ *
+ * @param functionCallInfoMap function information map
+ * @param output data output
+ */
+ public static void writeFunctionCallInformationMapField(Map<String, FunctionCallInformation> functionCallInfoMap,
+ DataOutput output) throws IOException {
+ output.writeInt(functionCallInfoMap.size());
+ for (Map.Entry<String, FunctionCallInformation> info : functionCallInfoMap.entrySet()) {
+ output.writeUTF(info.getKey());
+ info.getValue().writeFields(output);
+ }
+ }
+
+ /**
+ * Deserialize function call information map
+ *
+ * @param input data input
+ * @return deserialized function call information map
+ */
+ public static Map<String, FunctionCallInformation> createFunctionCallInformationMap(DataInput input)
+ throws IOException {
+ int size = input.readInt();
+ Map<String, FunctionCallInformation> functionCallInfoMap = new HashMap<>();
+ for (int i = 0; i < size; i++) {
+ String key = input.readUTF();
+ FunctionCallInformation functionCallInfo = FunctionCallInformation.create(input);
+ functionCallInfoMap.put(key, functionCallInfo);
+ }
+ return functionCallInfoMap;
+ }
+
+ private static ARecordType createType(String typeName) {
+ return new ARecordType(typeName, new String[] {}, new IAType[] {}, true);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
new file mode 100644
index 0000000..5cb26fd
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/projection/FunctionCallInformation.java
@@ -0,0 +1,112 @@
+/*
+ * 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.runtime.projection;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+
+/**
+ * Function call information that holds {@link FunctionIdentifier#getName()} and {@link SourceLocation}
+ */
+public class FunctionCallInformation implements Serializable {
+ private static final long serialVersionUID = -7884346933746232736L;
+ private final String functionName;
+ private final SourceLocation sourceLocation;
+ private Set<ATypeTag> typeMismatches;
+
+ public FunctionCallInformation(String functionName, SourceLocation sourceLocation) {
+ this(functionName, sourceLocation, Collections.emptySet());
+ }
+
+ private FunctionCallInformation(String functionName, SourceLocation sourceLocation, Set<ATypeTag> typeMismatches) {
+ this.functionName = functionName;
+ this.sourceLocation = sourceLocation;
+ this.typeMismatches = typeMismatches;
+ }
+
+ public String getFunctionName() {
+ return functionName;
+ }
+
+ public SourceLocation getSourceLocation() {
+ return sourceLocation;
+ }
+
+ public Warning createTypeMismatchWarning(ATypeTag expectedType, ATypeTag actualType) {
+ if (typeMismatches == null) {
+ typeMismatches = EnumSet.noneOf(ATypeTag.class);
+ } else if (typeMismatches.contains(actualType)) {
+ //We already issued a warning containing the same actual type. So, we ignore it
+ return null;
+ }
+ typeMismatches.add(actualType);
+ return Warning.of(getSourceLocation(), ErrorCode.TYPE_MISMATCH_FUNCTION, getFunctionName(),
+ ExceptionUtil.indexToPosition(0), expectedType, actualType);
+ }
+
+ public void writeFields(DataOutput output) throws IOException {
+ output.writeUTF(functionName);
+ SourceLocation.writeFields(sourceLocation, output);
+ output.writeInt(typeMismatches.size());
+ for (ATypeTag typeTag : typeMismatches) {
+ output.write(typeTag.serialize());
+ }
+ }
+
+ public static FunctionCallInformation create(DataInput in) throws IOException {
+ String functionName = in.readUTF();
+ SourceLocation sourceLocation = SourceLocation.create(in);
+ int typeMismatchesLength = in.readInt();
+ Set<ATypeTag> typeMismatches = EnumSet.noneOf(ATypeTag.class);
+ for (int i = 0; i < typeMismatchesLength; i++) {
+ typeMismatches.add(ATypeTag.VALUE_TYPE_MAPPING[in.readByte()]);
+ }
+ return new FunctionCallInformation(functionName, sourceLocation, typeMismatches);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(functionName, sourceLocation);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ FunctionCallInformation that = (FunctionCallInformation) o;
+ return Objects.equals(functionName, that.functionName) && Objects.equals(sourceLocation, that.sourceLocation);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
index 0306118..95de235 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/TidRunningAggregateDescriptor.java
@@ -22,7 +22,6 @@
import org.apache.asterix.om.base.AInt64;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
@@ -40,12 +39,7 @@
public class TidRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new TidRunningAggregateDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = TidRunningAggregateDescriptor::new;
@Override
public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
index 3f5f9f6..8a872c5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
@@ -21,6 +21,7 @@
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
@@ -60,7 +61,7 @@
for (int i = 0; i < args.length; i++) {
evals[i] = args[i].createScalarEvaluator(ctx);
}
- return new WinMarkFirstMissingRunningAggregateEvaluator(evals);
+ return new WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag.MISSING, evals);
}
};
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java
deleted file mode 100644
index c872921..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java
+++ /dev/null
@@ -1,97 +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.
- */
-
-package org.apache.asterix.runtime.runningaggregates.std;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IWindowAggregateEvaluator;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-public final class WinMarkFirstMissingRunningAggregateEvaluator implements IWindowAggregateEvaluator {
-
- @SuppressWarnings({ "rawtypes" })
- private final ISerializerDeserializer boolSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
-
- private final IScalarEvaluator[] argEvals;
-
- private final TaggedValuePointable argValue;
-
- private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-
- private final DataOutput dataOutput = resultStorage.getDataOutput();
-
- private boolean first;
-
- private boolean firstAllMissing;
-
- WinMarkFirstMissingRunningAggregateEvaluator(IScalarEvaluator[] argEvals) {
- this.argEvals = argEvals;
- argValue = TaggedValuePointable.FACTORY.createPointable();
- }
-
- @Override
- public void init() {
- }
-
- @Override
- public void initPartition(long partitionLength) {
- first = true;
- }
-
- @Override
- public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- boolean value = compute(tuple);
- resultStorage.reset();
- boolSerde.serialize(ABoolean.valueOf(value), dataOutput);
- result.set(resultStorage);
- }
-
- private boolean compute(IFrameTupleReference tuple) throws HyracksDataException {
- if (first) {
- firstAllMissing = everyArgIsMissing(tuple);
- first = false;
- return true;
- } else {
- boolean thisAllMissing = firstAllMissing || everyArgIsMissing(tuple);
- return !thisAllMissing;
- }
- }
-
- private boolean everyArgIsMissing(IFrameTupleReference tuple) throws HyracksDataException {
- for (IScalarEvaluator argEval : argEvals) {
- argEval.evaluate(tuple, argValue);
- if (argValue.getTag() != ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
- return false;
- }
- }
- return true;
- }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java
new file mode 100644
index 0000000..f8c2f3f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java
@@ -0,0 +1,72 @@
+/*
+ * 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.runtime.runningaggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This internal window function returns {@code TRUE} in the following two cases:
+ * <ol>
+ * <li>the argument is not NULL</li>
+ * <li>the argument is NULL and it comes from the first tuple in the current window partition</li>
+ * </ol>
+ * In all other cases the function returns {@code FALSE}.
+ * <p>
+ * The underlying assumption is that tuples in each window partition are sorted on the function's argument in the
+ * descending order.
+ */
+public class WinMarkFirstNullRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = WinMarkFirstNullRunningAggregateDescriptor::new;
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IRunningAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
+ throws HyracksDataException {
+ IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
+ for (int i = 0; i < args.length; i++) {
+ evals[i] = args[i].createScalarEvaluator(ctx);
+ }
+ return new WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag.NULL, evals);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.WIN_MARK_FIRST_NULL_IMPL;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
new file mode 100644
index 0000000..93ba4ef
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
@@ -0,0 +1,100 @@
+/*
+ * 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.runtime.runningaggregates.std;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IWindowAggregateEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public final class WinMarkFirstUnknownRunningAggregateEvaluator implements IWindowAggregateEvaluator {
+
+ @SuppressWarnings({ "rawtypes" })
+ private final ISerializerDeserializer boolSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+
+ private final IScalarEvaluator[] argEvals;
+
+ private final byte unknownTypeTag;
+
+ private final TaggedValuePointable argValue;
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+
+ private final DataOutput dataOutput = resultStorage.getDataOutput();
+
+ private boolean first;
+
+ private boolean firstAllUnknown;
+
+ WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag unknownTypeTag, IScalarEvaluator[] argEvals) {
+ this.argEvals = argEvals;
+ this.unknownTypeTag = unknownTypeTag.serialize();
+ argValue = TaggedValuePointable.FACTORY.createPointable();
+ }
+
+ @Override
+ public void init() {
+ }
+
+ @Override
+ public void initPartition(long partitionLength) {
+ first = true;
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ boolean value = compute(tuple);
+ resultStorage.reset();
+ boolSerde.serialize(ABoolean.valueOf(value), dataOutput);
+ result.set(resultStorage);
+ }
+
+ private boolean compute(IFrameTupleReference tuple) throws HyracksDataException {
+ if (first) {
+ firstAllUnknown = everyArgIsUnknown(tuple);
+ first = false;
+ return true;
+ } else {
+ boolean thisAllMissing = firstAllUnknown || everyArgIsUnknown(tuple);
+ return !thisAllMissing;
+ }
+ }
+
+ private boolean everyArgIsUnknown(IFrameTupleReference tuple) throws HyracksDataException {
+ for (IScalarEvaluator argEval : argEvals) {
+ argEval.evaluate(tuple, argValue);
+ if (argValue.getTag() != unknownTypeTag) {
+ return false;
+ }
+ }
+ return true;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
index dbaee0c..2292519 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/RangeDescriptor.java
@@ -21,7 +21,6 @@
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
@@ -43,12 +42,7 @@
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new RangeDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = RangeDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index 1e956cf..ff6dd89 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.runtime.evaluators.common.ListAccessor;
@@ -44,12 +43,7 @@
public class ScanCollectionDescriptor extends AbstractUnnestingFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new ScanCollectionDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = ScanCollectionDescriptor::new;
@Override
public FunctionIdentifier getIdentifier() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SpatialTileDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SpatialTileDescriptor.java
new file mode 100644
index 0000000..c16fc59
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SpatialTileDescriptor.java
@@ -0,0 +1,199 @@
+/*
+ * 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.runtime.unnestingfunctions.std;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class SpatialTileDescriptor extends AbstractUnnestingFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new SpatialTileDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.SPATIAL_TILE;
+ }
+
+ @Override
+ public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IUnnestingEvaluatorFactory() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IUnnestingEvaluator createUnnestingEvaluator(final IEvaluatorContext ctx)
+ throws HyracksDataException {
+ final IHyracksTaskContext hyracksTaskContext = ctx.getTaskContext();
+
+ return new IUnnestingEvaluator() {
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private List<Integer> tileValues = new ArrayList<>();
+ private final IPointable inputArg0 = new VoidPointable();
+ private final IPointable inputArg1 = new VoidPointable();
+ private final IPointable inputArg2 = new VoidPointable();
+ private final IPointable inputArg3 = new VoidPointable();
+ private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval2 = args[2].createScalarEvaluator(ctx);
+ private final IScalarEvaluator eval3 = args[3].createScalarEvaluator(ctx);
+
+ private final AMutableInt32 aInt32 = new AMutableInt32(0);
+ int pos;
+
+ @SuppressWarnings("unchecked")
+ private final ISerializerDeserializer intSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+ @Override
+ public void init(IFrameTupleReference tuple) throws HyracksDataException {
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+ eval2.evaluate(tuple, inputArg2);
+ eval3.evaluate(tuple, inputArg3);
+
+ byte[] bytes0 = inputArg0.getByteArray();
+ byte[] bytes1 = inputArg1.getByteArray();
+ byte[] bytes2 = inputArg2.getByteArray();
+ byte[] bytes3 = inputArg3.getByteArray();
+
+ int offset0 = inputArg0.getStartOffset();
+ int offset1 = inputArg1.getStartOffset();
+ int offset2 = inputArg2.getStartOffset();
+ int offset3 = inputArg3.getStartOffset();
+
+ ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[offset2]);
+ ATypeTag tag3 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes3[offset3]);
+
+ if (tag0 != ATypeTag.RECTANGLE) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+ if (tag1 != ATypeTag.RECTANGLE) {
+ throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+ ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+ }
+
+ double x1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double y1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double x2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double y2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+ double minX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+ double minY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+ double maxX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+ double maxY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+ + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+ int rows = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes2, offset2);
+ int columns = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes3, offset3);
+
+ // Unnest iff two input datasets overlap, which means partitioning MBR is not [(0,0),(0,0)]
+ tileValues.clear();
+ pos = 0;
+ if (!((minX == 0.0) && (minY == 0.0) && (maxX == 0.0) && (maxY == 0.0))) {
+ // Unnest iff the record overlaps with the partitioning MBR
+ if (!((x1 > maxX) || (minX > x2) || (y1 > maxY) || (minY > y2))) {
+ int row1 = (int) Math.ceil((y1 - minY) * rows / (maxY - minY));
+ int col1 = (int) Math.ceil((x1 - minX) * columns / (maxX - minX));
+ int row2 = (int) Math.ceil((y2 - minY) * rows / (maxY - minY));
+ int col2 = (int) Math.ceil((x2 - minX) * columns / (maxX - minX));
+
+ row1 = Math.min(Math.max(1, row1), rows * columns);
+ col1 = Math.min(Math.max(1, col1), rows * columns);
+ row2 = Math.min(Math.max(1, row2), rows * columns);
+ col2 = Math.min(Math.max(1, col2), rows * columns);
+
+ int minRow = Math.min(row1, row2);
+ int maxRow = Math.max(row1, row2);
+ int minCol = Math.min(col1, col2);
+ int maxCol = Math.max(col1, col2);
+
+ for (int i = minRow; i <= maxRow; i++) {
+ for (int j = minCol; j <= maxCol; j++) {
+ int tileId = (i - 1) * columns + j;
+ tileValues.add(tileId);
+ }
+ }
+ }
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public boolean step(IPointable result) throws HyracksDataException {
+ if (pos < tileValues.size()) {
+ aInt32.setValue(tileValues.get(pos));
+ resultStorage.reset();
+ intSerde.serialize(aInt32, resultStorage.getDataOutput());
+ result.set(resultStorage);
+ ++pos;
+ return true;
+ }
+ return false;
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
index c441a4e..7f5aa0a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SubsetCollectionDescriptor.java
@@ -25,7 +25,6 @@
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
@@ -46,12 +45,7 @@
public class SubsetCollectionDescriptor extends AbstractUnnestingFunctionDynamicDescriptor {
private static final long serialVersionUID = 1L;
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new SubsetCollectionDescriptor();
- }
- };
+ public static final IFunctionDescriptorFactory FACTORY = SubsetCollectionDescriptor::new;
@Override
public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index 9099c88..66f0e73 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -19,6 +19,7 @@
package org.apache.asterix.runtime.utils;
import java.io.IOException;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;
import org.apache.asterix.common.api.IConfigValidator;
@@ -100,6 +101,7 @@
private final IRequestTracker requestTracker;
private final IConfigValidator configValidator;
private final IAdapterFactoryService adapterFactoryService;
+ private final ReentrantReadWriteLock compilationLock = new ReentrantReadWriteLock(true);
public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
Supplier<IMetadataBootstrap> metadataBootstrapSupplier, IGlobalRecoveryManager globalRecoveryManager,
@@ -314,4 +316,9 @@
public IAdapterFactoryService getAdapterFactoryService() {
return adapterFactoryService;
}
+
+ @Override
+ public ReentrantReadWriteLock getCompilationLock() {
+ return compilationLock;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index 98a97b0..6de1dda 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -139,19 +139,15 @@
}
@Override
- public synchronized void updateNodeState(String nodeId, boolean active, NcLocalCounters localCounters) {
+ public synchronized void updateNodeState(String nodeId, boolean active, NcLocalCounters localCounters,
+ Set<Integer> activePartitions) {
if (active) {
updateClusterCounters(nodeId, localCounters);
participantNodes.add(nodeId);
+ activateNodePartitions(nodeId, activePartitions);
} else {
participantNodes.remove(nodeId);
- }
- ClusterPartition[] nodePartitions = node2PartitionsMap.get(nodeId);
- // if this isn't a storage node, it will not have cluster partitions
- if (nodePartitions != null) {
- for (ClusterPartition p : nodePartitions) {
- updateClusterPartition(p.getPartitionId(), nodeId, active);
- }
+ deactivateNodePartitions(nodeId);
}
}
@@ -416,11 +412,10 @@
public synchronized void deregisterNodePartitions(String nodeId) throws HyracksDataException {
ClusterPartition[] nodePartitions = node2PartitionsMap.remove(nodeId);
if (nodePartitions == null) {
- LOGGER.info("deregisterNodePartitions unknown node " + nodeId + " (already removed?)");
+ LOGGER.info("deregisterNodePartitions unknown node {} (already removed?)", nodeId);
} else {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("deregisterNodePartitions for node " + nodeId + ": " + Arrays.toString(nodePartitions));
- }
+ LOGGER.info("deregisterNodePartitions for node {}: {}", () -> nodeId,
+ () -> Arrays.toString(nodePartitions));
for (ClusterPartition nodePartition : nodePartitions) {
clusterPartitions.remove(nodePartition.getPartitionId());
}
@@ -431,12 +426,12 @@
@Override
public synchronized void removePending(String nodeId) {
if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Registering intention to remove node id " + nodeId);
+ LOGGER.info("Registering intention to remove node id {}", nodeId);
}
if (participantNodes.contains(nodeId)) {
pendingRemoval.add(nodeId);
} else {
- LOGGER.warn("Cannot register unknown node " + nodeId + " for pending removal");
+ LOGGER.warn("Cannot register unknown node {} for pending removal", nodeId);
}
}
@@ -478,6 +473,16 @@
refreshState();
}
+ @Override
+ public Map<Integer, ClusterPartition> getClusterPartitions() {
+ return Collections.unmodifiableMap(clusterPartitions);
+ }
+
+ @Override
+ public synchronized boolean nodesFailed(Set<String> nodeIds) {
+ return nodeIds.stream().anyMatch(failedNodes::contains);
+ }
+
private void updateClusterCounters(String nodeId, NcLocalCounters localCounters) {
final IResourceIdManager resourceIdManager = appCtx.getResourceIdManager();
resourceIdManager.report(nodeId, localCounters.getMaxResourceId());
@@ -496,6 +501,19 @@
});
}
+ private synchronized void activateNodePartitions(String nodeId, Set<Integer> activePartitions) {
+ for (Integer partitionId : activePartitions) {
+ updateClusterPartition(partitionId, nodeId, true);
+ }
+ }
+
+ private synchronized void deactivateNodePartitions(String nodeId) {
+ clusterPartitions.values().stream()
+ .filter(partition -> partition.getActiveNodeId() != null && partition.getActiveNodeId().equals(nodeId))
+ .forEach(nodeActivePartition -> updateClusterPartition(nodeActivePartition.getPartitionId(), nodeId,
+ false));
+ }
+
private static InetSocketAddress getReplicaLocation(IClusterStateManager csm, String nodeId) {
final Map<IOption, Object> ncConfig = csm.getActiveNcConfiguration().get(nodeId);
if (ncConfig == null) {
diff --git a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
index cbde36c..a18b04f 100644
--- a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
+++ b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
@@ -221,7 +221,8 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) {
throw new IllegalStateException();
}
diff --git a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorTest.java b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorTest.java
index 1db124a..81b5c12 100644
--- a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorTest.java
+++ b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorTest.java
@@ -273,7 +273,7 @@
LSMSecondaryIndexCreationTupleProcessorNodePushable op =
new LSMSecondaryIndexCreationTupleProcessorNodePushable(ctx, 0, recDesc, MissingWriterFactory.INSTANCE,
- numTagFields, numSecondaryKeys, numPrimaryKeys, hasBuddyBTree);
+ numTagFields, numSecondaryKeys, numPrimaryKeys, hasBuddyBTree, true, true);
op.setOutputFrameWriter(0, resultWriter, recDesc);
op.open();
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 23a63d0..1c7208e 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -171,8 +171,21 @@
<url>https://raw.githubusercontent.com/mojohaus/appassembler/appassembler-2.0.0/LICENSE.txt</url>
</override>
<override>
- <gav>io.netty:netty-all:4.1.63.Final</gav>
- <noticeUrl>https://raw.githubusercontent.com/netty/netty/netty-4.1.63.Final/NOTICE.txt</noticeUrl>
+ <gavs>
+ <gav>io.netty:netty-buffer:4.1.69.Final</gav>
+ <gav>io.netty:netty-codec:4.1.69.Final</gav>
+ <gav>io.netty:netty-codec-http:4.1.69.Final</gav>
+ <gav>io.netty:netty-common:4.1.69.Final</gav>
+ <gav>io.netty:netty-handler:4.1.69.Final</gav>
+ <gav>io.netty:netty-resolver:4.1.69.Final</gav>
+ <gav>io.netty:netty-transport:4.1.69.Final</gav>
+ </gavs>
+ <noticeUrl>https://raw.githubusercontent.com/netty/netty/netty-4.1.69.Final/NOTICE.txt</noticeUrl>
+ </override>
+ <override>
+ <gav>io.netty:netty-tcnative-classes:2.0.46.Final</gav>
+ <url>http://www.apache.org/licenses/LICENSE-2.0</url>
+ <noticeUrl>https://raw.githubusercontent.com/netty/netty-tcnative/netty-tcnative-parent-2.0.46.Final/NOTICE.txt</noticeUrl>
</override>
<override>
<gav>org.reactivestreams:reactive-streams:1.0.2</gav>
@@ -180,6 +193,11 @@
<url>https://raw.githubusercontent.com/reactive-streams/reactive-streams-jvm/v1.0.2/LICENSE.txt</url>
</override>
<override>
+ <gav>org.reactivestreams:reactive-streams:1.0.3</gav>
+ <noticeUrl>https://raw.githubusercontent.com/reactive-streams/reactive-streams-jvm/v1.0.3/COPYING.txt</noticeUrl>
+ <url>https://raw.githubusercontent.com/reactive-streams/reactive-streams-jvm/v1.0.3/LICENSE.txt</url>
+ </override>
+ <override>
<gavs>
<gav>software.amazon.awssdk:sdk-core:2.10.83</gav>
<gav>software.amazon.awssdk:aws-core:2.10.83</gav>
@@ -199,19 +217,201 @@
</gavs>
<noticeUrl>https://raw.githubusercontent.com/aws/aws-sdk-java-v2/2.10.83/NOTICE.txt</noticeUrl>
</override>
+ <!-- Hadoop AWS SDK -->
+ <override>
+ <gavs>
+ <gav>com.amazonaws:aws-java-sdk-core:1.12.109</gav>
+ <gav>com.amazonaws:jmespath-java:1.12.109</gav>
+ <gav>com.amazonaws:aws-java-sdk-s3:1.12.109</gav>
+ <gav>com.amazonaws:aws-java-sdk-kms:1.12.109</gav>
+ <gav>com.amazonaws:aws-java-sdk-dynamodb:1.12.109</gav>
+ </gavs>
+ <noticeUrl>https://raw.githubusercontent.com/aws/aws-sdk-java/1.12.109/NOTICE.txt</noticeUrl>
+ </override>
<override>
<gav>software.amazon.eventstream:eventstream:1.0.1</gav>
<noticeUrl>https://raw.githubusercontent.com/awslabs/aws-eventstream-java/7be2dd80e12f8835674c8ffb0f4a2efb64c7b585/NOTICE</noticeUrl>
</override>
<override>
+ <gav>software.amazon.ion:ion-java:1.0.2</gav>
+ <noticeUrl>https://raw.githubusercontent.com/amzn/ion-java/v1.0.2/NOTICE</noticeUrl>
+ </override>
+ <override>
<gavs>
- <gav>com.azure:azure-core:1.4.0</gav>
- <gav>com.azure:azure-core-http-netty:1.5.0</gav>
- <gav>com.azure:azure-storage-blob:12.6.0</gav>
- <gav>com.azure:azure-storage-common:12.6.0</gav>
+ <gav>com.azure:azure-core:1.24.1</gav>
+ <gav>com.azure:azure-core-http-netty:1.11.6</gav>
+ <gav>com.azure:azure-storage-blob:12.14.3</gav>
+ <gav>com.azure:azure-storage-common:12.14.2</gav>
+ <gav>com.azure:azure-storage-internal-avro:12.1.3</gav>
</gavs>
- <noticeUrl>https://raw.githubusercontent.com/Azure/azure-sdk-for-java/master/NOTICE.txt</noticeUrl>
- <url>https://raw.githubusercontent.com/Azure/azure-sdk-for-java/master/LICENSE.txt</url>
+ <!-- azure-storage-blob-batch_12.11.3 release is at 89a32290750a18d1b99c27c16b1b11d42f16c622 -->
+ <noticeUrl>https://raw.githubusercontent.com/Azure/azure-sdk-for-java/89a32290750a18d1b99c27c16b1b11d42f16c622/NOTICE.txt</noticeUrl>
+ <url>https://raw.githubusercontent.com/Azure/azure-sdk-for-java/89a32290750a18d1b99c27c16b1b11d42f16c622/LICENSE.txt</url>
+ </override>
+ <override>
+ <gav>com.microsoft.azure:msal4j:1.11.0</gav>
+ <url>https://raw.githubusercontent.com/AzureAD/microsoft-authentication-library-for-java/v1.11.0/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.microsoft.azure:msal4j-persistence-extension:1.1.0</gav>
+ <url>https://raw.githubusercontent.com/AzureAD/microsoft-authentication-extensions-for-java/1.1.0/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.microsoft.azure:azure-keyvault-core:1.2.4</gav>
+ <url>https://raw.githubusercontent.com/Azure/azure-sdk-for-java/main/sdk/keyvault/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.nimbusds:nimbus-jose-jwt:9.9.3</gav>
+ <noticeUrl>https://bitbucket.org/connect2id/nimbus-jose-jwt/raw/50ae2a39a4a6d0dcbf05572af8a581377174ac96/COPYRIGHT.txt</noticeUrl>
+ </override>
+ <!-- com.nimbusds:oauth2-oidc-sdk:9.7 points to https://www.apache.org/licenses/LICENSE-2.0.html in the pom.xml but has bad characters in the URI, overriding to fix -->
+ <override>
+ <gav>com.nimbusds:oauth2-oidc-sdk:9.7</gav>
+ <url>https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions/raw/5d13925b57ace092ea5e1131c338f464d85545f4/LICENSE.txt</url>
+ <noticeUrl>https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions/raw/5d13925b57ace092ea5e1131c338f464d85545f4/COPYRIGHT.txt</noticeUrl>
+ </override>
+ <override>
+ <gav>com.nimbusds:lang-tag:1.5</gav>
+ <noticeUrl>https://bitbucket.org/connect2id/nimbus-language-tags/raw/ead5120f62b5849309069808509b4cc6e57a0841/COPYRIGHT.txt</noticeUrl>
+ </override>
+ <override>
+ <gav>com.nimbusds:content-type:2.1</gav>
+ <noticeUrl>https://bitbucket.org/connect2id/nimbus-content-type/raw/c6d2701ed5cd57e0f88728c6f3e6303db97c5bcf/COPYRIGHT.txt</noticeUrl>
+ </override>
+ <override>
+ <gavs>
+ <gav>io.projectreactor.netty:reactor-netty:1.0.7</gav>
+ <gav>io.projectreactor.netty:reactor-netty-core:1.0.7</gav>
+ <gav>io.projectreactor.netty:reactor-netty-http:1.0.7</gav>
+ <gav>io.projectreactor.netty:reactor-netty-http-brave:1.0.7</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/reactor/reactor-netty/v1.0.7/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>io.projectreactor.netty:reactor-netty-core:1.0.11</gav>
+ <gav>io.projectreactor.netty:reactor-netty-http:1.0.11</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/reactor/reactor-netty/v1.0.11/LICENSE</url>
+ </override>
+ <override>
+ <gav>io.projectreactor:reactor-core:3.4.6</gav>
+ <url>https://raw.githubusercontent.com/reactor/reactor-core/v3.4.6/LICENSE</url>
+ </override>
+ <override>
+ <gav>io.projectreactor:reactor-core:3.4.10</gav>
+ <url>https://raw.githubusercontent.com/reactor/reactor-core/v3.4.10/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>stax:stax-api:1.0.1</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/codehaus/stax/master/dev/ASF2.0.txt</url>
+ </override>
+ <override>
+ <gav>org.ow2.asm:asm:9.1</gav>
+ <url>https://raw.githubusercontent.com/llbit/ow2-asm/master/LICENSE.txt</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.http-client:google-http-client-jackson2:1.39.2</gav>
+ <gav>com.google.http-client:google-http-client-appengine:1.39.2</gav>
+ <gav>com.google.http-client:google-http-client-gson:1.39.2</gav>
+ <gav>com.google.http-client:google-http-client-apache-v2:1.39.2</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/googleapis/google-http-java-client/v1.39.2/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.oauth-client:google-oauth-client:1.31.5</gav>
+ <url>https://raw.githubusercontent.com/googleapis/google-oauth-java-client/v1.31.5/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.protobuf:protobuf-java:3.16.0</gav>
+ <gav>com.google.protobuf:protobuf-java-util:3.16.0</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/protocolbuffers/protobuf/v3.16.0/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.auth:google-auth-library-oauth2-http:0.25.5</gav>
+ <gav>com.google.auth:google-auth-library-credentials:0.25.5</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/googleapis/google-auth-library-java/v0.25.5/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.cloud:google-cloud-core:1.94.8</gav>
+ <gav>com.google.cloud:google-cloud-core-http:1.94.8</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/googleapis/java-core/v1.94.8/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.cloud:google-cloud-storage:1.114.0</gav>
+ <url>https://raw.githubusercontent.com/googleapis/java-storage/v1.114.0/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>io.opencensus:opencensus-api:0.28.0</gav>
+ <gav>io.opencensus:opencensus-contrib-http-util:0.28.0</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/census-instrumentation/opencensus-java/v0.28.0/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api-client:google-api-client:1.31.5</gav>
+ <url>https://raw.githubusercontent.com/googleapis/google-api-java-client/v1.31.5/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api.grpc:proto-google-iam-v1:1.0.13</gav>
+ <url>https://raw.githubusercontent.com/googleapis/java-iam/v1.0.13/proto-google-iam-v1/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api.grpc:proto-google-common-protos:2.1.0</gav>
+ <url>https://raw.githubusercontent.com/googleapis/java-common-protos/v2.1.0/proto-google-common-protos/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.api:api-common:1.10.3</gav>
+ <url>https://raw.githubusercontent.com/googleapis/api-common-java/v1.10.3/LICENSE</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>com.google.api:gax-httpjson:0.81.0</gav>
+ <gav>com.google.api:gax:1.64.0</gav>
+ </gavs>
+ <url>https://raw.githubusercontent.com/googleapis/gax-java/v1.64.0/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.auto.value:auto-value-annotations:1.8.1</gav>
+ <url>https://raw.githubusercontent.com/google/auto/auto-value-1.8.1/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.code.findbugs:jsr305:3.0.2</gav>
+ <url>https://raw.githubusercontent.com/findbugsproject/findbugs/3.0.2_preview2/findbugs/licenses/LICENSE-jsr305.txt</url>
+ </override>
+ <override>
+ <gav>com.google.code.gson:gson:2.8.6</gav>
+ <url>https://raw.githubusercontent.com/google/gson/gson-parent-2.8.6/LICENSE</url>
+ </override>
+ <override>
+ <gav>com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava</gav>
+ <url>https://raw.githubusercontent.com/google/guava/master/COPYING</url>
+ </override>
+ <override>
+ <gav>com.google.j2objc:j2objc-annotations:1.3</gav>
+ <url>https://raw.githubusercontent.com/google/j2objc/1.3/LICENSE</url>
+ </override>
+ <override>
+ <gav>io.grpc:grpc-context:1.37.0</gav>
+ <url>https://raw.githubusercontent.com/grpc/grpc-java/v1.37.0/LICENSE</url>
+ <noticeUrl>https://raw.githubusercontent.com/grpc/grpc-java/v1.37.0/NOTICE.txt</noticeUrl>
+ </override>
+ <override>
+ <gav>org.checkerframework:checker-compat-qual:2.5.5</gav>
+ <url>https://raw.githubusercontent.com/typetools/checker-framework/checker-framework-2.5.5/LICENSE.txt</url>
+ </override>
+ <override>
+ <gav>com.google.api:api-common:1.10.3</gav>
+ <url>https://raw.githubusercontent.com/googleapis/api-common-java/v1.10.3/LICENSE</url>
</override>
<override>
<gav>org.mindrot:jbcrypt:0.4</gav>
@@ -221,6 +421,10 @@
<gav>org.msgpack:msgpack-core:0.8.20</gav>
<noticeUrl>https://raw.githubusercontent.com/msgpack/msgpack-java/0.8.20/NOTICE</noticeUrl>
</override>
+ <override>
+ <gav>com.github.luben:zstd-jni:1.4.9-1</gav>
+ <url>https://raw.githubusercontent.com/luben/zstd-jni/v1.4.9-1/LICENSE</url>
+ </override>
</overrides>
<licenses>
<license>
@@ -254,7 +458,21 @@
<aliasUrl>http://www.apache.org/licenses/LICENSE-2.0</aliasUrl>
<aliasUrl>https://www.apache.org/licenses/LICENSE-2.0.txt</aliasUrl>
<aliasUrl>http://www.apache.org/licenses/LICENSE-2.0.html</aliasUrl>
+ <aliasUrl>https://www.apache.org/licenses/LICENSE-2.0.html</aliasUrl>
<aliasUrl>https://aws.amazon.com/apache2.0</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/jorabin/KeePassJava2/KeePassJava2-2.1.4/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/reactor/reactor-netty/v1.0.7/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/reactor/reactor-netty/v1.0.11/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/reactor/reactor-core/v3.4.6/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/reactor/reactor-core/v3.4.10/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/codehaus/stax/master/dev/ASF2.0.txt</aliasUrl>
+ <aliasUrl>https://bitbucket.org/connect2id/oauth-2.0-sdk-with-openid-connect-extensions/raw/5d13925b57ace092ea5e1131c338f464d85545f4/LICENSE.txt</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/auto/auto-value-1.8.1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/google/j2objc/1.3/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-common-protos/v2.1.0/proto-google-common-protos/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-iam/v1.0.13/proto-google-iam-v1/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/googleapis/java-storage/v1.114.0/LICENSE</aliasUrl>
+ <aliasUrl>http://repository.jboss.org/licenses/apache-2.0.txt</aliasUrl>
</aliasUrls>
<metric>1</metric>
</license>
@@ -303,6 +521,8 @@
<aliasUrls>
<aliasUrl>http://www.opensource.org/licenses/mit-license.php</aliasUrl>
<aliasUrl>http://opensource.org/licenses/MIT</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/AzureAD/microsoft-authentication-library-for-java/v1.11.0/LICENSE</aliasUrl>
+ <aliasUrl>https://raw.githubusercontent.com/AzureAD/microsoft-authentication-extensions-for-java/1.1.0/LICENSE</aliasUrl>
</aliasUrls>
</license>
<license>
@@ -312,20 +532,38 @@
</aliasUrls>
</license>
<license>
- <url>https://opensource.org/licenses/bsd-license.php</url>
- <aliasUrls>http://www.opensource.org/licenses/bsd-license.php</aliasUrls>
- </license>
- <license>
<!-- this is special case- handled in ftl template -->
<url>PUBLIC_DOMAIN</url>
<aliasUrls>Public Domain</aliasUrls>
</license>
+ <license>
+ <url>https://creativecommons.org/publicdomain/zero/1.0/legalcode.txt</url>
+ <aliasUrls>https://creativecommons.org/publicdomain/zero/1.0/legalcode</aliasUrls>
+ <aliasUrls>https://creativecommons.org/publicdomain/zero/1.0/</aliasUrls>
+ <aliasUrls>http://creativecommons.org/publicdomain/zero/1.0/legalcode</aliasUrls>
+ <aliasUrls>http://creativecommons.org/publicdomain/zero/1.0/</aliasUrls>
+ </license>
+ <license>
+ <displayName>The 2-Clause BSD License</displayName>
+ <url>https://opensource.org/licenses/BSD-2-Clause</url>
+ <contentFile>raw.githubusercontent.com_luben_zstd-jni_v1.4.9-1_LICENSE.txt</contentFile>
+ <aliasUrls>
+ <aliasURL>https://opensource.org/licenses/bsd-license.php</aliasURL>
+ <aliasUrl>https://raw.githubusercontent.com/luben/zstd-jni/v1.4.9-1/LICENSE</aliasUrl>
+ </aliasUrls>
+ </license>
+ <license>
+ <displayName>The Go license</displayName>
+ <url>http://golang.org/LICENSE</url>
+ <aliasUrls>
+ <aliasUrl>https://golang.org/LICENSE</aliasUrl>
+ </aliasUrls>
+ </license>
</licenses>
<templateProperties>
<packageName>Apache AsterixDB Server Install</packageName>
<asterixAppLocation>repo/asterix-app-${project.version}.jar, within this assembly</asterixAppLocation>
<hyracksControlCcLocation>repo/hyracks-control-cc-${project.version}.jar, within this assembly</hyracksControlCcLocation>
- <hivecompatLocation>repo/asterix-hivecompat-${project.version}.jar, within this assembly</hivecompatLocation>
<asterixDashboardLocation>repo/asterix-dashboard-${project.version}.jar, within this assembly</asterixDashboardLocation>
</templateProperties>
</configuration>
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm
index 7e92c87..a34a57a 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm
@@ -2,6 +2,7 @@
"partition" : 0,
"replicas" : [ {
"location" : "127.0.0.1:2002",
+ "nodeId" : "asterix_nc2",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm
index 7e92c87..a34a57a 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm
@@ -2,6 +2,7 @@
"partition" : 0,
"replicas" : [ {
"location" : "127.0.0.1:2002",
+ "nodeId" : "asterix_nc2",
"status" : "IN_SYNC"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm
index 4da3d42..348beca 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm
@@ -2,6 +2,7 @@
"partition" : 0,
"replicas" : [ {
"location" : "127.0.0.1:2002",
+ "nodeId" : "asterix_nc2",
"status" : "DISCONNECTED"
} ]
} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
index 859e0bf..6e458a4 100644
--- a/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
+++ b/asterixdb/asterix-test-framework/src/main/java/org/apache/asterix/testframework/context/TestCaseContext.java
@@ -21,6 +21,7 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.BitSet;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -37,77 +38,18 @@
public class TestCaseContext {
- /**
- * For specifying the desired output formatting of results.
- */
- public enum OutputFormat {
- NONE("", ""),
- ADM("adm", "application/x-adm"),
- LOSSLESS_JSON("json", "application/json; lossless=true"),
- CLEAN_JSON("json", "application/json"),
- CSV("csv", "text/csv"),
- CSV_HEADER("csv-header", "text/csv; header=present"),
- AST("ast", "application/x-ast"),
- PLAN("plan", "application/x-plan"),
- BINARY("", "application/octet-stream");
-
- private final String extension;
- private final String mimetype;
-
- OutputFormat(String ext, String mime) {
- this.extension = ext;
- this.mimetype = mime;
- }
-
- public String extension() {
- return extension;
- }
-
- public String mimeType() {
- return mimetype;
- }
-
- //
- public static OutputFormat forCompilationUnit(CompilationUnit cUnit) {
- switch (cUnit.getOutputDir().getCompare()) {
- case TEXT:
- return OutputFormat.ADM;
- case LOSSLESS_JSON:
- return OutputFormat.LOSSLESS_JSON;
- case CLEAN_JSON:
- return OutputFormat.CLEAN_JSON;
- case CSV:
- return OutputFormat.CSV;
- case CSV_HEADER:
- return OutputFormat.CSV_HEADER;
- case BINARY:
- return OutputFormat.BINARY;
- case INSPECT:
- case IGNORE:
- return OutputFormat.NONE;
- case AST:
- return OutputFormat.AST;
- case PLAN:
- return OutputFormat.PLAN;
- default:
- assert false : "Unknown ComparisonEnum!";
- return OutputFormat.NONE;
- }
- }
- }
-
public static final String DEFAULT_TESTSUITE_XML_NAME = "testsuite.xml";
public static final String ONLY_TESTSUITE_XML_NAME = "only.xml";
public static final String DEFAULT_REPEATED_TESTSUITE_XML_NAME = "repeatedtestsuite.xml";
private File tsRoot;
-
private TestSuite testSuite;
-
private TestGroup[] testGroups;
-
private TestCase testCase;
private Map<String, Object> kv;
+ public int numOfErrors;
+ public List<String> expectedErrors;
+ public BitSet expectedWarnings;
public TestCaseContext(File tsRoot, TestSuite testSuite, TestGroup[] testGroups, TestCase testCase) {
this.tsRoot = tsRoot;
@@ -234,16 +176,16 @@
}
public static class Builder {
- private final boolean m_doSlow;
- private final Pattern m_re;
+ private final boolean doSlow;
+ private final Pattern re;
public Builder() {
- m_doSlow = System.getProperty("runSlowAQLTests", "false").equals("true");
- String re = System.getProperty("testre");
- if (re == null) {
- m_re = null;
+ doSlow = System.getProperty("runSlowAQLTests", "false").equals("true");
+ String testre = System.getProperty("testre");
+ if (testre == null) {
+ this.re = null;
} else {
- m_re = Pattern.compile(re);
+ this.re = Pattern.compile(testre);
}
}
@@ -280,13 +222,13 @@
private void addContexts(File tsRoot, TestSuite ts, List<TestGroup> tgPath, List<TestCaseContext> tccs) {
TestGroup tg = tgPath.get(tgPath.size() - 1);
for (TestCase tc : tg.getTestCase()) {
- if (m_doSlow || tc.getCategory() != CategoryEnum.SLOW) {
+ if (doSlow || tc.getCategory() != CategoryEnum.SLOW) {
boolean matches = false;
- if (m_re != null) {
+ if (re != null) {
// Check all compilation units for matching
// name. If ANY match, add the test.
for (TestCase.CompilationUnit cu : tc.getCompilationUnit()) {
- if (m_re.matcher(cu.getName()).find()) {
+ if (re.matcher(cu.getName()).find()) {
matches = true;
break;
}
@@ -304,4 +246,66 @@
addContexts(tsRoot, ts, tgPath, tg.getTestGroup(), tccs);
}
}
+
+ /**
+ * For specifying the desired output formatting of results.
+ */
+ public enum OutputFormat {
+ NONE("", ""),
+ ADM("adm", "application/x-adm"),
+ LOSSLESS_JSON("json", "application/json; lossless=true"),
+ LOSSLESS_ADM_JSON("json", "application/json; lossless-adm=true"),
+ CLEAN_JSON("json", "application/json"),
+ CSV("csv", "text/csv"),
+ CSV_HEADER("csv-header", "text/csv; header=present"),
+ AST("ast", "application/x-ast"),
+ PLAN("plan", "application/x-plan"),
+ BINARY("", "application/octet-stream");
+
+ private final String extension;
+ private final String mimetype;
+
+ OutputFormat(String ext, String mime) {
+ this.extension = ext;
+ this.mimetype = mime;
+ }
+
+ public String extension() {
+ return extension;
+ }
+
+ public String mimeType() {
+ return mimetype;
+ }
+
+ //
+ public static OutputFormat forCompilationUnit(CompilationUnit cUnit) {
+ switch (cUnit.getOutputDir().getCompare()) {
+ case TEXT:
+ return OutputFormat.ADM;
+ case LOSSLESS_JSON:
+ return OutputFormat.LOSSLESS_JSON;
+ case LOSSLESS_ADM_JSON:
+ return OutputFormat.LOSSLESS_ADM_JSON;
+ case CLEAN_JSON:
+ return OutputFormat.CLEAN_JSON;
+ case CSV:
+ return OutputFormat.CSV;
+ case CSV_HEADER:
+ return OutputFormat.CSV_HEADER;
+ case BINARY:
+ return OutputFormat.BINARY;
+ case INSPECT:
+ case IGNORE:
+ return OutputFormat.NONE;
+ case AST:
+ return OutputFormat.AST;
+ case PLAN:
+ return OutputFormat.PLAN;
+ default:
+ assert false : "Unknown ComparisonEnum!";
+ return OutputFormat.NONE;
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
index 4ea9e0e..303a4d7 100644
--- a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
+++ b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
@@ -252,6 +252,7 @@
<xs:enumeration value="Ignore"/>
<xs:enumeration value="Clean-JSON"/>
<xs:enumeration value="Lossless-JSON"/>
+ <xs:enumeration value="Lossless-ADM-JSON"/>
<xs:enumeration value="CSV"/>
<xs:enumeration value="CSV_Header"/>
<xs:enumeration value="AST"/>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 29dedf7..6577367 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -19,10 +19,11 @@
package org.apache.asterix.transaction.management.resource;
import static org.apache.asterix.common.storage.ResourceReference.getComponentSequence;
-import static org.apache.asterix.common.utils.StorageConstants.INDEX_CHECKPOINT_FILE_PREFIX;
+import static org.apache.asterix.common.utils.StorageConstants.INDEX_NON_DATA_FILES_PREFIX;
import static org.apache.asterix.common.utils.StorageConstants.METADATA_FILE_NAME;
import static org.apache.hyracks.api.exceptions.ErrorCode.CANNOT_CREATE_FILE;
import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.COMPONENT_FILES_FILTER;
+import static org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager.UNINITIALIZED_COMPONENT_SEQ;
import java.io.File;
import java.io.FilenameFilter;
@@ -32,7 +33,9 @@
import java.nio.file.Paths;
import java.text.ParseException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -57,7 +60,6 @@
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.filefilter.IOFileFilter;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IIOManager;
@@ -89,45 +91,14 @@
private static final String METADATA_FILE_MASK_NAME =
StorageConstants.MASK_FILE_PREFIX + StorageConstants.METADATA_FILE_NAME;
private static final FilenameFilter LSM_INDEX_FILES_FILTER =
- (dir, name) -> !name.startsWith(INDEX_CHECKPOINT_FILE_PREFIX);
+ (dir, name) -> name.startsWith(METADATA_FILE_NAME) || !name.startsWith(INDEX_NON_DATA_FILES_PREFIX);
private static final FilenameFilter MASK_FILES_FILTER =
(dir, name) -> name.startsWith(StorageConstants.MASK_FILE_PREFIX);
private static final int MAX_CACHED_RESOURCES = 1000;
- private static final IOFileFilter METADATA_FILES_FILTER = new IOFileFilter() {
- @Override
- public boolean accept(File file) {
- return file.getName().equals(StorageConstants.METADATA_FILE_NAME);
- }
-
- @Override
- public boolean accept(File dir, String name) {
- return false;
- }
- };
-
- private static final IOFileFilter METADATA_MASK_FILES_FILTER = new IOFileFilter() {
- @Override
- public boolean accept(File file) {
- return file.getName().equals(METADATA_FILE_MASK_NAME);
- }
-
- @Override
- public boolean accept(File dir, String name) {
- return false;
- }
- };
-
- private static final IOFileFilter ALL_DIR_FILTER = new IOFileFilter() {
- @Override
- public boolean accept(File file) {
- return true;
- }
-
- @Override
- public boolean accept(File dir, String name) {
- return true;
- }
- };
+ private static final FilenameFilter METADATA_FILES_FILTER =
+ (dir, name) -> name.equals(StorageConstants.METADATA_FILE_NAME);
+ private static final FilenameFilter METADATA_MASK_FILES_FILTER =
+ (dir, name) -> name.equals(METADATA_FILE_MASK_NAME);
// Finals
private final IIOManager ioManager;
@@ -136,7 +107,7 @@
private boolean isReplicationEnabled = false;
private Set<String> filesToBeReplicated;
private IReplicationManager replicationManager;
- private final Path[] storageRoots;
+ private final List<Path> storageRoots;
private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
private final IPersistedResourceRegistry persistedResourceRegistry;
@@ -146,11 +117,11 @@
this.ioManager = ioManager;
this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
this.persistedResourceRegistry = persistedResourceRegistry;
- storageRoots = new Path[ioManager.getIODevices().size()];
+ storageRoots = new ArrayList<>();
final List<IODeviceHandle> ioDevices = ioManager.getIODevices();
for (int i = 0; i < ioDevices.size(); i++) {
- storageRoots[i] =
- Paths.get(ioDevices.get(i).getMount().getAbsolutePath(), StorageConstants.STORAGE_ROOT_DIR_NAME);
+ storageRoots.add(
+ Paths.get(ioDevices.get(i).getMount().getAbsolutePath(), StorageConstants.STORAGE_ROOT_DIR_NAME));
}
createStorageRoots();
resourceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHED_RESOURCES).build();
@@ -198,8 +169,8 @@
createResourceFileMask(resourceFile);
byte[] bytes = OBJECT_MAPPER.writeValueAsBytes(resource.toJson(persistedResourceRegistry));
FileUtil.writeAndForce(Paths.get(resourceFile.getAbsolutePath()), bytes);
- indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(Long.MIN_VALUE, 0,
- LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId());
+ indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(UNINITIALIZED_COMPONENT_SEQ,
+ 0, LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId(), null);
deleteResourceFileMask(resourceFile);
} catch (Exception e) {
cleanup(resourceFile);
@@ -211,7 +182,11 @@
resourceCache.put(resource.getPath(), resource);
//if replication enabled, send resource metadata info to remote nodes
if (isReplicationEnabled) {
- createReplicationJob(ReplicationOperation.REPLICATE, resourceFile);
+ try {
+ createReplicationJob(ReplicationOperation.REPLICATE, resourceFile);
+ } catch (Exception e) {
+ LOGGER.error("failed to send resource file {} to replicas", resourceFile);
+ }
}
}
@@ -232,8 +207,10 @@
FileReference resourceFile = getLocalResourceFileByName(ioManager, relativePath);
try {
if (resourceFile.getFile().exists()) {
- if (isReplicationEnabled) {
+ try {
createReplicationJob(ReplicationOperation.DELETE, resourceFile);
+ } catch (Exception e) {
+ LOGGER.error("failed to delete resource file {} from replicas", resourceFile);
}
final LocalResource localResource = readLocalResource(resourceFile.getFile());
IoUtil.delete(resourceFile);
@@ -248,17 +225,20 @@
}
}
- private static FileReference getLocalResourceFileByName(IIOManager ioManager, String resourcePath)
+ public static FileReference getLocalResourceFileByName(IIOManager ioManager, String resourcePath)
throws HyracksDataException {
String fileName = resourcePath + File.separator + StorageConstants.METADATA_FILE_NAME;
return ioManager.resolve(fileName);
}
- public synchronized Map<Long, LocalResource> getResources(Predicate<LocalResource> filter)
+ public synchronized Map<Long, LocalResource> getResources(Predicate<LocalResource> filter, List<Path> roots)
throws HyracksDataException {
Map<Long, LocalResource> resourcesMap = new HashMap<>();
- for (Path root : storageRoots) {
- final Collection<File> files = FileUtils.listFiles(root.toFile(), METADATA_FILES_FILTER, ALL_DIR_FILTER);
+ for (Path root : roots) {
+ if (!Files.exists(root) || !Files.isDirectory(root)) {
+ continue;
+ }
+ final Collection<File> files = IoUtil.getMatchingFiles(root, METADATA_FILES_FILTER);
try {
for (File file : files) {
final LocalResource localResource = readLocalResource(file);
@@ -273,12 +253,44 @@
return resourcesMap;
}
+ public synchronized Map<Long, LocalResource> getResources(Predicate<LocalResource> filter)
+ throws HyracksDataException {
+ return getResources(filter, storageRoots);
+ }
+
+ public synchronized Map<Long, LocalResource> getResources(Predicate<LocalResource> filter, Set<Integer> partitions)
+ throws HyracksDataException {
+ List<Path> partitionsRoots = new ArrayList<>();
+ for (Integer partition : partitions) {
+ partitionsRoots.add(getPartitionRoot(partition));
+ }
+ return getResources(filter, partitionsRoots);
+ }
+
+ public synchronized void deleteInvalidIndexes(Predicate<LocalResource> filter) throws HyracksDataException {
+ for (Path root : storageRoots) {
+ final Collection<File> files = IoUtil.getMatchingFiles(root, METADATA_FILES_FILTER);
+ try {
+ for (File file : files) {
+ final LocalResource localResource = readLocalResource(file);
+ if (filter.test(localResource)) {
+ LOGGER.warn("deleting invalid metadata index {}", file.getParentFile());
+ IoUtil.delete(file.getParentFile());
+ }
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ resourceCache.invalidateAll();
+ }
+
public Map<Long, LocalResource> loadAndGetAllResources() throws HyracksDataException {
return getResources(p -> true);
}
@Override
- public long maxId() throws HyracksDataException {
+ public synchronized long maxId() throws HyracksDataException {
final Map<Long, LocalResource> allResources = loadAndGetAllResources();
final Optional<Long> max = allResources.keySet().stream().max(Long::compare);
return max.isPresent() ? max.get() : 0;
@@ -304,7 +316,7 @@
}
}
- public void setReplicationManager(IReplicationManager replicationManager) {
+ public synchronized void setReplicationManager(IReplicationManager replicationManager) {
this.replicationManager = replicationManager;
isReplicationEnabled = replicationManager.isReplicationEnabled();
@@ -331,7 +343,7 @@
*
* @throws IOException
*/
- public void deleteStorageData() throws IOException {
+ public synchronized void deleteStorageData() throws IOException {
for (Path root : storageRoots) {
final File rootFile = root.toFile();
if (rootFile.exists()) {
@@ -341,13 +353,13 @@
createStorageRoots();
}
- public Set<Integer> getAllPartitions() throws HyracksDataException {
+ public synchronized Set<Integer> getAllPartitions() throws HyracksDataException {
return loadAndGetAllResources().values().stream().map(LocalResource::getResource)
.map(DatasetLocalResource.class::cast).map(DatasetLocalResource::getPartition)
.collect(Collectors.toSet());
}
- public Optional<DatasetResourceReference> getLocalResourceReference(String absoluteFilePath)
+ public synchronized Optional<DatasetResourceReference> getLocalResourceReference(String absoluteFilePath)
throws HyracksDataException {
final String localResourcePath = StoragePathUtil.getIndexFileRelativePath(absoluteFilePath);
final LocalResource lr = get(localResourcePath);
@@ -362,11 +374,12 @@
* @return The set of indexes files
* @throws HyracksDataException
*/
- public Set<File> getPartitionIndexes(int partition) throws HyracksDataException {
+ public synchronized Set<File> getPartitionIndexes(int partition) throws HyracksDataException {
+ Path partitionRoot = getPartitionRoot(partition);
final Map<Long, LocalResource> partitionResourcesMap = getResources(resource -> {
DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
return dsResource.getPartition() == partition;
- });
+ }, Collections.singletonList(partitionRoot));
Set<File> indexes = new HashSet<>();
for (LocalResource localResource : partitionResourcesMap.values()) {
indexes.add(ioManager.resolve(localResource.getPath()).getFile());
@@ -374,14 +387,25 @@
return indexes;
}
- public Map<Long, LocalResource> getPartitionResources(int partition) throws HyracksDataException {
- return getResources(resource -> {
- DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
- return dsResource.getPartition() == partition;
- });
+ public synchronized Map<Long, LocalResource> getPartitionResources(int partition) throws HyracksDataException {
+ return getResources(r -> true, Collections.singleton(partition));
}
- public List<String> getPartitionReplicatedFiles(int partition, IReplicationStrategy strategy)
+ public synchronized Map<String, Long> getPartitionReplicatedResources(int partition, IReplicationStrategy strategy)
+ throws HyracksDataException {
+ final Map<String, Long> partitionReplicatedResources = new HashMap<>();
+ final Map<Long, LocalResource> partitionResources = getPartitionResources(partition);
+ for (LocalResource lr : partitionResources.values()) {
+ DatasetLocalResource datasetLocalResource = (DatasetLocalResource) lr.getResource();
+ if (strategy.isMatch(datasetLocalResource.getDatasetId())) {
+ DatasetResourceReference drr = DatasetResourceReference.of(lr);
+ partitionReplicatedResources.put(drr.getFileRelativePath().toString(), lr.getId());
+ }
+ }
+ return partitionReplicatedResources;
+ }
+
+ public synchronized List<String> getPartitionReplicatedFiles(int partition, IReplicationStrategy strategy)
throws HyracksDataException {
final List<String> partitionReplicatedFiles = new ArrayList<>();
final Set<File> replicatedIndexes = new HashSet<>();
@@ -398,7 +422,7 @@
return partitionReplicatedFiles;
}
- public long getReplicatedIndexesMaxComponentId(int partition, IReplicationStrategy strategy)
+ public synchronized long getReplicatedIndexesMaxComponentId(int partition, IReplicationStrategy strategy)
throws HyracksDataException {
long maxComponentId = LSMComponentId.MIN_VALID_COMPONENT_ID;
final Map<Long, LocalResource> partitionResources = getPartitionResources(partition);
@@ -434,7 +458,7 @@
}
}
- public void cleanup(int partition) throws HyracksDataException {
+ public synchronized void cleanup(int partition) throws HyracksDataException {
final Set<File> partitionIndexes = getPartitionIndexes(partition);
try {
for (File index : partitionIndexes) {
@@ -461,10 +485,9 @@
return resourcesStats;
}
- public void deleteCorruptedResources() throws HyracksDataException {
+ public synchronized void deleteCorruptedResources() throws HyracksDataException {
for (Path root : storageRoots) {
- final Collection<File> metadataMaskFiles =
- FileUtils.listFiles(root.toFile(), METADATA_MASK_FILES_FILTER, ALL_DIR_FILTER);
+ final Collection<File> metadataMaskFiles = IoUtil.getMatchingFiles(root, METADATA_MASK_FILES_FILTER);
for (File metadataMaskFile : metadataMaskFiles) {
final File resourceFile = new File(metadataMaskFile.getParent(), METADATA_FILE_NAME);
if (resourceFile.exists()) {
@@ -561,12 +584,13 @@
return null;
}
- public long getDatasetSize(DatasetCopyIdentifier datasetIdentifier) throws HyracksDataException {
+ public long getDatasetSize(DatasetCopyIdentifier datasetIdentifier, Set<Integer> nodePartitions)
+ throws HyracksDataException {
long totalSize = 0;
final Map<Long, LocalResource> dataverse = getResources(lr -> {
final ResourceReference resourceReference = ResourceReference.ofIndex(lr.getPath());
return datasetIdentifier.isMatch(resourceReference);
- });
+ }, nodePartitions);
final List<DatasetResourceReference> allResources =
dataverse.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
for (DatasetResourceReference res : allResources) {
@@ -603,4 +627,51 @@
private static boolean isComponentFile(File indexDir, String fileName) {
return COMPONENT_FILES_FILTER.accept(indexDir, fileName);
}
+
+ public List<Path> getStorageRoots() {
+ return storageRoots;
+ }
+
+ public synchronized void keepPartitions(Set<Integer> keepPartitions) {
+ List<File> onDiskPartitions = getOnDiskPartitions();
+ for (File onDiskPartition : onDiskPartitions) {
+ int partitionNum = StoragePathUtil.getPartitionNumFromRelativePath(onDiskPartition.getAbsolutePath());
+ if (!keepPartitions.contains(partitionNum)) {
+ LOGGER.warn("deleting partition {} since it is not on partitions to keep {}", partitionNum,
+ keepPartitions);
+ FileUtils.deleteQuietly(onDiskPartition);
+ }
+ }
+ }
+
+ public synchronized List<File> getOnDiskPartitions() {
+ List<File> onDiskPartitions = new ArrayList<>();
+ for (Path root : storageRoots) {
+ File[] partitions = root.toFile().listFiles(
+ (dir, name) -> dir.isDirectory() && name.startsWith(StorageConstants.PARTITION_DIR_PREFIX));
+ if (partitions != null) {
+ onDiskPartitions.addAll(Arrays.asList(partitions));
+ }
+ }
+ return onDiskPartitions;
+ }
+
+ public Path getPartitionRoot(int partition) throws HyracksDataException {
+ Path path =
+ Paths.get(StorageConstants.STORAGE_ROOT_DIR_NAME, StorageConstants.PARTITION_DIR_PREFIX + partition);
+ FileReference resolve = ioManager.resolve(path.toString());
+ return resolve.getFile().toPath();
+ }
+
+ public void deletePartition(int partitionId) {
+ List<File> onDiskPartitions = getOnDiskPartitions();
+ for (File onDiskPartition : onDiskPartitions) {
+ int partitionNum = StoragePathUtil.getPartitionNumFromRelativePath(onDiskPartition.getAbsolutePath());
+ if (partitionNum == partitionId) {
+ LOGGER.warn("deleting partition {}", partitionNum);
+ FileUtils.deleteQuietly(onDiskPartition);
+ return;
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
index 6582670..f09248f 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManager.java
@@ -46,7 +46,7 @@
private static final long NO_SECURED_LSN = -1L;
private final long datasetCheckpointIntervalNanos;
private final Map<TxnId, Long> securedLSNs;
- private boolean suspended = false;
+ private int suspendCount = 0;
public CheckpointManager(ITransactionSubsystem txnSubsystem, CheckpointProperties checkpointProperties) {
super(txnSubsystem, checkpointProperties);
@@ -84,7 +84,7 @@
}
final long minFirstLSN = txnSubsystem.getRecoveryManager().getMinFirstLSN();
boolean checkpointSucceeded = minFirstLSN >= checkpointTargetLSN;
- if (!checkpointSucceeded && !suspended) {
+ if (!checkpointSucceeded && !isSuspended()) {
// Flush datasets with indexes behind target checkpoint LSN
final IDatasetLifecycleManager dlcm = txnSubsystem.getApplicationContext().getDatasetLifecycleManager();
dlcm.asyncFlushMatchingIndexes(newLaggingDatasetPredicate(checkpointTargetLSN));
@@ -109,21 +109,25 @@
@Override
public synchronized void checkpointIdleDatasets() throws HyracksDataException {
- if (suspended) {
+ if (isSuspended()) {
return;
}
final IDatasetLifecycleManager dlcm = txnSubsystem.getApplicationContext().getDatasetLifecycleManager();
dlcm.asyncFlushMatchingIndexes(newIdleDatasetPredicate());
}
+ private synchronized boolean isSuspended() {
+ return suspendCount != 0;
+ }
+
@Override
public synchronized void suspend() {
- suspended = true;
+ suspendCount++;
}
@Override
public synchronized void resume() {
- suspended = false;
+ suspendCount--;
}
private synchronized long getMinSecuredLSN() {
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index a2eb167..09c5f4a 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -79,16 +79,21 @@
<prepare-env.stage>none</prepare-env.stage>
<shim.stage>none</shim.stage>
<pytestlib.stage>none</pytestlib.stage>
+ <skip-npm-touch.stage>none</skip-npm-touch.stage>
<!-- Versions under dependencymanagement or used in many projects via properties -->
<algebricks.version>0.3.8-SNAPSHOT</algebricks.version>
<hyracks.version>0.3.8-SNAPSHOT</hyracks.version>
- <hadoop.version>2.8.5</hadoop.version>
+ <hadoop.version>3.3.1</hadoop.version>
<jacoco.version>0.7.6.201602180812</jacoco.version>
- <log4j.version>2.14.1</log4j.version>
- <awsjavasdk.version>2.10.83</awsjavasdk.version>
- <azurejavasdk.version>12.6.0</azurejavasdk.version>
- <parquet.version>1.8.2</parquet.version>
+ <log4j.version>2.17.1</log4j.version>
+ <awsjavasdk.version>2.17.116</awsjavasdk.version>
+ <parquet.version>1.12.0</parquet.version>
+ <hadoop-awsjavasdk.version>1.12.109</hadoop-awsjavasdk.version>
+ <azureblobjavasdk.version>12.14.2</azureblobjavasdk.version>
+ <azuredatalakejavasdk.version>12.7.2</azuredatalakejavasdk.version>
+ <gcsjavasdk.version>1.114.0</gcsjavasdk.version>
+ <hadoop-azuresdk.version>8.6.6</hadoop-azuresdk.version>
<implementation.title>Apache AsterixDB - ${project.name}</implementation.title>
<implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -285,7 +290,7 @@
<condition property="skipPomEnforcement">
<not><equals arg1="${project.packaging}" arg2="pom"/></not>
</condition>
- <echo message="will enforce non-existence of test source dir due to ${project.packaging} packaging" unless:set="skipPomEnforcement"/>
+ <echo level="info" message="will enforce non-existence of test source dir due to ${project.packaging} packaging" unless:set="skipPomEnforcement"/>
</target>
</configuration>
</execution>
@@ -566,7 +571,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-antrun-plugin</artifactId>
- <version>1.8</version>
+ <version>3.0.0</version>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -612,6 +617,13 @@
<groupId>org.jvnet.jaxb2.maven2</groupId>
<artifactId>maven-jaxb2-plugin</artifactId>
<version>0.14.0</version>
+ <dependencies>
+ <dependency>
+ <groupId>org.glassfish.jaxb</groupId>
+ <artifactId>jaxb-runtime</artifactId>
+ <version>2.3.5</version>
+ </dependency>
+ </dependencies>
</plugin>
<plugin>
<groupId>pl.project13.maven</groupId>
@@ -711,6 +723,18 @@
</properties>
</profile>
<profile>
+ <id>skip-npm</id>
+ <activation>
+ <property>
+ <name>skip.npm</name>
+ <value>true</value>
+ </property>
+ </activation>
+ <properties>
+ <skip-npm-touch.stage>process-sources</skip-npm-touch.stage>
+ </properties>
+ </profile>
+ <profile>
<id>invalid-tests</id>
<properties>
<invalid.tests />
@@ -901,7 +925,6 @@
<module>asterix-runtime</module>
<module>asterix-om</module>
<module>asterix-external-data</module>
- <module>asterix-hivecompat</module>
<module>asterix-examples</module>
<module>asterix-metadata</module>
<module>asterix-test-framework</module>
@@ -932,17 +955,17 @@
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-simple</artifactId>
- <version>1.7.28</version>
+ <version>1.7.33</version>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-plugin-api</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-model</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
@@ -953,6 +976,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -964,6 +995,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -972,11 +1011,15 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>nimbus-jose-jwt</artifactId>
+ </exclusion>
+ <exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
<exclusion>
- <groupId>stax</groupId>
+ <groupId>org.codehaus.woodstox</groupId>
<artifactId>stax-api</artifactId>
</exclusion>
<exclusion>
@@ -995,6 +1038,34 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.fasterxml</groupId>
+ <artifactId>woodstox-core</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>dnsjava</groupId>
+ <artifactId>dnsjava</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.wildfly.openssl</groupId>
+ <artifactId>wildfly-openssl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.jline</groupId>
+ <artifactId>jline</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>javax.ws.rs</groupId>
+ <artifactId>jsr311-api</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1006,6 +1077,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1015,9 +1094,21 @@
<classifier>tests</classifier>
<exclusions>
<exclusion>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>nimbus-jose-jwt</artifactId>
+ </exclusion>
+ <exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1033,6 +1124,18 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.jline</groupId>
+ <artifactId>jline</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1044,6 +1147,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1055,6 +1166,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -1071,27 +1190,20 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-serde</artifactId>
- <version>0.13.0</version>
- <exclusions>
<exclusion>
- <groupId>commons-logging</groupId>
- <artifactId>commons-logging</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
</exclusion>
<exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-compress</artifactId>
- <version>1.20</version>
+ <version>1.21</version>
</dependency>
<dependency>
<groupId>commons-logging</groupId>
@@ -1104,36 +1216,6 @@
<version>1.2</version>
</dependency>
<dependency>
- <groupId>org.apache.hive.shims</groupId>
- <artifactId>hive-shims-common</artifactId>
- <version>0.13.0</version>
- <exclusions>
- <exclusion>
- <groupId>commons-logging</groupId>
- <artifactId>commons-logging</artifactId>
- </exclusion>
- <exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-common</artifactId>
- <version>0.13.0</version>
- <exclusions>
- <exclusion>
- <groupId>commons-logging</groupId>
- <artifactId>commons-logging</artifactId>
- </exclusion>
- <exclusion>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks-common</artifactId>
<version>${algebricks.version}</version>
@@ -1376,7 +1458,7 @@
<dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
- <version>8.5.4</version>
+ <version>8.5.6</version>
</dependency>
<dependency>
<groupId>com.sun.xml.bind</groupId>
@@ -1401,12 +1483,12 @@
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-core</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-compat</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
</dependency>
<dependency>
<groupId>com.esri.geometry</groupId>
@@ -1509,8 +1591,8 @@
<!-- Azure Blob Storage start -->
<dependency>
<groupId>com.azure</groupId>
- <artifactId>azure-storage-blob</artifactId>
- <version>${azurejavasdk.version}</version>
+ <artifactId>azure-identity</artifactId>
+ <version>1.4.1</version>
<exclusions>
<exclusion>
<groupId>io.netty</groupId>
@@ -1552,12 +1634,107 @@
<groupId>io.netty</groupId>
<artifactId>netty-tcnative-boringssl-static</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-dns</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-kqueue</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns-native-macos</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>com.azure</groupId>
+ <artifactId>azure-storage-blob</artifactId>
+ <version>12.14.3</version>
+ <exclusions>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-handler</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-handler-proxy</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http2</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-buffer</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-common</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-epoll</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-unix-common</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-tcnative-boringssl-static</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-dns</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-kqueue</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns-native-macos</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.azure</groupId>
<artifactId>azure-storage-common</artifactId>
- <version>${azurejavasdk.version}</version>
+ <version>12.14.2</version>
<exclusions>
<exclusion>
<groupId>io.netty</groupId>
@@ -1599,9 +1776,59 @@
<groupId>io.netty</groupId>
<artifactId>netty-tcnative-boringssl-static</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-dns</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-kqueue</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns-native-macos</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<!-- Azure Blob Storage end -->
+ <!-- Google Cloud Storage start -->
+ <dependency>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-storage</artifactId>
+ <version>${gcsjavasdk.version}</version>
+ <exclusions>
+ <!-- substitute org.apache.tomcat:tomcat-annotations-api, as it has a nicer license -->
+ <exclusion>
+ <groupId>javax.annotation</groupId>
+ <artifactId>javax.annotation-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tomcat</groupId>
+ <artifactId>tomcat-annotations-api</artifactId>
+ <version>10.0.12</version>
+ </dependency>
+ <!-- Google Cloud Storage end -->
+ <!-- Azure Data Lake start -->
+ <dependency>
+ <groupId>com.azure</groupId>
+ <artifactId>azure-storage-file-datalake</artifactId>
+ <version>${azuredatalakejavasdk.version}</version>
+ </dependency>
+ <!-- Azure Data Lake end -->
<dependency>
<groupId>org.mindrot</groupId>
<artifactId>jbcrypt</artifactId>
@@ -1610,17 +1837,17 @@
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-csv</artifactId>
- <version>1.8</version>
+ <version>1.9.0</version>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>postgresql</artifactId>
- <version>1.13.0</version>
+ <version>1.16.2</version>
</dependency>
<dependency>
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
- <version>42.2.10</version>
+ <version>42.2.24</version>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
@@ -1636,17 +1863,84 @@
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-column</artifactId>
<version>${parquet.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>${parquet.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
<groupId>org.kitesdk</groupId>
<artifactId>kite-data-core</artifactId>
<version>1.1.0</version>
</dependency>
+ <!-- Hadoop AWS start -->
+ <dependency>
+ <!-- Pick a newer AWS SDK -->
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-s3</artifactId>
+ <version>${hadoop-awsjavasdk.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-dynamodb</artifactId>
+ <version>${hadoop-awsjavasdk.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-aws</artifactId>
+ <version>${hadoop.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-bundle</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <!-- Hadoop AWS end -->
+ <!-- Hadoop Azure start -->
+ <dependency>
+ <!-- Pick a newer Azure connector -->
+ <groupId>com.microsoft.azure</groupId>
+ <artifactId>azure-storage</artifactId>
+ <version>${hadoop-azuresdk.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-azure</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ <!-- Hadoop Azure end -->
</dependencies>
</dependencyManagement>
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index c352fe5..61ac8b5 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -12,6 +12,9 @@
</licenses>
<properties>
<verifiedVersions>1.1.1-dev,1.2.0</verifiedVersions>
+ <license.ignoreMissingEmbeddedNotice>1.2.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.2.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreLicenseOverride>1.2.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -157,12 +160,144 @@
<supplement>
<project>
<groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
+ <artifactId>netty-transport</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
- <license.ignoreMissingEmbeddedLicense>4.1.63.Final</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>4.1.63.Final</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>4.1.63.Final</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-classes-epoll</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.72.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.72.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.712.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-unix-common</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.72.Final,4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.72.Final,4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.72.Final,4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-dns</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http2</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-handler</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-buffer</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-common</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.73.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.73.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.73.Final</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -174,8 +309,9 @@
<!-- guava is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
<!-- see https://github.com/google/guava/blob/v18.0/COPYING -->
<!-- see https://github.com/google/guava/tree/v28.[012]/COPYING -->
- <license.ignoreMissingEmbeddedNotice>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre</license.ignoreMissingEmbeddedNotice>
- <license.ignoreMissingEmbeddedLicense>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre</license.ignoreMissingEmbeddedLicense>
+ <!-- see https://github.com/google/guava/blob/v31.0.1/COPYING -->
+ <license.ignoreMissingEmbeddedNotice>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre,31.0.1-jre</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>18.0,28.0-jre,28.1-jre,28.2-jre,30.1-jre,31.0.1-jre</license.ignoreMissingEmbeddedLicense>
</properties>
</project>
</supplement>
@@ -196,9 +332,10 @@
<groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId>
<!-- see https://github.com/google/error-prone/tree/v2.3.[24]/ -->
+ <!-- see https://github.com/google/error-prone/blob/v2.7.1/COPYING -->
<properties>
- <license.ignoreMissingEmbeddedLicense>2.3.2,2.3.4</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.3.2,2.3.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.3.2,2.3.4,2.7.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.3.2,2.3.4,2.7.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -229,7 +366,7 @@
<artifactId>jackson-annotations</artifactId>
<properties>
<!-- jackson-annotations does not provide an embedded NOTICE file -->
- <license.ignoreMissingEmbeddedNotice>2.12.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>2.12.3,2.13.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -250,9 +387,11 @@
<artifactId>fastutil</artifactId>
<properties>
<!-- fastutil is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <!-- see https://github.com/vigna/fastutil/blob/8.3.0/LICENSE-2.0 -->
<!-- see https://github.com/vigna/fastutil/blob/8.5.4/LICENSE-2.0 -->
- <license.ignoreMissingEmbeddedLicense>8.5.4</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>8.5.4</license.ignoreMissingEmbeddedNotice>
+ <!-- see https://github.com/vigna/fastutil/blob/8.5.6/LICENSE-2.0 -->
+ <license.ignoreMissingEmbeddedLicense>8.3.0,8.5.4,8.5.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>8.3.0,8.5.4,8.5.6</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -263,8 +402,9 @@
<properties>
<!-- fastutil is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
<!-- see https://github.com/vigna/fastutil/blob/8.5.4/LICENSE-2.0 -->
- <license.ignoreMissingEmbeddedLicense>8.5.4</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>8.5.4</license.ignoreMissingEmbeddedNotice>
+ <!-- see https://github.com/vigna/fastutil/blob/8.5.6/LICENSE-2.0 -->
+ <license.ignoreMissingEmbeddedLicense>8.5.4,8.5.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>8.5.4,8.5.6</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -287,9 +427,9 @@
<properties>
<!-- snappy-java is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
<!-- license override not needed, ALv2 is specified in its pom.xml -->
- <!-- see https://github.com/xerial/snappy-java/blob/1.1.7.1/LICENSE -->
- <license.ignoreMissingEmbeddedLicense>1.1.7.1</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.1.7.1</license.ignoreMissingEmbeddedNotice>
+ <!-- see https://raw.githubusercontent.com/xerial/snappy-java/1.1.8.4/LICENSE -->
+ <license.ignoreMissingEmbeddedLicense>1.1.8.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.1.8.4</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -397,76 +537,21 @@
</properties>
</project>
</supplement>
+
<supplement>
<project>
<groupId>software.amazon.awssdk</groupId>
- <artifactId>annotations</artifactId>
+ <artifactId>third-party-jackson-core</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.10.83</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.10.83</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>2.10.83</license.ignoreNoticeOverride>
- </properties>
- </project>
- </supplement>
- <supplement>
- <project>
- <groupId>software.amazon.awssdk</groupId>
- <artifactId>apache-client</artifactId>
- <properties>
- <license.ignoreMissingEmbeddedLicense>2.10.83</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.10.83</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>2.10.83</license.ignoreNoticeOverride>
- </properties>
- </project>
- </supplement>
- <supplement>
- <project>
- <groupId>software.amazon.awssdk</groupId>
- <artifactId>utils</artifactId>
- <properties>
- <license.ignoreMissingEmbeddedLicense>2.10.83</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.10.83</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>2.10.83</license.ignoreNoticeOverride>
- </properties>
- </project>
- </supplement>
- <supplement>
- <project>
- <groupId>software.amazon.awssdk</groupId>
- <artifactId>http-client-spi</artifactId>
- <properties>
- <license.ignoreMissingEmbeddedLicense>2.10.83</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.10.83</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>2.10.83</license.ignoreNoticeOverride>
- </properties>
- </project>
- </supplement>
- <supplement>
- <project>
- <groupId>software.amazon.awssdk</groupId>
- <artifactId>aws-query-protocol</artifactId>
- <properties>
- <license.ignoreMissingEmbeddedLicense>2.10.83</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.10.83</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>2.10.83</license.ignoreNoticeOverride>
- </properties>
- </project>
- </supplement>
- <supplement>
- <project>
- <groupId>software.amazon.awssdk</groupId>
- <artifactId>profiles</artifactId>
- <properties>
- <license.ignoreMissingEmbeddedLicense>2.10.83</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.10.83</license.ignoreMissingEmbeddedNotice>
- <license.ignoreNoticeOverride>2.10.83</license.ignoreNoticeOverride>
+ <license.alternateNoticeFile>2.17.116:META-INF/NOTICE.txt</license.alternateNoticeFile>
+ <license.alternateLicenseFile>2.17.116:META-INF/LICENSE.txt</license.alternateLicenseFile>
</properties>
</project>
</supplement>
<!-- software.amazon.eventstream is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
<!-- license override not needed, ALv2 is specified in its pom.xml -->
- <!-- see https://github.com/awslabs/aws-eventstream-java -->
+ <!-- see https://github.com/aws/aws-sdk-java -->
<supplement>
<project>
<groupId>software.amazon.eventstream</groupId>
@@ -480,6 +565,81 @@
</supplement>
<!-- AWS SDK end -->
+ <!-- AWS Hadoop SDK start -->
+ <!-- software.amazon.awssdk is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <!-- license override not needed, ALv2 is specified in its pom.xml -->
+ <!-- see https://github.com/aws/aws-sdk-java-v2/blob/master/LICENSE.txt -->
+ <supplement>
+ <project>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>jmespath-java</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-s3</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-kms</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.amazonaws</groupId>
+ <artifactId>aws-java-sdk-dynamodb</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.12.109</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.109</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- software.amazon.ion is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <!-- license override not needed, ALv2 is specified in its pom.xml -->
+ <!-- see https://github.com/amzn/ion-java -->
+ <supplement>
+ <project>
+ <groupId>software.amazon.ion</groupId>
+ <artifactId>ion-java</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>1.0.2</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <!-- AWS Hadoop SDK end -->
+
<!-- com.typesafe.netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
<!-- license override not needed, ALv2 is specified in its pom.xml -->
<!-- see https://github.com/playframework/netty-reactive-streams -->
@@ -488,8 +648,8 @@
<groupId>com.typesafe.netty</groupId>
<artifactId>netty-reactive-streams</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.0.4</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.0.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.0.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.0.5</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -499,8 +659,8 @@
<groupId>com.typesafe.netty</groupId>
<artifactId>netty-reactive-streams-http</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>2.0.4</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>2.0.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>2.0.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.0.5</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -512,10 +672,9 @@
<groupId>org.reactivestreams</groupId>
<artifactId>reactive-streams</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.0.2</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.0.2</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.0.2</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>1.0.2</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>1.0.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>1.0.3</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -560,10 +719,24 @@
<groupId>com.azure</groupId>
<artifactId>azure-storage-blob</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>12.6.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>12.6.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>12.6.0</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>12.6.0</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>12.14.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>12.14.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>12.14.3</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>12.14.3</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.azure does not contain any embedded LICENSE or NOTICE file -->
+ <!-- see https://github.com/Azure/azure-sdk-for-java -->
+ <supplement>
+ <project>
+ <groupId>com.azure</groupId>
+ <artifactId>azure-storage-file-datalake</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>12.7.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>12.7.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>12.7.2</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
@@ -573,10 +746,23 @@
<groupId>com.azure</groupId>
<artifactId>azure-storage-common</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>12.6.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>12.6.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>12.6.0</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>12.6.0</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>12.14.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>12.14.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>12.14.2</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>12.14.2</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.azure</groupId>
+ <artifactId>azure-storage-internal-avro</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>12.1.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>12.1.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>12.1.3</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>12.1.3</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -586,10 +772,10 @@
<groupId>com.azure</groupId>
<artifactId>azure-core</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.4.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.4.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.4.0</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>1.4.0</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>1.24.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.24.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.24.1</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>1.24.1</license.ignoreNoticeOverride>
</properties>
</project>
</supplement>
@@ -599,15 +785,733 @@
<groupId>com.azure</groupId>
<artifactId>azure-core-http-netty</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>1.5.0</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>1.5.0</license.ignoreMissingEmbeddedNotice>
- <license.ignoreLicenseOverride>1.5.0</license.ignoreLicenseOverride>
- <license.ignoreNoticeOverride>1.5.0</license.ignoreNoticeOverride>
+ <license.ignoreMissingEmbeddedLicense>1.11.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.11.6</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.11.6</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>1.11.6</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.azure</groupId>
+ <artifactId>azure-identity</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.4.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.4.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.4.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.microsoft.azure:msal4j has MIT license embedded in pom.xml and no notice -->
+ <supplement>
+ <project>
+ <groupId>com.microsoft.azure</groupId>
+ <artifactId>msal4j</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.11.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.11.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.11.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.microsoft.azure:msal4j-persistence-extension has MIT license embedded in pom.xml and no notice -->
+ <supplement>
+ <project>
+ <groupId>com.microsoft.azure</groupId>
+ <artifactId>msal4j-persistence-extension</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.1.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.1.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.1.0</license.ignoreLicenseOverride>
</properties>
</project>
</supplement>
<!-- Azure SDK for Java end -->
+ <!-- Azure SDK for Hadoop start -->
+ <supplement>
+ <project>
+ <groupId>com.microsoft.azure</groupId>
+ <artifactId>azure-storage</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>8.6.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>8.6.6</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>8.6.6</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.microsoft.azure</groupId>
+ <artifactId>azure-keyvault-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.2.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.2.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.2.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+ <!-- Azure SDK for Hadoop end -->
+
+ <!-- jackson-dataformat-cbor does not contain embedded LICENSE and NOTICE -->
+ <!-- See https://github.com/FasterXML/jackson-modules-java8 -->
+ <supplement>
+ <project>
+ <groupId>com.fasterxml.jackson.dataformat</groupId>
+ <artifactId>jackson-dataformat-cbor</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.12.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.12.3</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.projectreactor:reactor-core uses ALv2 license and has no notice -->
+ <supplement>
+ <project>
+ <groupId>io.projectreactor</groupId>
+ <artifactId>reactor-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>3.4.13</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>3.4.13</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.projectreactor.netty:reactor-netty uses ALv2 license and has no notice -->
+ <supplement>
+ <project>
+ <groupId>io.projectreactor.netty</groupId>
+ <artifactId>reactor-netty</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.14</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.14</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.projectreactor.netty:reactor-netty-core uses ALv2 license and has no notice -->
+ <supplement>
+ <project>
+ <groupId>io.projectreactor.netty</groupId>
+ <artifactId>reactor-netty-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.14</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.14</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.projectreactor.netty:reactor-netty-http uses ALv2 license and has no notice -->
+ <supplement>
+ <project>
+ <groupId>io.projectreactor.netty</groupId>
+ <artifactId>reactor-netty-http</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.14</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.14</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.projectreactor.netty:reactor-netty-http-brave uses ALv2 license and has no notice -->
+ <supplement>
+ <project>
+ <groupId>io.projectreactor.netty</groupId>
+ <artifactId>reactor-netty-http-brave</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.7</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.7</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.nimbusds:oauth2-oidc-sdk has ALv2 embedded in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>oauth2-oidc-sdk</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>9.7</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>9.7</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>9.7</license.ignoreLicenseOverride>
+ <license.ignoreNoticeOverride>9.7</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.nimbusds:nimbus-jose-jwt has ALv2 embedded in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>nimbus-jose-jwt</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>9.9.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>9.9.3</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.nimbusds:lang-tag has ALv2 embedded in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>lang-tag</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.5</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.nimbusds:content-type has ALv2 embedded in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>content-type</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.linguafranca.pwdb:KeePassJava2 uses ALv2 and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.linguafranca.pwdb</groupId>
+ <artifactId>KeePassJava2</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.linguafranca.pwdb:KeePassJava2-dom uses ALv2 and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.linguafranca.pwdb</groupId>
+ <artifactId>KeePassJava2-dom</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.linguafranca.pwdb:KeePassJava2-jaxb uses ALv2 and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.linguafranca.pwdb</groupId>
+ <artifactId>KeePassJava2-jaxb</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.linguafranca.pwdb:KeePassJava2-kdb uses ALv2 and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.linguafranca.pwdb</groupId>
+ <artifactId>KeePassJava2-kdb</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.linguafranca.pwdb:KeePassJava2-kdbx uses ALv2 and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.linguafranca.pwdb</groupId>
+ <artifactId>KeePassJava2-kdbx</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.linguafranca.pwdb:KeePassJava2-simple uses ALv2 and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.linguafranca.pwdb</groupId>
+ <artifactId>KeePassJava2-simple</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.linguafranca.pwdb:database uses ALv2 and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.linguafranca.pwdb</groupId>
+ <artifactId>database</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.4</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.4</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.1.4</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- net.minidev:json-smart has ALv2 embedded in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>net.minidev</groupId>
+ <artifactId>json-smart</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.4.7</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.4.7</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- net.minidev:accessors-smart has ALv2 embedded in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>net.minidev</groupId>
+ <artifactId>accessors-smart</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.4.7</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.4.7</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>org.ow2.asm</groupId>
+ <artifactId>asm</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>9.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>9.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>9.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- GCS SDK for Java start -->
+ <!-- com.google.http-client is ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client-jackson2</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.39.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.39.2</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api</groupId>
+ <artifactId>api-common</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.10.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.10.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.10.3</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api:gax-httpjson has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api</groupId>
+ <artifactId>gax-httpjson</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.81.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.81.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.81.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api:gax has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api</groupId>
+ <artifactId>gax</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.64.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.64.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.64.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api-client uses non-fixed ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api-client</groupId>
+ <artifactId>google-api-client</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.31.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.31.5</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.31.5</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses non-fixed ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-api</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.28.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.28.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.28.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.opencensus uses non-fixed ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>io.opencensus</groupId>
+ <artifactId>opencensus-contrib-http-util</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.28.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.28.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.28.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud:google-cloud-core is non-fixed ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-core</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.94.8</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.94.8</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.94.8</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud:google-cloud-core-http is non-fixed ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-core-http</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.94.8</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.94.8</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.94.8</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.auth has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.auth</groupId>
+ <artifactId>google-auth-library-credentials</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.25.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.25.5</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.25.5</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.grpc:grpc-context uses non-fixed ALv2 and has no NOTICE -->
+ <supplement>
+ <project>
+ <groupId>io.grpc</groupId>
+ <artifactId>grpc-context</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.37.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.37.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.37.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.protobuf has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.protobuf</groupId>
+ <artifactId>protobuf-java-util</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>3.16.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>3.16.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>3.16.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.auth has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.auth</groupId>
+ <artifactId>google-auth-library-oauth2-http</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>0.25.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>0.25.5</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>0.25.5</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.protobuf has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.protobuf</groupId>
+ <artifactId>protobuf-java</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>3.16.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>3.16.0</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>3.16.0</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.threeten has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>org.threeten</groupId>
+ <artifactId>threetenbp</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>1.5.1</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.http-client is ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.39.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.39.2</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.http-client is ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client-appengine</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.39.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.39.2</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.http-client is ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client-apache-v2</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.39.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.39.2</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.http-client is ALv2 with no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.http-client</groupId>
+ <artifactId>google-http-client-gson</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.39.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.39.2</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.code.findbugs:jsr305 has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.code.findbugs</groupId>
+ <artifactId>jsr305</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>3.0.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>3.0.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>3.0.2</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.code.gson:gson uses non-fixed ALv2 and has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.code.gson</groupId>
+ <artifactId>gson</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.8.6</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.8.6</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.8.6</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.guava:listenablefuture uses non-fixed ALv2 and has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.guava</groupId>
+ <artifactId>listenablefuture</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>9999.0-empty-to-avoid-conflict-with-guava</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>9999.0-empty-to-avoid-conflict-with-guava</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>9999.0-empty-to-avoid-conflict-with-guava</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.checkerframework:checker-compat-qual has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>org.checkerframework</groupId>
+ <artifactId>checker-compat-qual</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.5.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.5.5</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>2.5.5</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api:api-common has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api</groupId>
+ <artifactId>api-common</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.10.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.10.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.10.3</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.auto.value:auto-value-annotations has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.auto.value</groupId>
+ <artifactId>auto-value-annotations</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.8.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.8.1</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- javax.annotation:javax.annotation-api has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>javax.annotation</groupId>
+ <artifactId>javax.annotation-api</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.3.2</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.3.2</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.apis:google-api-services-storage has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.apis</groupId>
+ <artifactId>google-api-services-storage</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>v1-rev20210127-1.31.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>v1-rev20210127-1.31.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.oauth-client:google-oauth-client:1.31.5 uses non-fixed ALv2 and has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.oauth-client</groupId>
+ <artifactId>google-oauth-client</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.31.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.31.5</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.31.5</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api.grpc:proto-google-iam-v1 has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api.grpc</groupId>
+ <artifactId>proto-google-iam-v1</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.13</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.13</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.api.grpc:proto-google-common-protos has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.api.grpc</groupId>
+ <artifactId>proto-google-common-protos</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.1.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.1.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.cloud:google-cloud-storage has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.cloud</groupId>
+ <artifactId>google-cloud-storage</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.114.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.114.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.google.j2objc:j2objc-annotations has no NOTICE file -->
+ <supplement>
+ <project>
+ <groupId>com.google.j2objc</groupId>
+ <artifactId>j2objc-annotations</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.3</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+ <!-- GCS SDK for Java end -->
+
<!-- jackson-datatype-jsr contains embedded license but has no NOTICE -->
<!-- See https://github.com/FasterXML/jackson-modules-java8 -->
<supplement>
@@ -615,7 +1519,7 @@
<groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-jsr310</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>2.10.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>2.13.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -627,7 +1531,7 @@
<groupId>com.fasterxml.woodstox</groupId>
<artifactId>woodstox-core</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>6.0.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>6.2.7</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
@@ -639,29 +1543,188 @@
<groupId>org.codehaus.woodstox</groupId>
<artifactId>stax2-api</artifactId>
<properties>
- <license.ignoreMissingEmbeddedNotice>4.2</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedNotice>4.2.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
<supplement>
<project>
- <groupId>io.projectreactor</groupId>
- <artifactId>reactor-core</artifactId>
+ <groupId>stax</groupId>
+ <artifactId>stax-api</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>3.3.3.RELEASE</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>3.3.3.RELEASE</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.0.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.0.1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- net.java.dev.jna:jna has embedded license but no notice -->
+ <supplement>
+ <project>
+ <groupId>net.java.dev.jna</groupId>
+ <artifactId>jna</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>5.5.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- net.java.dev.jna:jna-platform has embedded license but no notice -->
+ <supplement>
+ <project>
+ <groupId>net.java.dev.jna</groupId>
+ <artifactId>jna-platform</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>5.6.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.zipkin2:zipkin has embedded license but no notice -->
+ <supplement>
+ <project>
+ <groupId>io.zipkin.zipkin2</groupId>
+ <artifactId>zipkin</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>2.23.2</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.zipkin.reporter2:zipkin-reporter has embedded license but no notice -->
+ <supplement>
+ <project>
+ <groupId>io.zipkin.reporter2</groupId>
+ <artifactId>zipkin-reporter</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>2.16.3</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- io.zipkin.reporter2:zipkin-reporter-brave has embedded license but no notice -->
+ <supplement>
+ <project>
+ <groupId>io.zipkin.reporter2</groupId>
+ <artifactId>zipkin-reporter-brave</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>2.16.3</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.fasterxml:aalto-xml has embedded license but no notice -->
+ <supplement>
+ <project>
+ <groupId>com.fasterxml</groupId>
+ <artifactId>aalto-xml</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>1.0.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- com.github.stephenc.jcip:jcip-annotations is ALv2 embedded in pom.xml and has no notice -->
+ <supplement>
+ <project>
+ <groupId>com.github.stephenc.jcip</groupId>
+ <artifactId>jcip-annotations</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0-1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0-1</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.jetbrains:annotations is ALv2 embedded in pom.xml and has no notice -->
+ <supplement>
+ <project>
+ <groupId>org.jetbrains</groupId>
+ <artifactId>annotations</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>15.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>15.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.simpleframework:simple-xml is ALv2 embedded in pom.xml -->
+ <supplement>
+ <project>
+ <groupId>org.simpleframework</groupId>
+ <artifactId>simple-xml</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.7.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.7.1</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
<supplement>
<project>
- <groupId>io.projectreactor.netty</groupId>
- <artifactId>reactor-netty</artifactId>
+ <groupId>com.github.luben</groupId>
+ <artifactId>zstd-jni</artifactId>
<properties>
- <license.ignoreMissingEmbeddedLicense>0.9.5.RELEASE</license.ignoreMissingEmbeddedLicense>
- <license.ignoreMissingEmbeddedNotice>0.9.5.RELEASE</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreMissingEmbeddedLicense>1.4.9-1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.4.9-1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreLicenseOverride>1.4.9-1</license.ignoreLicenseOverride>
+ </properties>
+ </project>
+ </supplement>
+ <!-- Apache Parquet -->
+ <supplement>
+ <project>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-format-structures</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.12.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-common</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.12.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- Contains embedded LICENSE but missing NOTICE -->
+ <supplement>
+ <project>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-column</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- Contains embedded LICENSE but missing NOTICE -->
+ <supplement>
+ <project>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-encoding</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- Contains embedded LICENSE but missing NOTICE -->
+ <supplement>
+ <project>
+ <groupId>org.apache.parquet</groupId>
+ <artifactId>parquet-hadoop</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedNotice>1.12.0</license.ignoreMissingEmbeddedNotice>
</properties>
</project>
</supplement>
diff --git a/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_nimbus-content-type_raw_c6d2701ed5cd57e0f88728c6f3e6303db97c5bcf_COPYRIGHT.txt b/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_nimbus-content-type_raw_c6d2701ed5cd57e0f88728c6f3e6303db97c5bcf_COPYRIGHT.txt
new file mode 100644
index 0000000..e0f0264
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_nimbus-content-type_raw_c6d2701ed5cd57e0f88728c6f3e6303db97c5bcf_COPYRIGHT.txt
@@ -0,0 +1,14 @@
+Nimbus Content Type
+
+Copyright 2020, Connect2id Ltd.
+
+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 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.
diff --git a/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_nimbus-language-tags_raw_ead5120f62b5849309069808509b4cc6e57a0841_COPYRIGHT.txt b/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_nimbus-language-tags_raw_ead5120f62b5849309069808509b4cc6e57a0841_COPYRIGHT.txt
new file mode 100644
index 0000000..37a85f6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_nimbus-language-tags_raw_ead5120f62b5849309069808509b4cc6e57a0841_COPYRIGHT.txt
@@ -0,0 +1,14 @@
+Nimbus Language Tags
+
+Copyright 2012-2016, Connect2id Ltd.
+
+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 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.
diff --git a/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_oauth-2.0-sdk-with-openid-connect-extensions_raw_5d13925b57ace092ea5e1131c338f464d85545f4_COPYRIGHT.txt b/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_oauth-2.0-sdk-with-openid-connect-extensions_raw_5d13925b57ace092ea5e1131c338f464d85545f4_COPYRIGHT.txt
new file mode 100644
index 0000000..42e4fd7
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/bitbucket.org_connect2id_oauth-2.0-sdk-with-openid-connect-extensions_raw_5d13925b57ace092ea5e1131c338f464d85545f4_COPYRIGHT.txt
@@ -0,0 +1,14 @@
+Nimbus OAuth 2.0 SDK with OpenID Connect extensions
+
+Copyright 2012-2021, Connect2id Ltd and contributors.
+
+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 at
+
+ https://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.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.2_COPYING.txt b/asterixdb/src/main/licenses/content/creativecommons.org_publicdomain_zero_1.0_legalcode.txt
similarity index 100%
copy from asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.2_COPYING.txt
copy to asterixdb/src/main/licenses/content/creativecommons.org_publicdomain_zero_1.0_legalcode.txt
diff --git a/asterixdb/src/main/licenses/content/golang.org_LICENSE.txt b/asterixdb/src/main/licenses/content/golang.org_LICENSE.txt
new file mode 100644
index 0000000..ea5ea89
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/golang.org_LICENSE.txt
@@ -0,0 +1,27 @@
+Copyright (c) 2009 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/originals/raw.githubusercontent.com_netty_netty-tcnative_netty-tcnative-parent-2.0.46.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/originals/raw.githubusercontent.com_netty_netty-tcnative_netty-tcnative-parent-2.0.46.Final_NOTICE.txt
new file mode 100644
index 0000000..893572a
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/originals/raw.githubusercontent.com_netty_netty-tcnative_netty-tcnative-parent-2.0.46.Final_NOTICE.txt
@@ -0,0 +1,51 @@
+ The Netty Project
+ =================
+
+Please visit the Netty web site for more information:
+
+ * http://netty.io/
+
+Copyright 2016 The Netty Project
+
+The Netty Project 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.
+
+-------------------------------------------------------------------------------
+This product contains a forked and modified version of Tomcat Native
+
+ * LICENSE:
+ * license/LICENSE.tomcat-native.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * http://tomcat.apache.org/native-doc/
+ * https://svn.apache.org/repos/asf/tomcat/native/
+
+This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build.
+
+ * LICENSE:
+ * license/LICENSE.mvn-wrapper.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/takari/maven-wrapper
+
+This product contains small piece of code to support AIX, taken from netbsd.
+
+ * LICENSE:
+ * license/LICENSE.aix-netbsd.txt (OpenSSL License)
+ * HOMEPAGE:
+ * https://ftp.netbsd.org/pub/NetBSD/NetBSD-current/src/crypto/external/bsd/openssl/dist
+
+
+This product contains code from boringssl.
+
+ * LICENSE (Combination ISC and OpenSSL license)
+ * license/LICENSE.boringssl.txt (Combination ISC and OpenSSL license)
+ * HOMEPAGE:
+ * https://boringssl.googlesource.com/boringssl/
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_master_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_89a32290750a18d1b99c27c16b1b11d42f16c622_LICENSE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_master_LICENSE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_89a32290750a18d1b99c27c16b1b11d42f16c622_LICENSE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_89a32290750a18d1b99c27c16b1b11d42f16c622_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_89a32290750a18d1b99c27c16b1b11d42f16c622_NOTICE.txt
new file mode 100644
index 0000000..9cee825
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_89a32290750a18d1b99c27c16b1b11d42f16c622_NOTICE.txt
@@ -0,0 +1,514 @@
+NOTICES AND INFORMATION
+Do Not Translate or Localize
+
+This software incorporates material from third parties. Microsoft makes certain
+open source code available at https://3rdpartysource.microsoft.com, or you may
+send a check or money order for US $5.00, including the product name, the open
+source component name, and version number, to:
+
+Source Code Compliance Team
+Microsoft Corporation
+One Microsoft Way
+Redmond, WA 98052
+USA
+
+Notwithstanding any other terms, you may reverse engineer this software to the
+extent required to debug changes to any libraries licensed under the GNU Lesser
+General Public License.
+
+------------------------------------------------------------------------------
+
+Azure SDK for Java uses third-party libraries or other resources that may be
+distributed under licenses different than the Azure SDK for Java software.
+
+In the event that we accidentally failed to list a required notice, please
+bring it to our attention. Post an issue or email us:
+
+ azjavasdkhelp@microsoft.com
+
+The attached notices are provided for information only.
+
+License notice for Hamcrest
+------------------------------------------------------------------------------
+
+The 3-Clause BSD License
+
+Copyright (c) 2000-2015 www.hamcrest.org
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of
+conditions and the following disclaimer. Redistributions in binary form must reproduce
+the above copyright notice, this list of conditions and the following disclaimer in
+the documentation and/or other materials provided with the distribution.
+
+Neither the name of Hamcrest nor the names of its contributors may be used to endorse
+or promote products derived from this software without specific prior written
+permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
+SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
+TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
+BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
+WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+DAMAGE.
+
+License notice for Slf4j API
+------------------------------------------------------------------------------
+
+ Copyright (c) 2004-2017 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free of charge, to any person obtaining
+ a copy of this software and associated documentation files (the
+ "Software"), to deal in the Software without restriction, including
+ without limitation the rights to use, copy, modify, merge, publish,
+ distribute, sublicense, and/or sell copies of the Software, and to
+ permit persons to whom the Software is furnished to do so, subject to
+ the following conditions:
+
+ The above copyright notice and this permission notice shall be
+ included in all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+ EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+ MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+License notice for Slf4j Simple
+------------------------------------------------------------------------------
+
+ Copyright (c) 2004-2017 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free of charge, to any person obtaining
+ a copy of this software and associated documentation files (the
+ "Software"), to deal in the Software without restriction, including
+ without limitation the rights to use, copy, modify, merge, publish,
+ distribute, sublicense, and/or sell copies of the Software, and to
+ permit persons to whom the Software is furnished to do so, subject to
+ the following conditions:
+
+ The above copyright notice and this permission notice shall be
+ included in all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+ EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+ MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+License notice for Guava (https://github.com/google/guava)
+------------------------------------------------------------------------------
+
+Copyright (C) 2010 The Guava Authors
+
+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 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.
+
+License notice for Netty
+------------------------------------------------------------------------------
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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.
+
+License notice for JUG Java Uuid Generator
+------------------------------------------------------------------------------
+
+JUG Java Uuid Generator
+
+Copyright (c) 2002- Tatu Saloranta, tatu.saloranta@iki.fi
+
+Licensed under the License specified in the file LICENSE which is
+included with the source code.
+You may not use this file except in compliance with the License.
+
+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.
+
+
+License notice for Jackson
+------------------------------------------------------------------------------
+
+Copyright (c) 2007 Jackson Project
+
+Jackson-annotations (http://github.com/FasterXML/jackson
+Jackson-core (https://github.com/FasterXML/jackson-core
+jackson-databind (http://github.com/FasterXML/jackson
+Jackson-dataformat-XML (https://github.com/FasterXML/jackson-dataformat-xml)
+Jackson datatype: JSR310 (https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
+Jackson module: Afterburner (https://github.com/FasterXML/jackson-modules-base)
+Jackson module: JAXB Annotations (https://github.com/FasterXML/jackson-modules-base)
+Woodstox (https://github.com/FasterXML/woodstox)
+
+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 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.
+
+
+License notice for Metrics Core
+------------------------------------------------------------------------------
+
+Copyright (c) 2010-2013 Coda Hale, Yammer.com, 2014-2020 Dropwizard Team
+
+Metrics Core (https://github.com/dropwizard/metrics)
+
+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 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.
+
+License notice for micrometer-core
+------------------------------------------------------------------------------
+
+Copyright (c) 2017-Present VMware, Inc. All Rights Reserved.
+
+micrometer-core (https://github.com/micrometer-metrics/micrometer)
+
+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 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.
+
+
+License notice for project Reactor
+------------------------------------------------------------------------------
+
+Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved.
+
+Non-Blocking Reactive Foundation for the JVM (https://github.com/reactor/reactor-core)
+reactor-scala-extensions (https://github.com/reactor/reactor-scala-extensions)
+Reactive Streams Netty driver (https://github.com/reactor/reactor-netty)
+
+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 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.
+
+
+License notice for JavaBeans Activation Framework API
+------------------------------------------------------------------------------
+
+JavaBeans Activation Framework API jar (https://github.com/eclipse-ee4j/jaf/jakarta.activation-api)
+
+Eclipse Distribution License - v 1.0
+Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice,
+this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation and/or other materials
+provided with the distribution.
+Neither the name of the Eclipse Foundation, Inc. nor the names of its contributors may be used to
+endorse or promote products derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR
+IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
+IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for jakarta.xml.bind-api
+------------------------------------------------------------------------------
+
+jakarta.xml.bind-api (https://github.com/eclipse-ee4j/jaxb-api/jakarta.xml.bind-api)
+
+Eclipse Distribution License - v 1.0
+Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice,
+this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice,
+this list of conditions and the following disclaimer in the documentation and/or other materials
+provided with the distribution.
+Neither the name of the Eclipse Foundation, Inc. nor the names of its contributors may be used to
+endorse or promote products derived from this software without specific prior written permission.
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR
+IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
+IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for jakarta.xml.bind-api
+------------------------------------------------------------------------------
+
+Stax2 API (http://github.com/FasterXML/stax2-api)
+Copyright (c) 2008 FasterXML LLC info@fasterxml.com
+
+This source code is licensed under standard BSD license, which is compatible with all Free and Open Software (OSS) licenses.
+
+
+License notice for HdrHistogrami
+------------------------------------------------------------------------------
+HdrHistogram (http://hdrhistogram.github.io/HdrHistogram/)
+
+The code in this repository code was Written by Gil Tene, Michael Barker,
+and Matt Warren, and released to the public domain, as explained at
+http://creativecommons.org/publicdomain/zero/1.0/
+
+For users of this code who wish to consume it under the "BSD" license
+rather than under the public domain or CC0 contribution text mentioned
+above, the code found under this directory is *also* provided under the
+following license (commonly referred to as the BSD 2-Clause License). This
+license does not detract from the above stated release of the code into
+the public domain, and simply represents an additional license granted by
+the Author.
+
+-----------------------------------------------------------------------------
+** Beginning of "BSD 2-Clause License" text. **
+
+ Copyright (c) 2012, 2013, 2014, 2015, 2016 Gil Tene
+ Copyright (c) 2014 Michael Barker
+ Copyright (c) 2014 Matt Warren
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are met:
+
+ 1. Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+
+ 2. Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+ THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for LatencyUtils
+------------------------------------------------------------------------------
+
+LatencyUtils (http://latencyutils.github.io/LatencyUtils/)
+
+ * This code was Written by Gil Tene of Azul Systems, and released to the
+ * public domain, as explained at http://creativecommons.org/publicdomain/zero/1.0/
+
+ For users of this code who wish to consume it under the "BSD" license
+ rather than under the public domain or CC0 contribution text mentioned
+ above, the code found under this directory is *also* provided under the
+ following license (commonly referred to as the BSD 2-Clause License). This
+ license does not detract from the above stated release of the code into
+ the public domain, and simply represents an additional license granted by
+ the Author.
+
+ -----------------------------------------------------------------------------
+ ** Beginning of "BSD 2-Clause License" text. **
+
+ Copyright (c) 2012, 2013, 2014 Gil Tene
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are met:
+
+ 1. Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+
+ 2. Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+ THE POSSIBILITY OF SUCH DAMAGE.
+
+
+License notice for reactive-streams
+------------------------------------------------------------------------------
+
+reactive-streams (http://www.reactive-streams.org/)
+
+Copyright Statement for Contributions to the Reactive Streams Project
+=====================================================================
+
+I hereby represent that all present, past and future contributions I make to
+the Reactive Streams project (which includes all repositories owned by the
+“reactive-streams” github organization) are governed by the Creative Commons
+Zero 1.0 Universal copyright statement, placing my contributions in the public
+domain. This entails that to the extent possible under law I waive all
+copyright and related or neighboring rights to the code or documents I
+contribute. I also represent that I have the authority to perform the above
+waiver with respect to the entirety of my contributions.
+
+The text of the copyright statement is included in the COPYING file at the root
+of the reactive-streams repository at
+https://github.com/reactive-streams/reactive-streams-jvm/blob/master/COPYING.
+
+Underwriting parties:
+
+github name | Real Name, Email Address used for git commits, Company
+---------------+----------------------------------------------------------------------------
+rkuhn | Roland Kuhn, rk@rkuhn.info, Typesafe Inc.
+benjchristensen| Ben Christensen, benjchristensen@gmail.com, Netflix Inc.
+viktorklang | Viktor Klang, viktor.klang@gmail.com, Typesafe Inc.
+smaldini | Stephane Maldini, stephane.maldini@gmail.com, Pivotal Software Inc.
+savulchik | Stanislav Savulchik, s.savulchik@gmail.com
+ktoso | Konrad Malawski, konrad.malawski@project13.pl, Typesafe Inc.
+ouertani | Slim Ouertani, ouertani@gmail.com
+2m | Martynas Mickevičius, mmartynas@gmail.com, Typesafe Inc.
+ldaley | Luke Daley, luke.daley@gradleware.com, Gradleware Inc.
+colinrgodsey | Colin Godsey, crgodsey@gmail.com, MediaMath Inc.
+davidmoten | Dave Moten, davidmoten@gmail.com
+briantopping | Brian Topping, brian.topping@gmail.com, Mauswerks LLC
+rstoyanchev | Rossen Stoyanchev, rstoyanchev@pivotal.io, Pivotal
+BjornHamels | Björn Hamels, bjorn@hamels.nl
+JakeWharton | Jake Wharton, jakewharton@gmail.com
+anthonyvdotbe | Anthony Vanelverdinghe, anthonyv.be@outlook.com
+seratch | Kazuhiro Sera, seratch@gmail.com, SmartNews, Inc.
+akarnokd | David Karnok, akarnokd@gmail.com
+egetman | Evgeniy Getman, getman.eugene@gmail.com
+patriknw | Patrik Nordwall, patrik.nordwall@gmail.com, Lightbend Inc
+angelsanz | Ángel Sanz, angelsanz@users.noreply.github.com
+shenghaiyang | 盛海洋, shenghaiyang@aliyun.com
+kiiadi | Kyle Thomson, kylthoms@amazon.com, Amazon.com
+jroper | James Roper, james@jazzy.id.au, Lightbend Inc.
+olegdokuka | Oleh Dokuka, shadowgun@.i.ua, Netifi Inc.
+Scottmitch | Scott Mitchell, scott_mitchell@apple.com, Apple Inc.
+retronym | Jason Zaugg, jzaugg@gmail.com, Lightbend Inc.
+
+Licensed under Public Domain (CC0)
+
+To the extent possible under law, the person who associated CC0 with
+this code has waived all copyright and related or neighboring
+rights to this code.
+
+You should have received a copy of the CC0 legalcode along with this
+work. If not, see <http://creativecommons.org/publicdomain/zero/1.0/>.
+
+License notice for Bouncy Castle
+------------------------------------------------------------------------------
+
+Copyright (c) 2000 - 2021 The Legion of the Bouncy Castle Inc. (https://www.bouncycastle.org)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+documentation files (the "Software"), to deal in the Software without restriction, including without limitation
+the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and
+to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
+WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS
+OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+
+License notice for KeePassJava2
+------------------------------------------------------------------------------
+
+Copyright 2015 Jo Rabin
+
+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 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.
+
+License notice for openkeepass
+------------------------------------------------------------------------------
+
+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 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.
+
+Openkeepass License Link: https://github.com/cternes/openkeepass/blob/master/LICENSE.txt
+-------------------------------------------------------------------------------------------------
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_main_sdk_keyvault_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_main_sdk_keyvault_LICENSE.txt
new file mode 100644
index 0000000..d1ca00f
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_main_sdk_keyvault_LICENSE.txt
@@ -0,0 +1,21 @@
+ MIT License
+
+ Copyright (c) Microsoft Corporation. All rights reserved.
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_master_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_master_NOTICE.txt
deleted file mode 100644
index 76791aa..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_Azure_azure-sdk-for-java_master_NOTICE.txt
+++ /dev/null
@@ -1,159 +0,0 @@
-NOTICES AND INFORMATION
-Do Not Translate or Localize
-
-This software incorporates material from third parties. Microsoft makes certain
-open source code available at https://3rdpartysource.microsoft.com, or you may
-send a check or money order for US $5.00, including the product name, the open
-source component name, and version number, to:
-
-Source Code Compliance Team
-Microsoft Corporation
-One Microsoft Way
-Redmond, WA 98052
-USA
-
-Notwithstanding any other terms, you may reverse engineer this software to the
-extent required to debug changes to any libraries licensed under the GNU Lesser
-General Public License.
-
-------------------------------------------------------------------------------
-
-Azure SDK for Java uses third-party libraries or other resources that may be
-distributed under licenses different than the Azure SDK for Java software.
-
-In the event that we accidentally failed to list a required notice, please
-bring it to our attention. Post an issue or email us:
-
- azjavasdkhelp@microsoft.com
-
-The attached notices are provided for information only.
-
-License notice for Hamcrest
-------------------------------------------------------------------------------
-
-The 3-Clause BSD License
-
-Copyright (c) 2000-2015 www.hamcrest.org
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-
-Redistributions of source code must retain the above copyright notice, this list of
-conditions and the following disclaimer. Redistributions in binary form must reproduce
-the above copyright notice, this list of conditions and the following disclaimer in
-the documentation and/or other materials provided with the distribution.
-
-Neither the name of Hamcrest nor the names of its contributors may be used to endorse
-or promote products derived from this software without specific prior written
-permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
-EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
-OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
-SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
-INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
-TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
-BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY
-WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
-DAMAGE.
-
-License notice for Slf4j API
-------------------------------------------------------------------------------
-
- Copyright (c) 2004-2017 QOS.ch
- All rights reserved.
-
- Permission is hereby granted, free of charge, to any person obtaining
- a copy of this software and associated documentation files (the
- "Software"), to deal in the Software without restriction, including
- without limitation the rights to use, copy, modify, merge, publish,
- distribute, sublicense, and/or sell copies of the Software, and to
- permit persons to whom the Software is furnished to do so, subject to
- the following conditions:
-
- The above copyright notice and this permission notice shall be
- included in all copies or substantial portions of the Software.
-
- THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
- EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
- MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
- NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
- LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
- OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
- WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-License notice for Slf4j Simple
-------------------------------------------------------------------------------
-
- Copyright (c) 2004-2017 QOS.ch
- All rights reserved.
-
- Permission is hereby granted, free of charge, to any person obtaining
- a copy of this software and associated documentation files (the
- "Software"), to deal in the Software without restriction, including
- without limitation the rights to use, copy, modify, merge, publish,
- distribute, sublicense, and/or sell copies of the Software, and to
- permit persons to whom the Software is furnished to do so, subject to
- the following conditions:
-
- The above copyright notice and this permission notice shall be
- included in all copies or substantial portions of the Software.
-
- THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
- EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
- MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
- NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
- LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
- OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
- WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-License notice for Guava (https://github.com/google/guava)
-------------------------------------------------------------------------------
-
-Copyright (C) 2010 The Guava Authors
-
-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 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.
-
-License notice for Netty
-------------------------------------------------------------------------------
-
-Copyright 2014 The Netty Project
-
-The Netty Project 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.
-
-License notice for JUG Java Uuid Generator
-------------------------------------------------------------------------------
-
-JUG Java Uuid Generator
-
-Copyright (c) 2002- Tatu Saloranta, tatu.saloranta@iki.fi
-
-Licensed under the License specified in the file LICENSE which is
-included with the source code.
-You may not use this file except in compliance with the License.
-
-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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_amzn_ion-java_v1.0.2_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_amzn_ion-java_v1.0.2_NOTICE.txt
new file mode 100644
index 0000000..91ea216
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_amzn_ion-java_v1.0.2_NOTICE.txt
@@ -0,0 +1,2 @@
+Amazon Ion Java
+Copyright 2007-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aslom_xpp3_master_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aslom_xpp3_master_LICENSE.txt
new file mode 100644
index 0000000..5d28ee1
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aslom_xpp3_master_LICENSE.txt
@@ -0,0 +1,46 @@
+Indiana University Extreme! Lab Software License
+
+Version 1.1.1
+
+Copyright (c) 2002 Extreme! Lab, Indiana University. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+1. Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright
+ notice, this list of conditions and the following disclaimer in
+ the documentation and/or other materials provided with the distribution.
+
+3. The end-user documentation included with the redistribution, if any,
+ must include the following acknowledgment:
+
+ "This product includes software developed by the Indiana University
+ Extreme! Lab (http://www.extreme.indiana.edu/)."
+
+Alternately, this acknowledgment may appear in the software itself,
+if and wherever such third-party acknowledgments normally appear.
+
+4. The names "Indiana Univeristy" and "Indiana Univeristy Extreme! Lab"
+must not be used to endorse or promote products derived from this
+software without prior written permission. For written permission,
+please contact http://www.extreme.indiana.edu/.
+
+5. Products derived from this software may not use "Indiana Univeristy"
+name nor may "Indiana Univeristy" appear in their name, without prior
+written permission of the Indiana University.
+
+THIS SOFTWARE IS PROVIDED "AS IS" AND ANY EXPRESSED OR IMPLIED
+WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+IN NO EVENT SHALL THE AUTHORS, COPYRIGHT HOLDERS OR ITS CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
+BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR
+OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java-v2_2.17.79_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java-v2_2.17.79_NOTICE.txt
new file mode 100644
index 0000000..58d7690
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java-v2_2.17.79_NOTICE.txt
@@ -0,0 +1,25 @@
+AWS SDK for Java 2.0
+Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+
+This product includes software developed by
+Amazon Technologies, Inc (http://www.amazon.com/).
+
+**********************
+THIRD PARTY COMPONENTS
+**********************
+This software includes third party software subject to the following copyrights:
+- XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty.
+- PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
+- Apache Commons Lang - https://github.com/apache/commons-lang
+- Netty Reactive Streams - https://github.com/playframework/netty-reactive-streams
+- Jackson-core - https://github.com/FasterXML/jackson-core
+- Jackson-dataformat-cbor - https://github.com/FasterXML/jackson-dataformats-binary
+
+The licenses for these third party components are included in LICENSE.txt
+
+- For Apache Commons Lang see also this required NOTICE:
+ Apache Commons Lang
+ Copyright 2001-2020 The Apache Software Foundation
+
+ This product includes software developed at
+ The Apache Software Foundation (https://www.apache.org/).
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java_1.12.109_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java_1.12.109_NOTICE.txt
new file mode 100644
index 0000000..797aa3c
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java_1.12.109_NOTICE.txt
@@ -0,0 +1,14 @@
+AWS SDK for Java
+Copyright 2010-2014 Amazon.com, Inc. or its affiliates. All Rights Reserved.
+
+This product includes software developed by
+Amazon Technologies, Inc (http://www.amazon.com/).
+
+**********************
+THIRD PARTY COMPONENTS
+**********************
+This software includes third party software subject to the following copyrights:
+- XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty.
+- PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
+
+The licenses for these third party components are included in LICENSE.txt
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_census-instrumentation_opencensus-java_v0.28.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_census-instrumentation_opencensus-java_v0.28.0_LICENSE.txt
new file mode 100644
index 0000000..7a4a3ea
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_census-instrumentation_opencensus-java_v0.28.0_LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_findbugsproject_findbugs_3.0.2_preview2_findbugs_licenses_LICENSE-jsr305.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_findbugsproject_findbugs_3.0.2_preview2_findbugs_licenses_LICENSE-jsr305.txt
new file mode 100644
index 0000000..cbe29d9
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_findbugsproject_findbugs_3.0.2_preview2_findbugs_licenses_LICENSE-jsr305.txt
@@ -0,0 +1,8 @@
+The JSR-305 reference implementation (lib/jsr305.jar) is
+distributed under the terms of the New BSD license:
+
+ http://www.opensource.org/licenses/bsd-license.php
+
+See the JSR-305 home page for more information:
+
+ http://code.google.com/p/jsr-305/
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_gson_gson-parent-2.8.6_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_gson_gson-parent-2.8.6_LICENSE.txt
new file mode 100644
index 0000000..7a4a3ea
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_gson_gson-parent-2.8.6_LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_guava_master_COPYING.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_guava_master_COPYING.txt
new file mode 100644
index 0000000..7a4a3ea
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_google_guava_master_COPYING.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_api-common-java_v1.10.3_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_api-common-java_v1.10.3_LICENSE.txt
new file mode 100644
index 0000000..4e19437
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_api-common-java_v1.10.3_LICENSE.txt
@@ -0,0 +1,27 @@
+Copyright 2016, Google Inc.
+All rights reserved.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v1.64.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v1.64.0_LICENSE.txt
new file mode 100644
index 0000000..267561b
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_gax-java_v1.64.0_LICENSE.txt
@@ -0,0 +1,27 @@
+Copyright 2016, Google Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-api-java-client_v1.31.5_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-api-java-client_v1.31.5_LICENSE.txt
new file mode 100644
index 0000000..f49a4e1
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-api-java-client_v1.31.5_LICENSE.txt
@@ -0,0 +1,201 @@
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-auth-library-java_v0.25.5_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-auth-library-java_v0.25.5_LICENSE.txt
new file mode 100644
index 0000000..2acb3d7
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-auth-library-java_v0.25.5_LICENSE.txt
@@ -0,0 +1,28 @@
+Copyright 2014, Google Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-http-java-client_v1.39.2_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-http-java-client_v1.39.2_LICENSE.txt
new file mode 100644
index 0000000..f49a4e1
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-http-java-client_v1.39.2_LICENSE.txt
@@ -0,0 +1,201 @@
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-oauth-java-client_v1.31.5_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-oauth-java-client_v1.31.5_LICENSE.txt
new file mode 100644
index 0000000..f49a4e1
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_google-oauth-java-client_v1.31.5_LICENSE.txt
@@ -0,0 +1,201 @@
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_java-core_v1.94.8_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_java-core_v1.94.8_LICENSE.txt
new file mode 100644
index 0000000..f49a4e1
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_googleapis_java-core_v1.94.8_LICENSE.txt
@@ -0,0 +1,201 @@
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.37.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.37.0_LICENSE.txt
new file mode 100644
index 0000000..7a4a3ea
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.37.0_LICENSE.txt
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ 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 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.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.37.0_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.37.0_NOTICE.txt
new file mode 100644
index 0000000..c5d3ec2
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_grpc_grpc-java_v1.37.0_NOTICE.txt
@@ -0,0 +1,62 @@
+Copyright 2014 The gRPC Authors
+
+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 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.
+
+-----------------------------------------------------------------------
+
+This product contains a modified portion of 'OkHttp', an open source
+HTTP & SPDY client for Android and Java applications, which can be obtained
+at:
+
+ * LICENSE:
+ * okhttp/third_party/okhttp/LICENSE (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/square/okhttp
+ * LOCATION_IN_GRPC:
+ * okhttp/third_party/okhttp
+
+This product contains a modified portion of 'Envoy', an open source
+cloud-native high-performance edge/middle/service proxy, which can be
+obtained at:
+
+ * LICENSE:
+ * xds/third_party/envoy/LICENSE (Apache License 2.0)
+ * NOTICE:
+ * xds/third_party/envoy/NOTICE
+ * HOMEPAGE:
+ * https://www.envoyproxy.io
+ * LOCATION_IN_GRPC:
+ * xds/third_party/envoy
+
+This product contains a modified portion of 'protoc-gen-validate (PGV)',
+an open source protoc plugin to generate polyglot message validators,
+which can be obtained at:
+
+ * LICENSE:
+ * xds/third_party/protoc-gen-validate/LICENSE (Apache License 2.0)
+ * NOTICE:
+ * xds/third_party/protoc-gen-validate/NOTICE
+ * HOMEPAGE:
+ * https://github.com/envoyproxy/protoc-gen-validate
+ * LOCATION_IN_GRPC:
+ * xds/third_party/protoc-gen-validate
+
+This product contains a modified portion of 'udpa',
+an open source universal data plane API, which can be obtained at:
+
+ * LICENSE:
+ * xds/third_party/udpa/LICENSE (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/cncf/udpa
+ * LOCATION_IN_GRPC:
+ * xds/third_party/udpa
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_javaee_javax.annotation_auto-value-1.8.1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_javaee_javax.annotation_auto-value-1.8.1_LICENSE.txt
new file mode 100644
index 0000000..68076ad
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_javaee_javax.annotation_auto-value-1.8.1_LICENSE.txt
@@ -0,0 +1,759 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.1
+
+1. Definitions.
+
+ 1.1. "Contributor" means each individual or entity that creates or
+ contributes to the creation of Modifications.
+
+ 1.2. "Contributor Version" means the combination of the Original
+ Software, prior Modifications used by a Contributor (if any), and
+ the Modifications made by that particular Contributor.
+
+ 1.3. "Covered Software" means (a) the Original Software, or (b)
+ Modifications, or (c) the combination of files containing Original
+ Software with files containing Modifications, in each case including
+ portions thereof.
+
+ 1.4. "Executable" means the Covered Software in any form other than
+ Source Code.
+
+ 1.5. "Initial Developer" means the individual or entity that first
+ makes Original Software available under this License.
+
+ 1.6. "Larger Work" means a work which combines Covered Software or
+ portions thereof with code not governed by the terms of this License.
+
+ 1.7. "License" means this document.
+
+ 1.8. "Licensable" means having the right to grant, to the maximum
+ extent possible, whether at the time of the initial grant or
+ subsequently acquired, any and all of the rights conveyed herein.
+
+ 1.9. "Modifications" means the Source Code and Executable form of
+ any of the following:
+
+ A. Any file that results from an addition to, deletion from or
+ modification of the contents of a file containing Original Software
+ or previous Modifications;
+
+ B. Any new file that contains any part of the Original Software or
+ previous Modification; or
+
+ C. Any new file that is contributed or otherwise made available
+ under the terms of this License.
+
+ 1.10. "Original Software" means the Source Code and Executable form
+ of computer software code that is originally released under this
+ License.
+
+ 1.11. "Patent Claims" means any patent claim(s), now owned or
+ hereafter acquired, including without limitation, method, process,
+ and apparatus claims, in any patent Licensable by grantor.
+
+ 1.12. "Source Code" means (a) the common form of computer software
+ code in which modifications are made and (b) associated
+ documentation included in or with such code.
+
+ 1.13. "You" (or "Your") means an individual or a legal entity
+ exercising rights under, and complying with all of the terms of,
+ this License. For legal entities, "You" includes any entity which
+ controls, is controlled by, or is under common control with You. For
+ purposes of this definition, "control" means (a) the power, direct
+ or indirect, to cause the direction or management of such entity,
+ whether by contract or otherwise, or (b) ownership of more than
+ fifty percent (50%) of the outstanding shares or beneficial
+ ownership of such entity.
+
+2. License Grants.
+
+ 2.1. The Initial Developer Grant.
+
+ Conditioned upon Your compliance with Section 3.1 below and subject
+ to third party intellectual property claims, the Initial Developer
+ hereby grants You a world-wide, royalty-free, non-exclusive license:
+
+ (a) under intellectual property rights (other than patent or
+ trademark) Licensable by Initial Developer, to use, reproduce,
+ modify, display, perform, sublicense and distribute the Original
+ Software (or portions thereof), with or without Modifications,
+ and/or as part of a Larger Work; and
+
+ (b) under Patent Claims infringed by the making, using or selling of
+ Original Software, to make, have made, use, practice, sell, and
+ offer for sale, and/or otherwise dispose of the Original Software
+ (or portions thereof).
+
+ (c) The licenses granted in Sections 2.1(a) and (b) are effective on
+ the date Initial Developer first distributes or otherwise makes the
+ Original Software available to a third party under the terms of this
+ License.
+
+ (d) Notwithstanding Section 2.1(b) above, no patent license is
+ granted: (1) for code that You delete from the Original Software, or
+ (2) for infringements caused by: (i) the modification of the
+ Original Software, or (ii) the combination of the Original Software
+ with other software or devices.
+
+ 2.2. Contributor Grant.
+
+ Conditioned upon Your compliance with Section 3.1 below and subject
+ to third party intellectual property claims, each Contributor hereby
+ grants You a world-wide, royalty-free, non-exclusive license:
+
+ (a) under intellectual property rights (other than patent or
+ trademark) Licensable by Contributor to use, reproduce, modify,
+ display, perform, sublicense and distribute the Modifications
+ created by such Contributor (or portions thereof), either on an
+ unmodified basis, with other Modifications, as Covered Software
+ and/or as part of a Larger Work; and
+
+ (b) under Patent Claims infringed by the making, using, or selling
+ of Modifications made by that Contributor either alone and/or in
+ combination with its Contributor Version (or portions of such
+ combination), to make, use, sell, offer for sale, have made, and/or
+ otherwise dispose of: (1) Modifications made by that Contributor (or
+ portions thereof); and (2) the combination of Modifications made by
+ that Contributor with its Contributor Version (or portions of such
+ combination).
+
+ (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective
+ on the date Contributor first distributes or otherwise makes the
+ Modifications available to a third party.
+
+ (d) Notwithstanding Section 2.2(b) above, no patent license is
+ granted: (1) for any code that Contributor has deleted from the
+ Contributor Version; (2) for infringements caused by: (i) third
+ party modifications of Contributor Version, or (ii) the combination
+ of Modifications made by that Contributor with other software
+ (except as part of the Contributor Version) or other devices; or (3)
+ under Patent Claims infringed by Covered Software in the absence of
+ Modifications made by that Contributor.
+
+3. Distribution Obligations.
+
+ 3.1. Availability of Source Code.
+
+ Any Covered Software that You distribute or otherwise make available
+ in Executable form must also be made available in Source Code form
+ and that Source Code form must be distributed only under the terms
+ of this License. You must include a copy of this License with every
+ copy of the Source Code form of the Covered Software You distribute
+ or otherwise make available. You must inform recipients of any such
+ Covered Software in Executable form as to how they can obtain such
+ Covered Software in Source Code form in a reasonable manner on or
+ through a medium customarily used for software exchange.
+
+ 3.2. Modifications.
+
+ The Modifications that You create or to which You contribute are
+ governed by the terms of this License. You represent that You
+ believe Your Modifications are Your original creation(s) and/or You
+ have sufficient rights to grant the rights conveyed by this License.
+
+ 3.3. Required Notices.
+
+ You must include a notice in each of Your Modifications that
+ identifies You as the Contributor of the Modification. You may not
+ remove or alter any copyright, patent or trademark notices contained
+ within the Covered Software, or any notices of licensing or any
+ descriptive text giving attribution to any Contributor or the
+ Initial Developer.
+
+ 3.4. Application of Additional Terms.
+
+ You may not offer or impose any terms on any Covered Software in
+ Source Code form that alters or restricts the applicable version of
+ this License or the recipients' rights hereunder. You may choose to
+ offer, and to charge a fee for, warranty, support, indemnity or
+ liability obligations to one or more recipients of Covered Software.
+ However, you may do so only on Your own behalf, and not on behalf of
+ the Initial Developer or any Contributor. You must make it
+ absolutely clear that any such warranty, support, indemnity or
+ liability obligation is offered by You alone, and You hereby agree
+ to indemnify the Initial Developer and every Contributor for any
+ liability incurred by the Initial Developer or such Contributor as a
+ result of warranty, support, indemnity or liability terms You offer.
+
+ 3.5. Distribution of Executable Versions.
+
+ You may distribute the Executable form of the Covered Software under
+ the terms of this License or under the terms of a license of Your
+ choice, which may contain terms different from this License,
+ provided that You are in compliance with the terms of this License
+ and that the license for the Executable form does not attempt to
+ limit or alter the recipient's rights in the Source Code form from
+ the rights set forth in this License. If You distribute the Covered
+ Software in Executable form under a different license, You must make
+ it absolutely clear that any terms which differ from this License
+ are offered by You alone, not by the Initial Developer or
+ Contributor. You hereby agree to indemnify the Initial Developer and
+ every Contributor for any liability incurred by the Initial
+ Developer or such Contributor as a result of any such terms You offer.
+
+ 3.6. Larger Works.
+
+ You may create a Larger Work by combining Covered Software with
+ other code not governed by the terms of this License and distribute
+ the Larger Work as a single product. In such a case, You must make
+ sure the requirements of this License are fulfilled for the Covered
+ Software.
+
+4. Versions of the License.
+
+ 4.1. New Versions.
+
+ Oracle is the initial license steward and may publish revised and/or
+ new versions of this License from time to time. Each version will be
+ given a distinguishing version number. Except as provided in Section
+ 4.3, no one other than the license steward has the right to modify
+ this License.
+
+ 4.2. Effect of New Versions.
+
+ You may always continue to use, distribute or otherwise make the
+ Covered Software available under the terms of the version of the
+ License under which You originally received the Covered Software. If
+ the Initial Developer includes a notice in the Original Software
+ prohibiting it from being distributed or otherwise made available
+ under any subsequent version of the License, You must distribute and
+ make the Covered Software available under the terms of the version
+ of the License under which You originally received the Covered
+ Software. Otherwise, You may also choose to use, distribute or
+ otherwise make the Covered Software available under the terms of any
+ subsequent version of the License published by the license steward.
+
+ 4.3. Modified Versions.
+
+ When You are an Initial Developer and You want to create a new
+ license for Your Original Software, You may create and use a
+ modified version of this License if You: (a) rename the license and
+ remove any references to the name of the license steward (except to
+ note that the license differs from this License); and (b) otherwise
+ make it clear that the license contains terms which differ from this
+ License.
+
+5. DISCLAIMER OF WARRANTY.
+
+ COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS,
+ WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED,
+ INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE
+ IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR
+ NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF
+ THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE
+ DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY
+ OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING,
+ REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN
+ ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS
+ AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+ 6.1. This License and the rights granted hereunder will terminate
+ automatically if You fail to comply with terms herein and fail to
+ cure such breach within 30 days of becoming aware of the breach.
+ Provisions which, by their nature, must remain in effect beyond the
+ termination of this License shall survive.
+
+ 6.2. If You assert a patent infringement claim (excluding
+ declaratory judgment actions) against Initial Developer or a
+ Contributor (the Initial Developer or Contributor against whom You
+ assert such claim is referred to as "Participant") alleging that the
+ Participant Software (meaning the Contributor Version where the
+ Participant is a Contributor or the Original Software where the
+ Participant is the Initial Developer) directly or indirectly
+ infringes any patent, then any and all rights granted directly or
+ indirectly to You by such Participant, the Initial Developer (if the
+ Initial Developer is not the Participant) and all Contributors under
+ Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice
+ from Participant terminate prospectively and automatically at the
+ expiration of such 60 day notice period, unless if within such 60
+ day period You withdraw Your claim with respect to the Participant
+ Software against such Participant either unilaterally or pursuant to
+ a written agreement with Participant.
+
+ 6.3. If You assert a patent infringement claim against Participant
+ alleging that the Participant Software directly or indirectly
+ infringes any patent where such claim is resolved (such as by
+ license or settlement) prior to the initiation of patent
+ infringement litigation, then the reasonable value of the licenses
+ granted by such Participant under Sections 2.1 or 2.2 shall be taken
+ into account in determining the amount or value of any payment or
+ license.
+
+ 6.4. In the event of termination under Sections 6.1 or 6.2 above,
+ all end user licenses that have been validly granted by You or any
+ distributor hereunder prior to termination (excluding licenses
+ granted to You by any distributor) shall survive termination.
+
+7. LIMITATION OF LIABILITY.
+
+ UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT
+ (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE
+ INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+ COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE
+ TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR
+ CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT
+ LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER
+ FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR
+ LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE
+ POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT
+ APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH
+ PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH
+ LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR
+ LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION
+ AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS.
+
+ The Covered Software is a "commercial item," as that term is defined
+ in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer
+ software" (as that term is defined at 48 C.F.R. §
+ 252.227-7014(a)(1)) and "commercial computer software documentation"
+ as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent
+ with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4
+ (June 1995), all U.S. Government End Users acquire Covered Software
+ with only those rights set forth herein. This U.S. Government Rights
+ clause is in lieu of, and supersedes, any other FAR, DFAR, or other
+ clause or provision that addresses Government rights in computer
+ software under this License.
+
+9. MISCELLANEOUS.
+
+ This License represents the complete agreement concerning subject
+ matter hereof. If any provision of this License is held to be
+ unenforceable, such provision shall be reformed only to the extent
+ necessary to make it enforceable. This License shall be governed by
+ the law of the jurisdiction specified in a notice contained within
+ the Original Software (except to the extent applicable law, if any,
+ provides otherwise), excluding such jurisdiction's conflict-of-law
+ provisions. Any litigation relating to this License shall be subject
+ to the jurisdiction of the courts located in the jurisdiction and
+ venue specified in a notice contained within the Original Software,
+ with the losing party responsible for costs, including, without
+ limitation, court costs and reasonable attorneys' fees and expenses.
+ The application of the United Nations Convention on Contracts for
+ the International Sale of Goods is expressly excluded. Any law or
+ regulation which provides that the language of a contract shall be
+ construed against the drafter shall not apply to this License. You
+ agree that You alone are responsible for compliance with the United
+ States export administration regulations (and the export control
+ laws and regulation of any other countries) when You use, distribute
+ or otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS.
+
+ As between Initial Developer and the Contributors, each party is
+ responsible for claims and damages arising, directly or indirectly,
+ out of its utilization of rights under this License and You agree to
+ work with Initial Developer and Contributors to distribute such
+ responsibility on an equitable basis. Nothing herein is intended or
+ shall be deemed to constitute any admission of liability.
+
+------------------------------------------------------------------------
+
+NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION
+LICENSE (CDDL)
+
+The code released under the CDDL shall be governed by the laws of the
+State of California (excluding conflict-of-law provisions). Any
+litigation relating to this License shall be subject to the jurisdiction
+of the Federal Courts of the Northern District of California and the
+state courts of the State of California, with venue lying in Santa Clara
+County, California.
+
+
+
+ The GNU General Public License (GPL) Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+51 Franklin Street, Fifth Floor
+Boston, MA 02110-1335
+USA
+
+Everyone is permitted to copy and distribute verbatim copies
+of this license document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to
+share and change it. By contrast, the GNU General Public License is
+intended to guarantee your freedom to share and change free software--to
+make sure the software is free for all its users. This General Public
+License applies to most of the Free Software Foundation's software and
+to any other program whose authors commit to using it. (Some other Free
+Software Foundation software is covered by the GNU Library General
+Public License instead.) You can apply it to your programs, too.
+
+When we speak of free software, we are referring to freedom, not price.
+Our General Public Licenses are designed to make sure that you have the
+freedom to distribute copies of free software (and charge for this
+service if you wish), that you receive source code or can get it if you
+want it, that you can change the software or use pieces of it in new
+free programs; and that you know you can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone
+to deny you these rights or to ask you to surrender the rights. These
+restrictions translate to certain responsibilities for you if you
+distribute copies of the software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis
+or for a fee, you must give the recipients all the rights that you have.
+You must make sure that they, too, receive or can get the source code.
+And you must show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and
+(2) offer you this license which gives you legal permission to copy,
+distribute and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain
+that everyone understands that there is no warranty for this free
+software. If the software is modified by someone else and passed on, we
+want its recipients to know that what they have is not the original, so
+that any problems introduced by others will not reflect on the original
+authors' reputations.
+
+Finally, any free program is threatened constantly by software patents.
+We wish to avoid the danger that redistributors of a free program will
+individually obtain patent licenses, in effect making the program
+proprietary. To prevent this, we have made it clear that any patent must
+be licensed for everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and
+modification follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a
+notice placed by the copyright holder saying it may be distributed under
+the terms of this General Public License. The "Program", below, refers
+to any such program or work, and a "work based on the Program" means
+either the Program or any derivative work under copyright law: that is
+to say, a work containing the Program or a portion of it, either
+verbatim or with modifications and/or translated into another language.
+(Hereinafter, translation is included without limitation in the term
+"modification".) Each licensee is addressed as "you".
+
+Activities other than copying, distribution and modification are not
+covered by this License; they are outside its scope. The act of running
+the Program is not restricted, and the output from the Program is
+covered only if its contents constitute a work based on the Program
+(independent of having been made by running the Program). Whether that
+is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source
+code as you receive it, in any medium, provided that you conspicuously
+and appropriately publish on each copy an appropriate copyright notice
+and disclaimer of warranty; keep intact all the notices that refer to
+this License and to the absence of any warranty; and give any other
+recipients of the Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and
+you may at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of
+it, thus forming a work based on the Program, and copy and distribute
+such modifications or work under the terms of Section 1 above, provided
+that you also meet all of these conditions:
+
+ a) You must cause the modified files to carry prominent notices
+ stating that you changed the files and the date of any change.
+
+ b) You must cause any work that you distribute or publish, that in
+ whole or in part contains or is derived from the Program or any part
+ thereof, to be licensed as a whole at no charge to all third parties
+ under the terms of this License.
+
+ c) If the modified program normally reads commands interactively
+ when run, you must cause it, when started running for such
+ interactive use in the most ordinary way, to print or display an
+ announcement including an appropriate copyright notice and a notice
+ that there is no warranty (or else, saying that you provide a
+ warranty) and that users may redistribute the program under these
+ conditions, and telling the user how to view a copy of this License.
+ (Exception: if the Program itself is interactive but does not
+ normally print such an announcement, your work based on the Program
+ is not required to print an announcement.)
+
+These requirements apply to the modified work as a whole. If
+identifiable sections of that work are not derived from the Program, and
+can be reasonably considered independent and separate works in
+themselves, then this License, and its terms, do not apply to those
+sections when you distribute them as separate works. But when you
+distribute the same sections as part of a whole which is a work based on
+the Program, the distribution of the whole must be on the terms of this
+License, whose permissions for other licensees extend to the entire
+whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest
+your rights to work written entirely by you; rather, the intent is to
+exercise the right to control the distribution of derivative or
+collective works based on the Program.
+
+In addition, mere aggregation of another work not based on the Program
+with the Program (or with a work based on the Program) on a volume of a
+storage or distribution medium does not bring the other work under the
+scope of this License.
+
+3. You may copy and distribute the Program (or a work based on it,
+under Section 2) in object code or executable form under the terms of
+Sections 1 and 2 above provided that you also do one of the following:
+
+ a) Accompany it with the complete corresponding machine-readable
+ source code, which must be distributed under the terms of Sections 1
+ and 2 above on a medium customarily used for software interchange; or,
+
+ b) Accompany it with a written offer, valid for at least three
+ years, to give any third party, for a charge no more than your cost
+ of physically performing source distribution, a complete
+ machine-readable copy of the corresponding source code, to be
+ distributed under the terms of Sections 1 and 2 above on a medium
+ customarily used for software interchange; or,
+
+ c) Accompany it with the information you received as to the offer to
+ distribute corresponding source code. (This alternative is allowed
+ only for noncommercial distribution and only if you received the
+ program in object code or executable form with such an offer, in
+ accord with Subsection b above.)
+
+The source code for a work means the preferred form of the work for
+making modifications to it. For an executable work, complete source code
+means all the source code for all modules it contains, plus any
+associated interface definition files, plus the scripts used to control
+compilation and installation of the executable. However, as a special
+exception, the source code distributed need not include anything that is
+normally distributed (in either source or binary form) with the major
+components (compiler, kernel, and so on) of the operating system on
+which the executable runs, unless that component itself accompanies the
+executable.
+
+If distribution of executable or object code is made by offering access
+to copy from a designated place, then offering equivalent access to copy
+the source code from the same place counts as distribution of the source
+code, even though third parties are not compelled to copy the source
+along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program
+except as expressly provided under this License. Any attempt otherwise
+to copy, modify, sublicense or distribute the Program is void, and will
+automatically terminate your rights under this License. However, parties
+who have received copies, or rights, from you under this License will
+not have their licenses terminated so long as such parties remain in
+full compliance.
+
+5. You are not required to accept this License, since you have not
+signed it. However, nothing else grants you permission to modify or
+distribute the Program or its derivative works. These actions are
+prohibited by law if you do not accept this License. Therefore, by
+modifying or distributing the Program (or any work based on the
+Program), you indicate your acceptance of this License to do so, and all
+its terms and conditions for copying, distributing or modifying the
+Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the
+Program), the recipient automatically receives a license from the
+original licensor to copy, distribute or modify the Program subject to
+these terms and conditions. You may not impose any further restrictions
+on the recipients' exercise of the rights granted herein. You are not
+responsible for enforcing compliance by third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues),
+conditions are imposed on you (whether by court order, agreement or
+otherwise) that contradict the conditions of this License, they do not
+excuse you from the conditions of this License. If you cannot distribute
+so as to satisfy simultaneously your obligations under this License and
+any other pertinent obligations, then as a consequence you may not
+distribute the Program at all. For example, if a patent license would
+not permit royalty-free redistribution of the Program by all those who
+receive copies directly or indirectly through you, then the only way you
+could satisfy both it and this License would be to refrain entirely from
+distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under
+any particular circumstance, the balance of the section is intended to
+apply and the section as a whole is intended to apply in other
+circumstances.
+
+It is not the purpose of this section to induce you to infringe any
+patents or other property right claims or to contest validity of any
+such claims; this section has the sole purpose of protecting the
+integrity of the free software distribution system, which is implemented
+by public license practices. Many people have made generous
+contributions to the wide range of software distributed through that
+system in reliance on consistent application of that system; it is up to
+the author/donor to decide if he or she is willing to distribute
+software through any other system and a licensee cannot impose that choice.
+
+This section is intended to make thoroughly clear what is believed to be
+a consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in
+certain countries either by patents or by copyrighted interfaces, the
+original copyright holder who places the Program under this License may
+add an explicit geographical distribution limitation excluding those
+countries, so that distribution is permitted only in or among countries
+not thus excluded. In such case, this License incorporates the
+limitation as if written in the body of this License.
+
+9. The Free Software Foundation may publish revised and/or new
+versions of the General Public License from time to time. Such new
+versions will be similar in spirit to the present version, but may
+differ in detail to address new problems or concerns.
+
+Each version is given a distinguishing version number. If the Program
+specifies a version number of this License which applies to it and "any
+later version", you have the option of following the terms and
+conditions either of that version or of any later version published by
+the Free Software Foundation. If the Program does not specify a version
+number of this License, you may choose any version ever published by the
+Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free
+programs whose distribution conditions are different, write to the
+author to ask for permission. For software which is copyrighted by the
+Free Software Foundation, write to the Free Software Foundation; we
+sometimes make exceptions for this. Our decision will be guided by the
+two goals of preserving the free status of all derivatives of our free
+software and of promoting the sharing and reuse of software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO
+WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW.
+EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR
+OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND,
+EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE
+ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH
+YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL
+NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN
+WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY
+AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR
+DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL
+DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM
+(INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED
+INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF
+THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR
+OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest
+possible use to the public, the best way to achieve this is to make it
+free software which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program. It is safest to
+attach them to the start of each source file to most effectively convey
+the exclusion of warranty; and each file should have at least the
+"copyright" line and a pointer to where the full notice is found.
+
+ One line to give the program's name and a brief idea of what it does.
+ Copyright (C) <year> <name of author>
+
+ This program is free software; you can redistribute it and/or modify
+ it under the terms of the GNU General Public License as published by
+ the Free Software Foundation; either version 2 of the License, or
+ (at your option) any later version.
+
+ This program is distributed in the hope that it will be useful, but
+ WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with this program; if not, write to the Free Software
+ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this
+when it starts in an interactive mode:
+
+ Gnomovision version 69, Copyright (C) year name of author
+ Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type
+ `show w'. This is free software, and you are welcome to redistribute
+ it under certain conditions; type `show c' for details.
+
+The hypothetical commands `show w' and `show c' should show the
+appropriate parts of the General Public License. Of course, the commands
+you use may be called something other than `show w' and `show c'; they
+could even be mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your
+school, if any, to sign a "copyright disclaimer" for the program, if
+necessary. Here is a sample; alter the names:
+
+ Yoyodyne, Inc., hereby disclaims all copyright interest in the
+ program `Gnomovision' (which makes passes at compilers) written by
+ James Hacker.
+
+ signature of Ty Coon, 1 April 1989
+ Ty Coon, President of Vice
+
+This General Public License does not permit incorporating your program
+into proprietary programs. If your program is a subroutine library, you
+may consider it more useful to permit linking proprietary applications
+with the library. If this is what you want to do, use the GNU Library
+General Public License instead of this License.
+
+#
+
+Certain source files distributed by Oracle America, Inc. and/or its
+affiliates are subject to the following clarification and special
+exception to the GPLv2, based on the GNU Project exception for its
+Classpath libraries, known as the GNU Classpath Exception, but only
+where Oracle has expressly included in the particular source file's
+header the words "Oracle designates this particular file as subject to
+the "Classpath" exception as provided by Oracle in the LICENSE file
+that accompanied this code."
+
+You should also note that Oracle includes multiple, independent
+programs in this software package. Some of those programs are provided
+under licenses deemed incompatible with the GPLv2 by the Free Software
+Foundation and others. For example, the package includes programs
+licensed under the Apache License, Version 2.0. Such programs are
+licensed to you under their original licenses.
+
+Oracle facilitates your further distribution of this package by adding
+the Classpath Exception to the necessary parts of its GPLv2 code, which
+permits you to use that code in combination with other independent
+modules not licensed under the GPLv2. However, note that this would
+not permit you to commingle code under an incompatible license with
+Oracle's GPLv2 licensed code by, for example, cutting and pasting such
+code into a file also containing Oracle's GPLv2 licensed code and then
+distributing the result. Additionally, if you were to remove the
+Classpath Exception from any of the files to which it applies and
+distribute the result, you would likely be required to license some or
+all of the other code in that distribution under the GPLv2 as well, and
+since the GPLv2 is incompatible with the license terms of some items
+included in the distribution by Oracle, removing the Classpath
+Exception could therefore effectively compromise your ability to
+further distribute the package.
+
+Proceed with caution and we recommend that you obtain the advice of a
+lawyer skilled in open source matters before removing the Classpath
+Exception or making modifications to this package which may
+subsequently be redistributed and/or involve the use of third party
+software.
+
+CLASSPATH EXCEPTION
+Linking this library statically or dynamically with other modules is
+making a combined work based on this library. Thus, the terms and
+conditions of the GNU General Public License version 2 cover the whole
+combination.
+
+As a special exception, the copyright holders of this library give you
+permission to link this library with independent modules to produce an
+executable, regardless of the license terms of these independent
+modules, and to copy and distribute the resulting executable under
+terms of your choice, provided that you also meet, for each linked
+independent module, the terms and conditions of the license of that
+module. An independent module is a module which is not derived from or
+based on this library. If you modify this library, you may extend this
+exception to your version of the library, but you are not obligated to
+do so. If you do not wish to do so, delete this exception statement
+from your version.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_llbit_ow2-asm_master_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_llbit_ow2-asm_master_LICENSE.txt
new file mode 100644
index 0000000..ed44300
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_llbit_ow2-asm_master_LICENSE.txt
@@ -0,0 +1,28 @@
+
+ ASM: a very small and fast Java bytecode manipulation framework
+ Copyright (c) 2000-2011 INRIA, France Telecom
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions
+ are met:
+ 1. Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+ 2. Redistributions in binary form must reproduce the above copyright
+ notice, this list of conditions and the following disclaimer in the
+ documentation and/or other materials provided with the distribution.
+ 3. Neither the name of the copyright holders nor the names of its
+ contributors may be used to endorse or promote products derived from
+ this software without specific prior written permission.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+ THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_luben_zstd-jni_v1.4.9-1_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_luben_zstd-jni_v1.4.9-1_LICENSE.txt
new file mode 100644
index 0000000..7bdccb6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_luben_zstd-jni_v1.4.9-1_LICENSE.txt
@@ -0,0 +1,26 @@
+Zstd-jni: JNI bindings to Zstd Library
+
+Copyright (c) 2015-present, Luben Karavelov/ All rights reserved.
+
+BSD License
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice, this
+ list of conditions and the following disclaimer in the documentation and/or
+ other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.63.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.63.Final_NOTICE.txt
deleted file mode 100644
index a771573..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.63.Final_NOTICE.txt
+++ /dev/null
@@ -1,248 +0,0 @@
-
- The Netty Project
- =================
-
-Please visit the Netty web site for more information:
-
- * https://netty.io/
-
-Copyright 2014 The Netty Project
-
-The Netty Project 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:
-
- https://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.
-
-Also, please refer to each LICENSE.<component>.txt file, which is located in
-the 'license' directory of the distribution file, for the license terms of the
-components that this product depends on.
-
--------------------------------------------------------------------------------
-This product contains the extensions to Java Collections Framework which has
-been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
-
- * LICENSE:
- * license/LICENSE.jsr166y.txt (Public Domain)
- * HOMEPAGE:
- * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
- * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
-
-This product contains a modified version of Robert Harder's Public Domain
-Base64 Encoder and Decoder, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.base64.txt (Public Domain)
- * HOMEPAGE:
- * http://iharder.sourceforge.net/current/java/base64/
-
-This product contains a modified portion of 'Webbit', an event based
-WebSocket and HTTP server, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.webbit.txt (BSD License)
- * HOMEPAGE:
- * https://github.com/joewalnes/webbit
-
-This product contains a modified portion of 'SLF4J', a simple logging
-facade for Java, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.slf4j.txt (MIT License)
- * HOMEPAGE:
- * https://www.slf4j.org/
-
-This product contains a modified portion of 'Apache Harmony', an open source
-Java SE, which can be obtained at:
-
- * NOTICE:
- * license/NOTICE.harmony.txt
- * LICENSE:
- * license/LICENSE.harmony.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://archive.apache.org/dist/harmony/
-
-This product contains a modified portion of 'jbzip2', a Java bzip2 compression
-and decompression library written by Matthew J. Francis. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jbzip2.txt (MIT License)
- * HOMEPAGE:
- * https://code.google.com/p/jbzip2/
-
-This product contains a modified portion of 'libdivsufsort', a C API library to construct
-the suffix array and the Burrows-Wheeler transformed string for any input string of
-a constant-size alphabet written by Yuta Mori. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.libdivsufsort.txt (MIT License)
- * HOMEPAGE:
- * https://github.com/y-256/libdivsufsort
-
-This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
- which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jctools.txt (ASL2 License)
- * HOMEPAGE:
- * https://github.com/JCTools/JCTools
-
-This product optionally depends on 'JZlib', a re-implementation of zlib in
-pure Java, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jzlib.txt (BSD style License)
- * HOMEPAGE:
- * http://www.jcraft.com/jzlib/
-
-This product optionally depends on 'Compress-LZF', a Java library for encoding and
-decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.compress-lzf.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/ning/compress
-
-This product optionally depends on 'lz4', a LZ4 Java compression
-and decompression library written by Adrien Grand. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.lz4.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/jpountz/lz4-java
-
-This product optionally depends on 'lzma-java', a LZMA Java compression
-and decompression library, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.lzma-java.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/jponge/lzma-java
-
-This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
-and decompression library written by William Kinney. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jfastlz.txt (MIT License)
- * HOMEPAGE:
- * https://code.google.com/p/jfastlz/
-
-This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
-interchange format, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.protobuf.txt (New BSD License)
- * HOMEPAGE:
- * https://github.com/google/protobuf
-
-This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
-a temporary self-signed X.509 certificate when the JVM does not provide the
-equivalent functionality. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.bouncycastle.txt (MIT License)
- * HOMEPAGE:
- * https://www.bouncycastle.org/
-
-This product optionally depends on 'Snappy', a compression library produced
-by Google Inc, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.snappy.txt (New BSD License)
- * HOMEPAGE:
- * https://github.com/google/snappy
-
-This product optionally depends on 'JBoss Marshalling', an alternative Java
-serialization API, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jboss-marshalling.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/jboss-remoting/jboss-marshalling
-
-This product optionally depends on 'Caliper', Google's micro-
-benchmarking framework, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.caliper.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/google/caliper
-
-This product optionally depends on 'Apache Commons Logging', a logging
-framework, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.commons-logging.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://commons.apache.org/logging/
-
-This product optionally depends on 'Apache Log4J', a logging framework, which
-can be obtained at:
-
- * LICENSE:
- * license/LICENSE.log4j.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://logging.apache.org/log4j/
-
-This product optionally depends on 'Aalto XML', an ultra-high performance
-non-blocking XML processor, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.aalto-xml.txt (Apache License 2.0)
- * HOMEPAGE:
- * http://wiki.fasterxml.com/AaltoHome
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.hpack.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/twitter/hpack
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Cory Benfield. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.hyper-hpack.txt (MIT License)
- * HOMEPAGE:
- * https://github.com/python-hyper/hpack/
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Tatsuhiro Tsujikawa. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.nghttp2-hpack.txt (MIT License)
- * HOMEPAGE:
- * https://github.com/nghttp2/nghttp2/
-
-This product contains a modified portion of 'Apache Commons Lang', a Java library
-provides utilities for the java.lang API, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.commons-lang.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://commons.apache.org/proper/commons-lang/
-
-
-This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build.
-
- * LICENSE:
- * license/LICENSE.mvn-wrapper.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/takari/maven-wrapper
-
-This product contains the dnsinfo.h header file, that provides a way to retrieve the system DNS configuration on MacOS.
-This private header is also used by Apple's open source
- mDNSResponder (https://opensource.apple.com/tarballs/mDNSResponder/).
-
- * LICENSE:
- * license/LICENSE.dnsinfo.txt (Apple Public Source License 2.0)
- * HOMEPAGE:
- * https://www.opensource.apple.com/source/configd/configd-453.19/dnsinfo/dnsinfo.h
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.73.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.73.Final_NOTICE.txt
new file mode 100644
index 0000000..ad62b68
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.73.Final_NOTICE.txt
@@ -0,0 +1,264 @@
+
+ The Netty Project
+ =================
+
+Please visit the Netty web site for more information:
+
+ * https://netty.io/
+
+Copyright 2014 The Netty Project
+
+The Netty Project 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:
+
+ https://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.
+
+Also, please refer to each LICENSE.<component>.txt file, which is located in
+the 'license' directory of the distribution file, for the license terms of the
+components that this product depends on.
+
+-------------------------------------------------------------------------------
+This product contains the extensions to Java Collections Framework which has
+been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+ * LICENSE:
+ * license/LICENSE.jsr166y.txt (Public Domain)
+ * HOMEPAGE:
+ * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+ * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+This product contains a modified version of Robert Harder's Public Domain
+Base64 Encoder and Decoder, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.base64.txt (Public Domain)
+ * HOMEPAGE:
+ * http://iharder.sourceforge.net/current/java/base64/
+
+This product contains a modified portion of 'Webbit', an event based
+WebSocket and HTTP server, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.webbit.txt (BSD License)
+ * HOMEPAGE:
+ * https://github.com/joewalnes/webbit
+
+This product contains a modified portion of 'SLF4J', a simple logging
+facade for Java, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.slf4j.txt (MIT License)
+ * HOMEPAGE:
+ * https://www.slf4j.org/
+
+This product contains a modified portion of 'Apache Harmony', an open source
+Java SE, which can be obtained at:
+
+ * NOTICE:
+ * license/NOTICE.harmony.txt
+ * LICENSE:
+ * license/LICENSE.harmony.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://archive.apache.org/dist/harmony/
+
+This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+and decompression library written by Matthew J. Francis. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jbzip2.txt (MIT License)
+ * HOMEPAGE:
+ * https://code.google.com/p/jbzip2/
+
+This product contains a modified portion of 'libdivsufsort', a C API library to construct
+the suffix array and the Burrows-Wheeler transformed string for any input string of
+a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.libdivsufsort.txt (MIT License)
+ * HOMEPAGE:
+ * https://github.com/y-256/libdivsufsort
+
+This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+ which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jctools.txt (ASL2 License)
+ * HOMEPAGE:
+ * https://github.com/JCTools/JCTools
+
+This product optionally depends on 'JZlib', a re-implementation of zlib in
+pure Java, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jzlib.txt (BSD style License)
+ * HOMEPAGE:
+ * http://www.jcraft.com/jzlib/
+
+This product optionally depends on 'Compress-LZF', a Java library for encoding and
+decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/ning/compress
+
+This product optionally depends on 'lz4', a LZ4 Java compression
+and decompression library written by Adrien Grand. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.lz4.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/jpountz/lz4-java
+
+This product optionally depends on 'lzma-java', a LZMA Java compression
+and decompression library, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.lzma-java.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/jponge/lzma-java
+
+This product optionally depends on 'zstd-jni', a zstd-jni Java compression
+and decompression library, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.zstd-jni.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/luben/zstd-jni
+
+This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+and decompression library written by William Kinney. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jfastlz.txt (MIT License)
+ * HOMEPAGE:
+ * https://code.google.com/p/jfastlz/
+
+This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+interchange format, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.protobuf.txt (New BSD License)
+ * HOMEPAGE:
+ * https://github.com/google/protobuf
+
+This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+a temporary self-signed X.509 certificate when the JVM does not provide the
+equivalent functionality. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.bouncycastle.txt (MIT License)
+ * HOMEPAGE:
+ * https://www.bouncycastle.org/
+
+This product optionally depends on 'Snappy', a compression library produced
+by Google Inc, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.snappy.txt (New BSD License)
+ * HOMEPAGE:
+ * https://github.com/google/snappy
+
+This product optionally depends on 'JBoss Marshalling', an alternative Java
+serialization API, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.jboss-marshalling.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/jboss-remoting/jboss-marshalling
+
+This product optionally depends on 'Caliper', Google's micro-
+benchmarking framework, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.caliper.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/google/caliper
+
+This product optionally depends on 'Apache Commons Logging', a logging
+framework, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.commons-logging.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://commons.apache.org/logging/
+
+This product optionally depends on 'Apache Log4J', a logging framework, which
+can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.log4j.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://logging.apache.org/log4j/
+
+This product optionally depends on 'Aalto XML', an ultra-high performance
+non-blocking XML processor, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://wiki.fasterxml.com/AaltoHome
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.hpack.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/twitter/hpack
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Cory Benfield. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.hyper-hpack.txt (MIT License)
+ * HOMEPAGE:
+ * https://github.com/python-hyper/hpack/
+
+This product contains a modified version of 'HPACK', a Java implementation of
+the HTTP/2 HPACK algorithm written by Tatsuhiro Tsujikawa. It can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.nghttp2-hpack.txt (MIT License)
+ * HOMEPAGE:
+ * https://github.com/nghttp2/nghttp2/
+
+This product contains a modified portion of 'Apache Commons Lang', a Java library
+provides utilities for the java.lang API, which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.commons-lang.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://commons.apache.org/proper/commons-lang/
+
+
+This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build.
+
+ * LICENSE:
+ * license/LICENSE.mvn-wrapper.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/takari/maven-wrapper
+
+This product contains the dnsinfo.h header file, that provides a way to retrieve the system DNS configuration on MacOS.
+This private header is also used by Apple's open source
+ mDNSResponder (https://opensource.apple.com/tarballs/mDNSResponder/).
+
+ * LICENSE:
+ * license/LICENSE.dnsinfo.txt (Apple Public Source License 2.0)
+ * HOMEPAGE:
+ * https://www.opensource.apple.com/source/configd/configd-453.19/dnsinfo/dnsinfo.h
+
+This product optionally depends on 'Brotli4j', Brotli compression and
+decompression for Java., which can be obtained at:
+
+ * LICENSE:
+ * license/LICENSE.brotli4j.txt (Apache License 2.0)
+ * HOMEPAGE:
+ * https://github.com/hyperxpro/Brotli4j
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_protocolbuffers_protobuf_v3.16.0_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_protocolbuffers_protobuf_v3.16.0_LICENSE.txt
new file mode 100644
index 0000000..19b305b
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_protocolbuffers_protobuf_v3.16.0_LICENSE.txt
@@ -0,0 +1,32 @@
+Copyright 2008 Google Inc. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+ * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+ * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it. This code is not
+standalone and requires a support library to be linked with it. This
+support library is itself covered by the above license.
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.2_COPYING.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.3_COPYING.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.2_COPYING.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.3_COPYING.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.2_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.3_LICENSE.txt
similarity index 100%
rename from asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.2_LICENSE.txt
rename to asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.3_LICENSE.txt
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_rtyley_spongycastle_sc-v1.54.0.0_LICENSE.html.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_rtyley_spongycastle_sc-v1.54.0.0_LICENSE.html.txt
new file mode 100644
index 0000000..6670b4b
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_rtyley_spongycastle_sc-v1.54.0.0_LICENSE.html.txt
@@ -0,0 +1,17 @@
+Copyright (c) 2000-2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software
+and associated documentation files (the "Software"), to deal in the Software without restriction,
+including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense,
+and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial
+portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED,
+INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
+DEALINGS IN THE SOFTWARE.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.5_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.5_LICENSE.txt
new file mode 100644
index 0000000..d542ab3
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_typetools_checker-framework_checker-framework-2.5.5_LICENSE.txt
@@ -0,0 +1,413 @@
+The Checker Framework
+Copyright 2004-present by the Checker Framework developers
+
+
+Most of the Checker Framework is licensed under the GNU General Public
+License, version 2 (GPL2), with the classpath exception. The text of this
+license appears below. This is the same license used for OpenJDK.
+
+A few parts of the Checker Framework have more permissive licenses, notably
+the parts that you might want to include with your own program.
+
+ * The annotations and utility files are licensed under the MIT License.
+ (The text of this license also appears below.) This applies to
+ checker-qual*.jar and checker-util.jar and all the files that appear in
+ them, which is all files in checker-qual and checker-util directories.
+ It also applies to the cleanroom implementations of
+ third-party annotations (in checker/src/testannotations/,
+ framework/src/main/java/org/jmlspecs/, and
+ framework/src/main/java/com/google/).
+
+The Checker Framework includes annotations for some libraries. Those in
+.astub files use the MIT License. Those in https://github.com/typetools/jdk
+(which appears in the annotated-jdk directory of file checker.jar) use the
+GPL2 license.
+
+Some external libraries that are included with the Checker Framework
+distribution have different licenses. Here are some examples.
+
+ * JavaParser is dual licensed under the LGPL or the Apache license -- you
+ may use it under whichever one you want. (The JavaParser source code
+ contains a file with the text of the GPL, but it is not clear why, since
+ JavaParser does not use the GPL.) See
+ https://github.com/typetools/stubparser .
+
+ * Annotation Tools (https://github.com/typetools/annotation-tools) uses
+ the MIT license.
+
+ * Libraries in plume-lib (https://github.com/plume-lib/) are licensed
+ under the MIT License.
+
+===========================================================================
+
+The GNU General Public License (GPL)
+
+Version 2, June 1991
+
+Copyright (C) 1989, 1991 Free Software Foundation, Inc.
+59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Everyone is permitted to copy and distribute verbatim copies of this license
+document, but changing it is not allowed.
+
+Preamble
+
+The licenses for most software are designed to take away your freedom to share
+and change it. By contrast, the GNU General Public License is intended to
+guarantee your freedom to share and change free software--to make sure the
+software is free for all its users. This General Public License applies to
+most of the Free Software Foundation's software and to any other program whose
+authors commit to using it. (Some other Free Software Foundation software is
+covered by the GNU Library General Public License instead.) You can apply it to
+your programs, too.
+
+When we speak of free software, we are referring to freedom, not price. Our
+General Public Licenses are designed to make sure that you have the freedom to
+distribute copies of free software (and charge for this service if you wish),
+that you receive source code or can get it if you want it, that you can change
+the software or use pieces of it in new free programs; and that you know you
+can do these things.
+
+To protect your rights, we need to make restrictions that forbid anyone to deny
+you these rights or to ask you to surrender the rights. These restrictions
+translate to certain responsibilities for you if you distribute copies of the
+software, or if you modify it.
+
+For example, if you distribute copies of such a program, whether gratis or for
+a fee, you must give the recipients all the rights that you have. You must
+make sure that they, too, receive or can get the source code. And you must
+show them these terms so they know their rights.
+
+We protect your rights with two steps: (1) copyright the software, and (2)
+offer you this license which gives you legal permission to copy, distribute
+and/or modify the software.
+
+Also, for each author's protection and ours, we want to make certain that
+everyone understands that there is no warranty for this free software. If the
+software is modified by someone else and passed on, we want its recipients to
+know that what they have is not the original, so that any problems introduced
+by others will not reflect on the original authors' reputations.
+
+Finally, any free program is threatened constantly by software patents. We
+wish to avoid the danger that redistributors of a free program will
+individually obtain patent licenses, in effect making the program proprietary.
+To prevent this, we have made it clear that any patent must be licensed for
+everyone's free use or not licensed at all.
+
+The precise terms and conditions for copying, distribution and modification
+follow.
+
+TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION
+
+0. This License applies to any program or other work which contains a notice
+placed by the copyright holder saying it may be distributed under the terms of
+this General Public License. The "Program", below, refers to any such program
+or work, and a "work based on the Program" means either the Program or any
+derivative work under copyright law: that is to say, a work containing the
+Program or a portion of it, either verbatim or with modifications and/or
+translated into another language. (Hereinafter, translation is included
+without limitation in the term "modification".) Each licensee is addressed as
+"you".
+
+Activities other than copying, distribution and modification are not covered by
+this License; they are outside its scope. The act of running the Program is
+not restricted, and the output from the Program is covered only if its contents
+constitute a work based on the Program (independent of having been made by
+running the Program). Whether that is true depends on what the Program does.
+
+1. You may copy and distribute verbatim copies of the Program's source code as
+you receive it, in any medium, provided that you conspicuously and
+appropriately publish on each copy an appropriate copyright notice and
+disclaimer of warranty; keep intact all the notices that refer to this License
+and to the absence of any warranty; and give any other recipients of the
+Program a copy of this License along with the Program.
+
+You may charge a fee for the physical act of transferring a copy, and you may
+at your option offer warranty protection in exchange for a fee.
+
+2. You may modify your copy or copies of the Program or any portion of it, thus
+forming a work based on the Program, and copy and distribute such modifications
+or work under the terms of Section 1 above, provided that you also meet all of
+these conditions:
+
+ a) You must cause the modified files to carry prominent notices stating
+ that you changed the files and the date of any change.
+
+ b) You must cause any work that you distribute or publish, that in whole or
+ in part contains or is derived from the Program or any part thereof, to be
+ licensed as a whole at no charge to all third parties under the terms of
+ this License.
+
+ c) If the modified program normally reads commands interactively when run,
+ you must cause it, when started running for such interactive use in the
+ most ordinary way, to print or display an announcement including an
+ appropriate copyright notice and a notice that there is no warranty (or
+ else, saying that you provide a warranty) and that users may redistribute
+ the program under these conditions, and telling the user how to view a copy
+ of this License. (Exception: if the Program itself is interactive but does
+ not normally print such an announcement, your work based on the Program is
+ not required to print an announcement.)
+
+These requirements apply to the modified work as a whole. If identifiable
+sections of that work are not derived from the Program, and can be reasonably
+considered independent and separate works in themselves, then this License, and
+its terms, do not apply to those sections when you distribute them as separate
+works. But when you distribute the same sections as part of a whole which is a
+work based on the Program, the distribution of the whole must be on the terms
+of this License, whose permissions for other licensees extend to the entire
+whole, and thus to each and every part regardless of who wrote it.
+
+Thus, it is not the intent of this section to claim rights or contest your
+rights to work written entirely by you; rather, the intent is to exercise the
+right to control the distribution of derivative or collective works based on
+the Program.
+
+In addition, mere aggregation of another work not based on the Program with the
+Program (or with a work based on the Program) on a volume of a storage or
+distribution medium does not bring the other work under the scope of this
+License.
+
+3. You may copy and distribute the Program (or a work based on it, under
+Section 2) in object code or executable form under the terms of Sections 1 and
+2 above provided that you also do one of the following:
+
+ a) Accompany it with the complete corresponding machine-readable source
+ code, which must be distributed under the terms of Sections 1 and 2 above
+ on a medium customarily used for software interchange; or,
+
+ b) Accompany it with a written offer, valid for at least three years, to
+ give any third party, for a charge no more than your cost of physically
+ performing source distribution, a complete machine-readable copy of the
+ corresponding source code, to be distributed under the terms of Sections 1
+ and 2 above on a medium customarily used for software interchange; or,
+
+ c) Accompany it with the information you received as to the offer to
+ distribute corresponding source code. (This alternative is allowed only
+ for noncommercial distribution and only if you received the program in
+ object code or executable form with such an offer, in accord with
+ Subsection b above.)
+
+The source code for a work means the preferred form of the work for making
+modifications to it. For an executable work, complete source code means all
+the source code for all modules it contains, plus any associated interface
+definition files, plus the scripts used to control compilation and installation
+of the executable. However, as a special exception, the source code
+distributed need not include anything that is normally distributed (in either
+source or binary form) with the major components (compiler, kernel, and so on)
+of the operating system on which the executable runs, unless that component
+itself accompanies the executable.
+
+If distribution of executable or object code is made by offering access to copy
+from a designated place, then offering equivalent access to copy the source
+code from the same place counts as distribution of the source code, even though
+third parties are not compelled to copy the source along with the object code.
+
+4. You may not copy, modify, sublicense, or distribute the Program except as
+expressly provided under this License. Any attempt otherwise to copy, modify,
+sublicense or distribute the Program is void, and will automatically terminate
+your rights under this License. However, parties who have received copies, or
+rights, from you under this License will not have their licenses terminated so
+long as such parties remain in full compliance.
+
+5. You are not required to accept this License, since you have not signed it.
+However, nothing else grants you permission to modify or distribute the Program
+or its derivative works. These actions are prohibited by law if you do not
+accept this License. Therefore, by modifying or distributing the Program (or
+any work based on the Program), you indicate your acceptance of this License to
+do so, and all its terms and conditions for copying, distributing or modifying
+the Program or works based on it.
+
+6. Each time you redistribute the Program (or any work based on the Program),
+the recipient automatically receives a license from the original licensor to
+copy, distribute or modify the Program subject to these terms and conditions.
+You may not impose any further restrictions on the recipients' exercise of the
+rights granted herein. You are not responsible for enforcing compliance by
+third parties to this License.
+
+7. If, as a consequence of a court judgment or allegation of patent
+infringement or for any other reason (not limited to patent issues), conditions
+are imposed on you (whether by court order, agreement or otherwise) that
+contradict the conditions of this License, they do not excuse you from the
+conditions of this License. If you cannot distribute so as to satisfy
+simultaneously your obligations under this License and any other pertinent
+obligations, then as a consequence you may not distribute the Program at all.
+For example, if a patent license would not permit royalty-free redistribution
+of the Program by all those who receive copies directly or indirectly through
+you, then the only way you could satisfy both it and this License would be to
+refrain entirely from distribution of the Program.
+
+If any portion of this section is held invalid or unenforceable under any
+particular circumstance, the balance of the section is intended to apply and
+the section as a whole is intended to apply in other circumstances.
+
+It is not the purpose of this section to induce you to infringe any patents or
+other property right claims or to contest validity of any such claims; this
+section has the sole purpose of protecting the integrity of the free software
+distribution system, which is implemented by public license practices. Many
+people have made generous contributions to the wide range of software
+distributed through that system in reliance on consistent application of that
+system; it is up to the author/donor to decide if he or she is willing to
+distribute software through any other system and a licensee cannot impose that
+choice.
+
+This section is intended to make thoroughly clear what is believed to be a
+consequence of the rest of this License.
+
+8. If the distribution and/or use of the Program is restricted in certain
+countries either by patents or by copyrighted interfaces, the original
+copyright holder who places the Program under this License may add an explicit
+geographical distribution limitation excluding those countries, so that
+distribution is permitted only in or among countries not thus excluded. In
+such case, this License incorporates the limitation as if written in the body
+of this License.
+
+9. The Free Software Foundation may publish revised and/or new versions of the
+General Public License from time to time. Such new versions will be similar in
+spirit to the present version, but may differ in detail to address new problems
+or concerns.
+
+Each version is given a distinguishing version number. If the Program
+specifies a version number of this License which applies to it and "any later
+version", you have the option of following the terms and conditions either of
+that version or of any later version published by the Free Software Foundation.
+If the Program does not specify a version number of this License, you may
+choose any version ever published by the Free Software Foundation.
+
+10. If you wish to incorporate parts of the Program into other free programs
+whose distribution conditions are different, write to the author to ask for
+permission. For software which is copyrighted by the Free Software Foundation,
+write to the Free Software Foundation; we sometimes make exceptions for this.
+Our decision will be guided by the two goals of preserving the free status of
+all derivatives of our free software and of promoting the sharing and reuse of
+software generally.
+
+NO WARRANTY
+
+11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR
+THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE
+STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE
+PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND
+PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE,
+YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION.
+
+12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL
+ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE
+PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY
+GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR
+INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA
+BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A
+FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER
+OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+END OF TERMS AND CONDITIONS
+
+How to Apply These Terms to Your New Programs
+
+If you develop a new program, and you want it to be of the greatest possible
+use to the public, the best way to achieve this is to make it free software
+which everyone can redistribute and change under these terms.
+
+To do so, attach the following notices to the program. It is safest to attach
+them to the start of each source file to most effectively convey the exclusion
+of warranty; and each file should have at least the "copyright" line and a
+pointer to where the full notice is found.
+
+ One line to give the program's name and a brief idea of what it does.
+
+ Copyright (C) <year> <name of author>
+
+ This program is free software; you can redistribute it and/or modify it
+ under the terms of the GNU General Public License as published by the Free
+ Software Foundation; either version 2 of the License, or (at your option)
+ any later version.
+
+ This program is distributed in the hope that it will be useful, but WITHOUT
+ ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for
+ more details.
+
+ You should have received a copy of the GNU General Public License along
+ with this program; if not, write to the Free Software Foundation, Inc., 59
+ Temple Place, Suite 330, Boston, MA 02111-1307 USA
+
+Also add information on how to contact you by electronic and paper mail.
+
+If the program is interactive, make it output a short notice like this when it
+starts in an interactive mode:
+
+ Gnomovision version 69, Copyright (C) year name of author Gnomovision comes
+ with ABSOLUTELY NO WARRANTY; for details type 'show w'. This is free
+ software, and you are welcome to redistribute it under certain conditions;
+ type 'show c' for details.
+
+The hypothetical commands 'show w' and 'show c' should show the appropriate
+parts of the General Public License. Of course, the commands you use may be
+called something other than 'show w' and 'show c'; they could even be
+mouse-clicks or menu items--whatever suits your program.
+
+You should also get your employer (if you work as a programmer) or your school,
+if any, to sign a "copyright disclaimer" for the program, if necessary. Here
+is a sample; alter the names:
+
+ Yoyodyne, Inc., hereby disclaims all copyright interest in the program
+ 'Gnomovision' (which makes passes at compilers) written by James Hacker.
+
+ signature of Ty Coon, 1 April 1989
+
+ Ty Coon, President of Vice
+
+This General Public License does not permit incorporating your program into
+proprietary programs. If your program is a subroutine library, you may
+consider it more useful to permit linking proprietary applications with the
+library. If this is what you want to do, use the GNU Library General Public
+License instead of this License.
+
+
+"CLASSPATH" EXCEPTION TO THE GPL
+
+Certain source files distributed by Oracle America and/or its affiliates are
+subject to the following clarification and special exception to the GPL, but
+only where Oracle has expressly included in the particular source file's header
+the words "Oracle designates this particular file as subject to the "Classpath"
+exception as provided by Oracle in the LICENSE file that accompanied this code."
+
+ Linking this library statically or dynamically with other modules is making
+ a combined work based on this library. Thus, the terms and conditions of
+ the GNU General Public License cover the whole combination.
+
+ As a special exception, the copyright holders of this library give you
+ permission to link this library with independent modules to produce an
+ executable, regardless of the license terms of these independent modules,
+ and to copy and distribute the resulting executable under terms of your
+ choice, provided that you also meet, for each linked independent module,
+ the terms and conditions of the license of that module. An independent
+ module is a module which is not derived from or based on this library. If
+ you modify this library, you may extend this exception to your version of
+ the library, but you are not obligated to do so. If you do not wish to do
+ so, delete this exception statement from your version.
+
+===========================================================================
+
+MIT License:
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+===========================================================================
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/templates/source_licenses.ftl b/asterixdb/src/main/licenses/templates/source_licenses.ftl
index 5cc8d60..97b032b 100644
--- a/asterixdb/src/main/licenses/templates/source_licenses.ftl
+++ b/asterixdb/src/main/licenses/templates/source_licenses.ftl
@@ -131,13 +131,6 @@
SOFTWARE.
</@license>
</#if>
-<#if !hivecompatSkip!false>
- <@license component="AsterixDB runtime" files="org/apache/asterix/hivecompat/io/*"
- licenseName="The Apache License, Version 2.0"
- location="${hivecompatLocation!}" filePrefix="${hivecompatPrefix!}">
-Source files in asterix-hivecompat are derived from portions of Apache Hive Query Language v0.13.0 (org.apache.hive:hive-exec).
- </@license>
-</#if>
<#if !asterixDashboardSkip!false>
<#include "../../../../asterix-dashboard/src/main/licenses/dashboard-source-license.ftl">
</#if>
\ No newline at end of file
diff --git a/hyracks-fullstack/NOTICE b/hyracks-fullstack/NOTICE
index 57c58439..2e33eed 100644
--- a/hyracks-fullstack/NOTICE
+++ b/hyracks-fullstack/NOTICE
@@ -1,5 +1,5 @@
Apache Hyracks and Algebricks
-Copyright 2015-2021 The Apache Software Foundation
+Copyright 2015-2022 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index 3f03cc8..c22d54d 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -66,6 +66,7 @@
protected IConflictingTypeResolver conflictingTypeResolver;
protected IExpressionEvalSizeComputer expressionEvalSizeComputer;
protected IMissingWriterFactory missingWriterFactory;
+ protected IMissingWriterFactory nullWriterFactory;
protected IUnnestingPositionWriterFactory unnestingPositionWriterFactory;
protected INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
protected IPartialAggregationTypeComputer partialAggregationTypeComputer;
@@ -189,6 +190,14 @@
return missingWriterFactory;
}
+ public void setNullWriterFactory(IMissingWriterFactory nullWriterFactory) {
+ this.nullWriterFactory = nullWriterFactory;
+ }
+
+ public IMissingWriterFactory getNullWriterFactory() {
+ return nullWriterFactory;
+ }
+
public void setUnnestingPositionWriterFactory(IUnnestingPositionWriterFactory unnestingPositionWriterFactory) {
this.unnestingPositionWriterFactory = unnestingPositionWriterFactory;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index c6a79a5..891980f 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -101,7 +101,7 @@
JobGenContext context = new JobGenContext(null, metadata, appContext,
serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
- binaryIntegerInspectorFactory, printerProvider, missingWriterFactory,
+ binaryIntegerInspectorFactory, printerProvider, missingWriterFactory, nullWriterFactory,
unnestingPositionWriterFactory, normalizedKeyComputerFactoryProvider,
expressionRuntimeProvider, expressionTypeComputer, oc, expressionEvalSizeComputer,
partialAggregationTypeComputer, predEvaluatorFactoryProvider,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index d590f71..4a900af 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -66,6 +66,7 @@
SINK_WRITE,
SORT_GROUP_BY,
SORT_MERGE_EXCHANGE,
+ SPATIAL_JOIN,
SPLIT,
STABLE_SORT,
STATS,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index 40e729d..e2a68da 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -57,6 +57,16 @@
public String toString() {
return "TRUE";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isTrue();
+ }
+
+ @Override
+ public int hashCode() {
+ return Boolean.TRUE.hashCode();
+ }
});
public static final ConstantExpression FALSE = new ConstantExpression(new IAlgebricksConstantValue() {
@@ -84,6 +94,53 @@
public String toString() {
return "FALSE";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isFalse();
+ }
+
+ @Override
+ public int hashCode() {
+ return Boolean.FALSE.hashCode();
+ }
+ });
+ public static final ConstantExpression NULL = new ConstantExpression(new IAlgebricksConstantValue() {
+
+ @Override
+ public boolean isTrue() {
+ return false;
+ }
+
+ @Override
+ public boolean isMissing() {
+ return false;
+ }
+
+ @Override
+ public boolean isNull() {
+ return true;
+ }
+
+ @Override
+ public boolean isFalse() {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "NULL";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isNull();
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
});
public static final ConstantExpression MISSING = new ConstantExpression(new IAlgebricksConstantValue() {
@@ -111,6 +168,16 @@
public String toString() {
return "MISSING";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isMissing();
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
});
public ConstantExpression(IAlgebricksConstantValue value) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
index 029f41a..c18e072 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
@@ -18,12 +18,18 @@
*/
package org.apache.hyracks.algebricks.core.algebra.expressions;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-
public interface IMissableTypeComputer {
- public Object makeMissableType(Object type) throws AlgebricksException;
+ Object makeMissableType(Object type);
- public boolean canBeMissing(Object type);
+ boolean canBeMissing(Object type);
- public Object getNonOptionalType(Object type);
+ Object makeNullableType(Object type);
+
+ boolean canBeNull(Object type);
+
+ Object getNonOptionalType(Object type);
+
+ Object getNonMissableType(Object type);
+
+ Object getNonNullableType(Object type);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
index e024147..d160e54 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
@@ -27,7 +27,8 @@
public interface IVariableTypeEnvironment {
public Object getVarType(LogicalVariable var) throws AlgebricksException;
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException;
public void setVarType(LogicalVariable var, Object type);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 8540d0b..77fdd74 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -219,7 +219,7 @@
IDataSourceIndex<I, S> dataSourceIndex, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalFilteringKeys, ILogicalExpression filterExpr,
- LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
+ ILogicalExpression prevFilterExpr, LogicalVariable operationVar, List<LogicalVariable> prevSecondaryKeys,
LogicalVariable prevAdditionalFilteringKeys, RecordDescriptor inputDesc, JobGenContext context,
JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java
index 9de591e..3c1a24d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java
@@ -18,17 +18,15 @@
*/
package org.apache.hyracks.algebricks.core.algebra.metadata;
-import java.util.List;
-
/**
* Generic interface to include the projection information for
* {@link org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator}
*/
public interface IProjectionInfo<T> {
/**
- * @return list of projected values' information
+ * @return projected values' information
*/
- List<T> getProjectionInfo();
+ T getProjectionInfo();
/**
* @return a copy of the {@link IProjectionInfo}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
index 1af0ce4..1179dc4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
@@ -84,7 +84,9 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
index 7b6ed26..0098817 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
@@ -45,7 +45,7 @@
* perform. In the case of bulk-loading, {@link #operation} will be INSERT and the {@link #bulkload} flag will be
* raised. {@link #additionalFilteringExpressions} and {@link #numberOfAdditionalNonFilteringFields} refers to the
* additionalFilteringExpressions, numberOfAdditionalNonFilteringFields found in the corresponding primary index
- * {@link InsertDeleteUpsertOperator} (i.e. to specify LSM filters). {@link #upsertIndicatorExpr} also originates from
+ * {@link InsertDeleteUpsertOperator} (i.e. to specify LSM filters). {@link #operationExpr} also originates from
* {@link InsertDeleteUpsertOperator}, and is only set when the operation is of kind UPSERT.
* <p>
*
@@ -78,23 +78,25 @@
// Otherwise, it contains secondary key information.
private List<Mutable<ILogicalExpression>> secondaryKeyExprs;
private Mutable<ILogicalExpression> filterExpr;
+ private Mutable<ILogicalExpression> beforeOpFilterExpr;
private final Kind operation;
private final boolean bulkload;
private List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
// used for upsert operations
private List<Mutable<ILogicalExpression>> prevSecondaryKeyExprs;
private Mutable<ILogicalExpression> prevAdditionalFilteringExpression;
- private Mutable<ILogicalExpression> upsertIndicatorExpr;
+ private Mutable<ILogicalExpression> operationExpr;
private final int numberOfAdditionalNonFilteringFields;
public IndexInsertDeleteUpsertOperator(IDataSourceIndex<?, ?> dataSourceIndex,
List<Mutable<ILogicalExpression>> primaryKeyExprs, List<Mutable<ILogicalExpression>> secondaryKeyExprs,
- Mutable<ILogicalExpression> filterExpr, Kind operation, boolean bulkload,
- int numberOfAdditionalNonFilteringFields) {
+ Mutable<ILogicalExpression> filterExpr, Mutable<ILogicalExpression> beforeOpFilterExpr, Kind operation,
+ boolean bulkload, int numberOfAdditionalNonFilteringFields) {
this.dataSourceIndex = dataSourceIndex;
this.primaryKeyExprs = primaryKeyExprs;
this.secondaryKeyExprs = secondaryKeyExprs;
this.filterExpr = filterExpr;
+ this.beforeOpFilterExpr = beforeOpFilterExpr;
this.operation = operation;
this.bulkload = bulkload;
this.numberOfAdditionalNonFilteringFields = numberOfAdditionalNonFilteringFields;
@@ -121,6 +123,12 @@
b = true;
}
}
+ // Old Filtering <For upsert>
+ if (beforeOpFilterExpr != null) {
+ if (visitor.transform(beforeOpFilterExpr)) {
+ b = true;
+ }
+ }
// Additional Filtering <For upsert>
if (additionalFilteringExpressions != null) {
for (int i = 0; i < additionalFilteringExpressions.size(); i++) {
@@ -129,8 +137,8 @@
}
}
}
- // Upsert indicator var <For upsert>
- if (upsertIndicatorExpr != null && visitor.transform(upsertIndicatorExpr)) {
+ // Operation indicator var <For upsert>
+ if (operationExpr != null && visitor.transform(operationExpr)) {
b = true;
}
// Old secondary <For upsert>
@@ -141,7 +149,7 @@
}
}
}
- // Old Filtering <For upsert>
+ // Old Additional Filtering <For upsert>
if (prevAdditionalFilteringExpression != null) {
visitor.transform(prevAdditionalFilteringExpression);
}
@@ -164,6 +172,9 @@
if (getFilterExpression() != null) {
getFilterExpression().getValue().getUsedVariables(vars);
}
+ if (getBeforeOpFilterExpression() != null) {
+ getBeforeOpFilterExpression().getValue().getUsedVariables(vars);
+ }
if (getAdditionalFilteringExpressions() != null) {
for (Mutable<ILogicalExpression> e : getAdditionalFilteringExpressions()) {
e.getValue().getUsedVariables(vars);
@@ -177,8 +188,8 @@
e.getValue().getUsedVariables(vars);
}
}
- if (getUpsertIndicatorExpr() != null) {
- getUpsertIndicatorExpr().getValue().getUsedVariables(vars);
+ if (getOperationExpr() != null) {
+ getOperationExpr().getValue().getUsedVariables(vars);
}
}
@@ -232,10 +243,18 @@
return filterExpr;
}
+ public Mutable<ILogicalExpression> getBeforeOpFilterExpression() {
+ return beforeOpFilterExpr;
+ }
+
public void setFilterExpression(Mutable<ILogicalExpression> filterExpr) {
this.filterExpr = filterExpr;
}
+ public void setBeforeOpFilterExpression(Mutable<ILogicalExpression> beforeOpFilterExpr) {
+ this.beforeOpFilterExpr = beforeOpFilterExpr;
+ }
+
public Kind getOperation() {
return operation;
}
@@ -273,11 +292,11 @@
return numberOfAdditionalNonFilteringFields;
}
- public Mutable<ILogicalExpression> getUpsertIndicatorExpr() {
- return upsertIndicatorExpr;
+ public Mutable<ILogicalExpression> getOperationExpr() {
+ return operationExpr;
}
- public void setUpsertIndicatorExpr(Mutable<ILogicalExpression> upsertIndicatorExpr) {
- this.upsertIndicatorExpr = upsertIndicatorExpr;
+ public void setOperationExpr(Mutable<ILogicalExpression> operationExpr) {
+ this.operationExpr = operationExpr;
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
index ce2f801..5a54c7e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
@@ -59,9 +59,9 @@
// previous additional fields (for UPSERT)
private List<LogicalVariable> prevAdditionalNonFilteringVars;
private List<Object> prevAdditionalNonFilteringTypes;
- // a boolean variable that indicates whether it's a delete operation (false) or upsert operation (true)
- private LogicalVariable upsertIndicatorVar;
- private Object upsertIndicatorVarType;
+ // int describing the upsert (e.g. upserting a new tuple or to an existing tuple or just deleting an existing one)
+ private LogicalVariable operationVar;
+ private Object operationVarType;
public InsertDeleteUpsertOperator(IDataSource<?> dataSource, Mutable<ILogicalExpression> payloadExpr,
List<Mutable<ILogicalExpression>> primaryKeyExprs,
@@ -88,7 +88,7 @@
public void recomputeSchema() throws AlgebricksException {
schema = new ArrayList<LogicalVariable>();
if (operation == Kind.UPSERT) {
- schema.add(upsertIndicatorVar);
+ schema.add(operationVar);
// The upsert case also produces the previous record
schema.add(prevRecordVar);
if (additionalNonFilteringExpressions != null) {
@@ -103,7 +103,7 @@
public void getProducedVariables(Collection<LogicalVariable> producedVariables) {
if (operation == Kind.UPSERT) {
- producedVariables.add(upsertIndicatorVar);
+ producedVariables.add(operationVar);
producedVariables.add(prevRecordVar);
if (prevAdditionalNonFilteringVars != null) {
producedVariables.addAll(prevAdditionalNonFilteringVars);
@@ -150,7 +150,7 @@
@Override
public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
if (operation == Kind.UPSERT) {
- target.addVariable(upsertIndicatorVar);
+ target.addVariable(operationVar);
target.addVariable(prevRecordVar);
if (prevAdditionalNonFilteringVars != null) {
for (LogicalVariable var : prevAdditionalNonFilteringVars) {
@@ -175,7 +175,7 @@
public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
if (operation == Kind.UPSERT) {
- env.setVarType(upsertIndicatorVar, upsertIndicatorVarType);
+ env.setVarType(operationVar, operationVarType);
env.setVarType(prevRecordVar, prevRecordType);
if (prevAdditionalNonFilteringVars != null) {
for (int i = 0; i < prevAdditionalNonFilteringVars.size(); i++) {
@@ -229,20 +229,20 @@
this.prevRecordVar = prevRecordVar;
}
- public LogicalVariable getUpsertIndicatorVar() {
- return upsertIndicatorVar;
+ public LogicalVariable getOperationVar() {
+ return operationVar;
}
- public void setUpsertIndicatorVar(LogicalVariable upsertIndicatorVar) {
- this.upsertIndicatorVar = upsertIndicatorVar;
+ public void setOperationVar(LogicalVariable operationVar) {
+ this.operationVar = operationVar;
}
- public Object getUpsertIndicatorVarType() {
- return upsertIndicatorVarType;
+ public Object getOperationVarType() {
+ return operationVarType;
}
- public void setUpsertIndicatorVarType(Object upsertIndicatorVarType) {
- this.upsertIndicatorVarType = upsertIndicatorVarType;
+ public void setOperationVarType(Object operationVarType) {
+ this.operationVarType = operationVarType;
}
public void setPrevRecordType(Object recordType) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
index 4e382d2..d709184 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
@@ -27,24 +27,40 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.properties.LeftOuterTypePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
public class LeftOuterJoinOperator extends AbstractBinaryJoinOperator {
- public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition) {
+ private IAlgebricksConstantValue missingValue;
+
+ public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition, IAlgebricksConstantValue missingValue) {
super(JoinKind.LEFT_OUTER, condition);
+ setMissingValue(missingValue);
}
public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition, Mutable<ILogicalOperator> input1,
- Mutable<ILogicalOperator> input2) {
+ Mutable<ILogicalOperator> input2, IAlgebricksConstantValue missingValue) {
super(JoinKind.LEFT_OUTER, condition, input1, input2);
+ setMissingValue(missingValue);
+ }
+
+ public IAlgebricksConstantValue getMissingValue() {
+ return missingValue;
+ }
+
+ public void setMissingValue(IAlgebricksConstantValue value) {
+ this.missingValue = validateMissingValue(value);
}
@Override
@@ -64,13 +80,40 @@
for (int i = 0; i < n; i++) {
envPointers[i] = new OpRefTypeEnvPointer(inputs.get(i), ctx);
}
- PropagatingTypeEnvironment env =
- new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMissableTypeComputer(),
- ctx.getMetadataProvider(), TypePropagationPolicy.LEFT_OUTER, envPointers);
+
+ TypePropagationPolicy typePropagationPolicy;
+ if (missingValue.isMissing()) {
+ typePropagationPolicy = LeftOuterTypePropagationPolicy.MISSABLE;
+ } else if (missingValue.isNull()) {
+ typePropagationPolicy = LeftOuterTypePropagationPolicy.NULLABLE;
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, String.valueOf(missingValue));
+ }
+ PropagatingTypeEnvironment env = new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
+ ctx.getMissableTypeComputer(), ctx.getMetadataProvider(), typePropagationPolicy, envPointers);
+
+ // live variables from right branch can be MISSING (or NULL) together
List<LogicalVariable> liveVars = new ArrayList<>();
- VariableUtilities.getLiveVariables(inputs.get(1).getValue(), liveVars); // live variables from right branch can be MISSING together
- env.getCorrelatedMissableVariableLists().add(liveVars);
+ VariableUtilities.getLiveVariables(inputs.get(1).getValue(), liveVars);
+ if (missingValue.isMissing()) {
+ env.getCorrelatedMissableVariableLists().add(liveVars);
+ } else if (missingValue.isNull()) {
+ env.getCorrelatedNullableVariableLists().add(liveVars);
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, String.valueOf(missingValue));
+ }
return env;
}
+ private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+ if (value == null) {
+ throw new NullPointerException();
+ } else if (value.isMissing()) {
+ return ConstantExpression.MISSING.getValue();
+ } else if (value.isNull()) {
+ return ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(String.valueOf(value));
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
index cd009c0..d91cf72 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
@@ -25,10 +25,13 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
/**
* Left-outer-unnest-map is similar to the unnest-map operator. The only
@@ -38,11 +41,21 @@
*/
public class LeftOuterUnnestMapOperator extends AbstractUnnestMapOperator {
+ private IAlgebricksConstantValue missingValue;
+
public LeftOuterUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
- List<Object> variableTypes, boolean propagateInput) {
- super(variables, expression, variableTypes, propagateInput);
+ List<Object> variableTypes, IAlgebricksConstantValue missingValue) {
// propagateInput is always set to true for this operator.
- this.propagateInput = true;
+ super(variables, expression, variableTypes, true);
+ setMissingValue(missingValue);
+ }
+
+ public IAlgebricksConstantValue getMissingValue() {
+ return missingValue;
+ }
+
+ public void setMissingValue(IAlgebricksConstantValue missingValue) {
+ this.missingValue = validateMissingValue(missingValue);
}
@Override
@@ -60,12 +73,32 @@
// Propagates all input variables that come from the outer branch.
PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
- // The produced variables of the this operator are missable because of the left outer semantics.
+ // The produced variables of the this operator are missable (or nullable) because of the left outer semantics.
for (int i = 0; i < variables.size(); i++) {
- env.setVarType(variables.get(i), ctx.getMissableTypeComputer().makeMissableType(variableTypes.get(i)));
+ Object varType = variableTypes.get(i);
+ Object outVarType;
+ if (missingValue.isMissing()) {
+ outVarType = ctx.getMissableTypeComputer().makeMissableType(varType);
+ } else if (missingValue.isNull()) {
+ outVarType = ctx.getMissableTypeComputer().makeNullableType(varType);
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, getSourceLocation(), "");
+ }
+ env.setVarType(variables.get(i), outVarType);
}
return env;
}
+ private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+ if (value == null) {
+ throw new NullPointerException();
+ } else if (value.isMissing()) {
+ return ConstantExpression.MISSING.getValue();
+ } else if (value.isNull()) {
+ return ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(String.valueOf(value));
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
index 14996dd..1e493ad 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
@@ -19,20 +19,37 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
public class LeftOuterUnnestOperator extends AbstractUnnestNonMapOperator {
- public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression) {
+ private IAlgebricksConstantValue missingValue;
+
+ public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression,
+ IAlgebricksConstantValue missingValue) {
super(variable, expression);
+ setMissingValue(missingValue);
}
public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression,
- LogicalVariable positionalVariable, Object positionalVariableType) {
+ LogicalVariable positionalVariable, Object positionalVariableType, IAlgebricksConstantValue missingValue) {
super(variable, expression, positionalVariable, positionalVariableType);
+ setMissingValue(missingValue);
+ }
+
+ public IAlgebricksConstantValue getMissingValue() {
+ return missingValue;
+ }
+
+ public void setMissingValue(IAlgebricksConstantValue missingValue) {
+ this.missingValue = validateMissingValue(missingValue);
}
@Override
@@ -45,10 +62,26 @@
PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
// The produced variables of the this operator are missable because of the left outer semantics.
+ Object outVarType, outPositionalVarType = null;
Object t = env.getType(expression.getValue());
- env.setVarType(variables.get(0), ctx.getMissableTypeComputer().makeMissableType(t));
+ IMissableTypeComputer tc = ctx.getMissableTypeComputer();
+ if (missingValue.isMissing()) {
+ outVarType = tc.makeMissableType(t);
+ if (positionalVariable != null) {
+ outPositionalVarType = tc.makeMissableType(positionalVariableType);
+ }
+ } else if (missingValue.isNull()) {
+ outVarType = tc.makeNullableType(t);
+ if (positionalVariable != null) {
+ outPositionalVarType = tc.makeNullableType(positionalVariableType);
+ }
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, getSourceLocation(), "");
+ }
+
+ env.setVarType(variables.get(0), outVarType);
if (positionalVariable != null) {
- env.setVarType(positionalVariable, ctx.getMissableTypeComputer().makeMissableType(positionalVariableType));
+ env.setVarType(positionalVariable, outPositionalVarType);
}
return env;
@@ -58,4 +91,16 @@
public LogicalOperatorTag getOperatorTag() {
return LogicalOperatorTag.LEFT_OUTER_UNNEST;
}
+
+ private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+ if (value == null) {
+ throw new NullPointerException();
+ } else if (value.isMissing()) {
+ return ConstantExpression.MISSING.getValue();
+ } else if (value.isNull()) {
+ return ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(String.valueOf(value));
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
index b2e2dfd..0b2c32d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.algebricks.core.algebra.operators.logical;
import java.util.ArrayList;
+import java.util.List;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -27,9 +28,12 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
@@ -41,13 +45,28 @@
public class SelectOperator extends AbstractLogicalOperator {
private final Mutable<ILogicalExpression> condition;
- private final boolean retainMissing;
+ private final IAlgebricksConstantValue retainMissingAsValue;
private final LogicalVariable nullPlaceholderVar;
- public SelectOperator(Mutable<ILogicalExpression> condition, boolean retainMissing,
+ public SelectOperator(Mutable<ILogicalExpression> condition) {
+ this(condition, null, null);
+ }
+
+ public SelectOperator(Mutable<ILogicalExpression> condition, IAlgebricksConstantValue retainMissingAsValue,
LogicalVariable nullPlaceholderVar) {
this.condition = condition;
- this.retainMissing = retainMissing;
+ if (retainMissingAsValue == null) {
+ this.retainMissingAsValue = null;
+ if (nullPlaceholderVar != null) {
+ throw new IllegalArgumentException(nullPlaceholderVar.toString());
+ }
+ } else if (retainMissingAsValue.isMissing()) {
+ this.retainMissingAsValue = ConstantExpression.MISSING.getValue();
+ } else if (retainMissingAsValue.isNull()) {
+ this.retainMissingAsValue = ConstantExpression.NULL.getValue();
+ } else {
+ throw new IllegalArgumentException(retainMissingAsValue.toString());
+ }
this.nullPlaceholderVar = nullPlaceholderVar;
}
@@ -60,11 +79,11 @@
return condition;
}
- public boolean getRetainMissing() {
- return retainMissing;
+ public IAlgebricksConstantValue getRetainMissingAsValue() {
+ return retainMissingAsValue;
}
- public LogicalVariable getMissingPlaceholderVariable() throws AlgebricksException {
+ public LogicalVariable getMissingPlaceholderVariable() {
return nullPlaceholderVar;
}
@@ -109,12 +128,11 @@
ILogicalExpression a1 = f1.getArguments().get(0).getValue();
if (a1.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
- if (f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
- ILogicalExpression a2 = f2.getArguments().get(0).getValue();
- if (a2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
- LogicalVariable var = ((VariableReferenceExpression) a2).getVariableReference();
- env.getNonMissableVariables().add(var);
- }
+ FunctionIdentifier f2id = f2.getFunctionIdentifier();
+ if (f2id.equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+ extractFunctionArgVarInto(f2, env.getNonMissableVariables());
+ } else if (f2id.equals(AlgebricksBuiltinFunctions.IS_NULL)) {
+ extractFunctionArgVarInto(f2, env.getNonNullableVariables());
}
}
return env;
@@ -124,4 +142,13 @@
public boolean requiresVariableReferenceExpressions() {
return false;
}
+
+ private static void extractFunctionArgVarInto(AbstractFunctionCallExpression callExpr,
+ List<? super LogicalVariable> outList) {
+ ILogicalExpression arg = callExpr.getArguments().get(0).getValue();
+ if (arg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ LogicalVariable var = ((VariableReferenceExpression) arg).getVariableReference();
+ outList.add(var);
+ }
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index a9f9626..e5d0241 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -225,6 +225,9 @@
return Boolean.FALSE;
}
LeftOuterJoinOperator joinOpArg = (LeftOuterJoinOperator) copyAndSubstituteVar(op, arg);
+ if (!op.getMissingValue().equals(joinOpArg.getMissingValue())) {
+ return Boolean.FALSE;
+ }
boolean isomorphic = op.getCondition().getValue().equals(joinOpArg.getCondition().getValue());
return isomorphic;
}
@@ -451,6 +454,10 @@
if (!isomorphic) {
return Boolean.FALSE;
}
+ isomorphic = op.getMissingValue().equals(loUnnestOpArg.getMissingValue());
+ if (!isomorphic) {
+ return Boolean.FALSE;
+ }
isomorphic = op.getExpressionRef().getValue().equals(loUnnestOpArg.getExpressionRef().getValue());
return isomorphic;
}
@@ -610,6 +617,7 @@
|| !Objects.equals(op.getPrimaryKeyExpressions(), insertOpArg.getPrimaryKeyExpressions())
|| !Objects.equals(op.getSecondaryKeyExpressions(), insertOpArg.getSecondaryKeyExpressions())
|| !Objects.equals(op.getFilterExpression(), insertOpArg.getFilterExpression())
+ || !Objects.equals(op.getBeforeOpFilterExpression(), insertOpArg.getBeforeOpFilterExpression())
|| !Objects.equals(op.getOperation(), insertOpArg.getOperation())
|| (op.isBulkload() != insertOpArg.isBulkload())
|| !Objects.equals(op.getAdditionalFilteringExpressions(),
@@ -617,7 +625,7 @@
|| !Objects.equals(op.getPrevSecondaryKeyExprs(), insertOpArg.getPrevSecondaryKeyExprs())
|| !Objects.equals(op.getPrevAdditionalFilteringExpression(),
insertOpArg.getPrevAdditionalFilteringExpression())
- || !Objects.equals(op.getUpsertIndicatorExpr(), insertOpArg.getUpsertIndicatorExpr())
+ || !Objects.equals(op.getOperationExpr(), insertOpArg.getOperationExpr())
|| (op.getNumberOfAdditionalNonFilteringFields() != insertOpArg
.getNumberOfAdditionalNonFilteringFields())) {
return Boolean.FALSE;
@@ -817,6 +825,10 @@
if (!isomorphic) {
return Boolean.FALSE;
}
+ isomorphic = op.getMissingValue().equals(unnestOpArg.getMissingValue());
+ if (!isomorphic) {
+ return Boolean.FALSE;
+ }
isomorphic = op.getExpressionRef().getValue().equals(unnestOpArg.getExpressionRef().getValue());
return isomorphic;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index e05d12a..199c2e1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -389,7 +389,7 @@
LeftOuterJoinOperator opCopy =
new LeftOuterJoinOperator(exprDeepCopyVisitor.deepCopyExpressionReference(op.getCondition()),
deepCopyOperatorReference(op.getInputs().get(0), arg),
- deepCopyOperatorReference(op.getInputs().get(1), arg));
+ deepCopyOperatorReference(op.getInputs().get(1), arg), op.getMissingValue());
copyAnnotations(op, opCopy);
copySourceLocation(op, opCopy);
opCopy.setExecutionMode(op.getExecutionMode());
@@ -474,7 +474,7 @@
@Override
public ILogicalOperator visitSelectOperator(SelectOperator op, ILogicalOperator arg) throws AlgebricksException {
SelectOperator opCopy = new SelectOperator(exprDeepCopyVisitor.deepCopyExpressionReference(op.getCondition()),
- op.getRetainMissing(), deepCopyVariable(op.getMissingPlaceholderVariable()));
+ op.getRetainMissingAsValue(), deepCopyVariable(op.getMissingPlaceholderVariable()));
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
@@ -544,7 +544,7 @@
throws AlgebricksException {
LeftOuterUnnestMapOperator opCopy = new LeftOuterUnnestMapOperator(deepCopyVariableList(op.getVariables()),
exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()), op.getVariableTypes(),
- op.propagatesInput());
+ op.getMissingValue());
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
@@ -590,7 +590,7 @@
throws AlgebricksException {
LeftOuterUnnestOperator opCopy = new LeftOuterUnnestOperator(deepCopyVariable(op.getVariable()),
exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()),
- deepCopyVariable(op.getPositionalVariable()), op.getPositionalVariableType());
+ deepCopyVariable(op.getPositionalVariable()), op.getPositionalVariableType(), op.getMissingValue());
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index c06ad5c..7b67af1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -133,7 +133,7 @@
@Override
public ILogicalOperator visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
return new LeftOuterJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0),
- op.getInputs().get(1));
+ op.getInputs().get(1), op.getMissingValue());
}
@Override
@@ -158,7 +158,7 @@
@Override
public ILogicalOperator visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
- return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainMissing(),
+ return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainMissingAsValue(),
op.getMissingPlaceholderVariable());
}
@@ -247,7 +247,7 @@
ArrayList<LogicalVariable> newInputList = new ArrayList<>();
newInputList.addAll(op.getVariables());
return new LeftOuterUnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
- new ArrayList<>(op.getVariableTypes()), op.propagatesInput());
+ new ArrayList<>(op.getVariableTypes()), op.getMissingValue());
}
@Override
@@ -324,11 +324,14 @@
deepCopyExpressionRefs(newSecondaryKeyExpressions, op.getSecondaryKeyExpressions());
Mutable<ILogicalExpression> newFilterExpression =
new MutableObject<>(((AbstractLogicalExpression) op.getFilterExpression()).cloneExpression());
+ Mutable<ILogicalExpression> newBeforeOpFilterExpression =
+ new MutableObject<>(((AbstractLogicalExpression) op.getBeforeOpFilterExpression()).cloneExpression());
List<Mutable<ILogicalExpression>> newLSMComponentFilterExpressions = new ArrayList<>();
deepCopyExpressionRefs(newLSMComponentFilterExpressions, op.getAdditionalFilteringExpressions());
- IndexInsertDeleteUpsertOperator indexInsertDeleteOp = new IndexInsertDeleteUpsertOperator(
- op.getDataSourceIndex(), newPrimaryKeyExpressions, newSecondaryKeyExpressions, newFilterExpression,
- op.getOperation(), op.isBulkload(), op.getNumberOfAdditionalNonFilteringFields());
+ IndexInsertDeleteUpsertOperator indexInsertDeleteOp =
+ new IndexInsertDeleteUpsertOperator(op.getDataSourceIndex(), newPrimaryKeyExpressions,
+ newSecondaryKeyExpressions, newFilterExpression, newBeforeOpFilterExpression, op.getOperation(),
+ op.isBulkload(), op.getNumberOfAdditionalNonFilteringFields());
indexInsertDeleteOp.setAdditionalFilteringExpressions(newLSMComponentFilterExpressions);
for (ILogicalPlan plan : op.getNestedPlans()) {
indexInsertDeleteOp.getNestedPlans().add(OperatorManipulationUtil.deepCopy(plan, indexInsertDeleteOp));
@@ -417,7 +420,7 @@
public ILogicalOperator visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void arg)
throws AlgebricksException {
return new LeftOuterUnnestOperator(op.getVariable(), deepCopyExpressionRef(op.getExpressionRef()),
- op.getPositionalVariable(), op.getPositionalVariableType());
+ op.getPositionalVariable(), op.getPositionalVariableType(), op.getMissingValue());
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 6e8b425..439e493 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -421,8 +421,8 @@
Pair<LogicalVariable, LogicalVariable> pair) throws AlgebricksException {
boolean producedVarFound = false;
if (op.getOperation() == InsertDeleteUpsertOperator.Kind.UPSERT) {
- if (op.getUpsertIndicatorVar() != null && op.getUpsertIndicatorVar().equals(pair.first)) {
- op.setUpsertIndicatorVar(pair.second);
+ if (op.getOperationVar() != null && op.getOperationVar().equals(pair.first)) {
+ op.setOperationVar(pair.second);
producedVarFound = true;
} else if (op.getBeforeOpRecordVar() != null && op.getBeforeOpRecordVar().equals(pair.first)) {
op.setPrevRecordVar(pair.second);
@@ -452,8 +452,9 @@
substUsedVariablesInExpr(op.getPrimaryKeyExpressions(), pair.first, pair.second);
substUsedVariablesInExpr(op.getSecondaryKeyExpressions(), pair.first, pair.second);
substUsedVariablesInExpr(op.getFilterExpression(), pair.first, pair.second);
+ substUsedVariablesInExpr(op.getBeforeOpFilterExpression(), pair.first, pair.second);
substUsedVariablesInExpr(op.getAdditionalFilteringExpressions(), pair.first, pair.second);
- substUsedVariablesInExpr(op.getUpsertIndicatorExpr(), pair.first, pair.second);
+ substUsedVariablesInExpr(op.getOperationExpr(), pair.first, pair.second);
substUsedVariablesInExpr(op.getPrevSecondaryKeyExprs(), pair.first, pair.second);
substUsedVariablesInExpr(op.getPrevAdditionalFilteringExpression(), pair.first, pair.second);
if (!op.getNestedPlans().isEmpty()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 23fe3b2..174b184 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -416,6 +416,9 @@
if (op.getFilterExpression() != null) {
op.getFilterExpression().getValue().getUsedVariables(usedVariables);
}
+ if (op.getBeforeOpFilterExpression() != null) {
+ op.getBeforeOpFilterExpression().getValue().getUsedVariables(usedVariables);
+ }
if (op.getAdditionalFilteringExpressions() != null) {
for (Mutable<ILogicalExpression> e : op.getAdditionalFilteringExpressions()) {
e.getValue().getUsedVariables(usedVariables);
@@ -429,8 +432,8 @@
e.getValue().getUsedVariables(usedVariables);
}
}
- if (op.getUpsertIndicatorExpr() != null) {
- op.getUpsertIndicatorExpr().getValue().getUsedVariables(usedVariables);
+ if (op.getOperationExpr() != null) {
+ op.getOperationExpr().getValue().getUsedVariables(usedVariables);
}
visitNestedPlans(op);
return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
index 059a357..69db58d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
@@ -30,8 +30,11 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -179,4 +182,10 @@
return varSet.equals(varArgSet);
}
+ public static LogicalVariable getVariable(ILogicalExpression expr) {
+ if (expr != null && expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+ return ((VariableReferenceExpression) expr).getVariableReference();
+ }
+ return null;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
index 90e88dc..45360d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
@@ -31,11 +31,13 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestNonMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.util.LogRedactionUtil;
@@ -84,8 +86,11 @@
int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
IUnnestingPositionWriterFactory positionWriterFactory =
unnest.hasPositionalVariable() ? context.getUnnestingPositionWriterFactory() : null;
+ IMissingWriterFactory missingWriterFactory = leftOuter
+ ? JobGenHelper.getMissingWriterFactory(context, ((LeftOuterUnnestOperator) op).getMissingValue())
+ : null;
UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList,
- positionWriterFactory, leftOuter, context.getMissingWriterFactory());
+ positionWriterFactory, leftOuter, missingWriterFactory);
unnestRuntime.setSourceLocation(unnest.getSourceLocation());
RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index c7c7588..cc89deb 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
@@ -108,10 +109,11 @@
IBinaryHashFunctionFamily[] rightHashFunFamilies =
JobGenHelper.variablesToBinaryHashFunctionFamilies(keysRightBranch, env, context);
- IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider =
- context.getPredicateEvaluatorFactoryProvider();
- IPredicateEvaluatorFactory predEvaluatorFactory = predEvaluatorFactoryProvider == null ? null
- : predEvaluatorFactoryProvider.getPredicateEvaluatorFactory(keysLeft, keysRight);
+ IPredicateEvaluatorFactoryProvider predEvalFactoryProvider = context.getPredicateEvaluatorFactoryProvider();
+ IPredicateEvaluatorFactory leftPredEvalFactory =
+ predEvalFactoryProvider == null ? null : predEvalFactoryProvider.getPredicateEvaluatorFactory(keysLeft);
+ IPredicateEvaluatorFactory rightPredEvalFactory = predEvalFactoryProvider == null ? null
+ : predEvalFactoryProvider.getPredicateEvaluatorFactory(keysRight);
RecordDescriptor recDescriptor =
JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
@@ -128,9 +130,9 @@
IOperatorDescriptorRegistry spec = builder.getJobSpec();
IOperatorDescriptor opDesc;
- opDesc = generateOptimizedHashJoinRuntime(context, inputSchemas, keysLeft, keysRight, leftHashFunFamilies,
- rightHashFunFamilies, comparatorFactory, reverseComparatorFactory, predEvaluatorFactory, recDescriptor,
- spec);
+ opDesc = generateOptimizedHashJoinRuntime(context, joinOp, inputSchemas, keysLeft, keysRight,
+ leftHashFunFamilies, rightHashFunFamilies, comparatorFactory, reverseComparatorFactory,
+ leftPredEvalFactory, rightPredEvalFactory, recDescriptor, spec);
opDesc.setSourceLocation(op.getSourceLocation());
contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
@@ -140,25 +142,24 @@
builder.contributeGraphEdge(src2, 0, op, 1);
}
- private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context, IOperatorSchema[] inputSchemas,
- int[] keysLeft, int[] keysRight, IBinaryHashFunctionFamily[] leftHashFunFamilies,
- IBinaryHashFunctionFamily[] rightHashFunFamilies, ITuplePairComparatorFactory comparatorFactory,
- ITuplePairComparatorFactory reverseComparatorFactory, IPredicateEvaluatorFactory predEvaluatorFactory,
- RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) {
+ private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context,
+ AbstractBinaryJoinOperator joinOp, IOperatorSchema[] inputSchemas, int[] keysLeft, int[] keysRight,
+ IBinaryHashFunctionFamily[] leftHashFunFamilies, IBinaryHashFunctionFamily[] rightHashFunFamilies,
+ ITuplePairComparatorFactory comparatorFactory, ITuplePairComparatorFactory reverseComparatorFactory,
+ IPredicateEvaluatorFactory leftPredEvalFactory, IPredicateEvaluatorFactory rightPredEvalFactory,
+ RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) throws AlgebricksException {
int memSizeInFrames = localMemoryRequirements.getMemoryBudgetInFrames();
switch (kind) {
case INNER:
return new OptimizedHybridHashJoinOperatorDescriptor(spec, memSizeInFrames, maxInputBuildSizeInFrames,
getFudgeFactor(), keysLeft, keysRight, leftHashFunFamilies, rightHashFunFamilies, recDescriptor,
- comparatorFactory, reverseComparatorFactory, predEvaluatorFactory);
+ comparatorFactory, reverseComparatorFactory, leftPredEvalFactory, rightPredEvalFactory);
case LEFT_OUTER:
- IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
- for (int j = 0; j < nonMatchWriterFactories.length; j++) {
- nonMatchWriterFactories[j] = context.getMissingWriterFactory();
- }
+ IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+ ((LeftOuterJoinOperator) joinOp).getMissingValue(), inputSchemas[1].getSize());
return new OptimizedHybridHashJoinOperatorDescriptor(spec, memSizeInFrames, maxInputBuildSizeInFrames,
getFudgeFactor(), keysLeft, keysRight, leftHashFunFamilies, rightHashFunFamilies, recDescriptor,
- comparatorFactory, reverseComparatorFactory, predEvaluatorFactory, true,
+ comparatorFactory, reverseComparatorFactory, leftPredEvalFactory, rightPredEvalFactory, true,
nonMatchWriterFactories);
default:
throw new NotImplementedException();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index 8f911b4..b96b887 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -43,8 +44,6 @@
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -92,11 +91,6 @@
IBinaryHashFunctionFactory[] rightHashFunFactories =
JobGenHelper.variablesToBinaryHashFunctionFactories(keysRightBranch, env, context);
- IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider =
- context.getPredicateEvaluatorFactoryProvider();
- IPredicateEvaluatorFactory predEvaluatorFactory = predEvaluatorFactoryProvider == null ? null
- : predEvaluatorFactoryProvider.getPredicateEvaluatorFactory(keysLeft, keysRight);
-
RecordDescriptor recDescriptor =
JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
IOperatorSchema[] conditionInputSchemas = new IOperatorSchema[1];
@@ -115,17 +109,14 @@
switch (kind) {
case INNER:
opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, leftHashFunFactories,
- rightHashFunFactories, comparatorFactory, recDescriptor, tableSize, predEvaluatorFactory,
- memSizeInFrames);
+ rightHashFunFactories, comparatorFactory, recDescriptor, tableSize, memSizeInFrames);
break;
case LEFT_OUTER:
- IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
- for (int j = 0; j < nonMatchWriterFactories.length; j++) {
- nonMatchWriterFactories[j] = context.getMissingWriterFactory();
- }
+ IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+ ((LeftOuterJoinOperator) joinOp).getMissingValue(), inputSchemas[1].getSize());
opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, leftHashFunFactories,
- rightHashFunFactories, comparatorFactory, predEvaluatorFactory, recDescriptor, true,
- nonMatchWriterFactories, tableSize, memSizeInFrames);
+ rightHashFunFactories, comparatorFactory, recDescriptor, true, nonMatchWriterFactories,
+ tableSize, memSizeInFrames);
break;
default:
throw new NotImplementedException();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
index 2d9dabe..3416163 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
@@ -55,18 +55,19 @@
private final List<LogicalVariable> primaryKeys;
private final List<LogicalVariable> secondaryKeys;
private final ILogicalExpression filterExpr;
+ private final ILogicalExpression prevFilterExpr;
private final IDataSourceIndex<?, ?> dataSourceIndex;
private final List<LogicalVariable> additionalFilteringKeys;
- private final LogicalVariable upsertIndicatorVar;
+ private final LogicalVariable operationVar;
private final List<LogicalVariable> prevSecondaryKeys;
private final LogicalVariable prevAdditionalFilteringKey;
private final int numOfAdditionalNonFilteringFields;
public IndexInsertDeleteUpsertPOperator(List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalFilteringKeys, Mutable<ILogicalExpression> filterExpr,
- IDataSourceIndex<?, ?> dataSourceIndex, LogicalVariable upsertIndicatorVar,
- List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKey,
- int numOfAdditionalNonFilteringFields) {
+ Mutable<ILogicalExpression> prevFilterExpr, IDataSourceIndex<?, ?> dataSourceIndex,
+ LogicalVariable operationVar, List<LogicalVariable> prevSecondaryKeys,
+ LogicalVariable prevAdditionalFilteringKey, int numOfAdditionalNonFilteringFields) {
this.primaryKeys = primaryKeys;
this.secondaryKeys = secondaryKeys;
if (filterExpr != null) {
@@ -74,9 +75,14 @@
} else {
this.filterExpr = null;
}
+ if (prevFilterExpr != null) {
+ this.prevFilterExpr = prevFilterExpr.getValue();
+ } else {
+ this.prevFilterExpr = null;
+ }
this.dataSourceIndex = dataSourceIndex;
this.additionalFilteringKeys = additionalFilteringKeys;
- this.upsertIndicatorVar = upsertIndicatorVar;
+ this.operationVar = operationVar;
this.prevSecondaryKeys = prevSecondaryKeys;
this.prevAdditionalFilteringKey = prevAdditionalFilteringKey;
this.numOfAdditionalNonFilteringFields = numOfAdditionalNonFilteringFields;
@@ -157,8 +163,9 @@
break;
case UPSERT:
runtimeAndConstraints = mp.getIndexUpsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
- typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, upsertIndicatorVar,
- prevSecondaryKeys, prevAdditionalFilteringKey, inputDesc, context, spec, secondaryKeyPipelines);
+ typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, prevFilterExpr,
+ operationVar, prevSecondaryKeys, prevAdditionalFilteringKey, inputDesc, context, spec,
+ secondaryKeyPipelines);
break;
default:
throw new AlgebricksException("Unsupported Operation " + operation);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index 0991db8..6c5c8d0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
@@ -135,10 +136,8 @@
null);
break;
case LEFT_OUTER:
- IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
- for (int j = 0; j < nonMatchWriterFactories.length; j++) {
- nonMatchWriterFactories[j] = context.getMissingWriterFactory();
- }
+ IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+ ((LeftOuterJoinOperator) join).getMissingValue(), inputSchemas[1].getSize());
opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, true,
nonMatchWriterFactories);
break;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index a519275..5af7f98 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -23,6 +23,7 @@
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.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
@@ -32,6 +33,7 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
public class StreamSelectPOperator extends AbstractPhysicalOperator {
@@ -66,10 +68,13 @@
IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
IScalarEvaluatorFactory cond = expressionRuntimeProvider.createEvaluatorFactory(
select.getCondition().getValue(), context.getTypeEnvironment(op), inputSchemas, context);
+ IAlgebricksConstantValue retainMissingAsValue = select.getRetainMissingAsValue();
+ boolean retainMissing = retainMissingAsValue != null;
+ IMissingWriterFactory missingWriterFactory =
+ retainMissing ? JobGenHelper.getMissingWriterFactory(context, retainMissingAsValue) : null;
StreamSelectRuntimeFactory runtime =
- new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(),
- select.getRetainMissing(), inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()),
- context.getMissingWriterFactory());
+ new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(), retainMissing,
+ inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()), missingWriterFactory);
runtime.setSourceLocation(select.getSourceLocation());
// contribute one Asterix framewriter
RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index f17fa70..64abaee 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -180,8 +181,9 @@
@Override
public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Integer indent) throws AlgebricksException {
- addIndent(indent).append("left outer join (").append(op.getCondition().getValue().accept(exprVisitor, indent))
- .append(")");
+ addIndent(indent).append("left outer join ").append("(")
+ .append(op.getCondition().getValue().accept(exprVisitor, indent)).append(")")
+ .append(printLeftOuterMissingValue(op.getMissingValue()));
return null;
}
@@ -244,8 +246,11 @@
@Override
public Void visitSelectOperator(SelectOperator op, Integer indent) throws AlgebricksException {
+ String retainMissing = op.getMissingPlaceholderVariable() != null
+ ? " retain-untrue (" + op.getMissingPlaceholderVariable() + " <- " + op.getRetainMissingAsValue() + ")"
+ : "";
addIndent(indent).append("select (").append(op.getCondition().getValue().accept(exprVisitor, indent))
- .append(")");
+ .append(")").append(retainMissing);
return null;
}
@@ -306,17 +311,18 @@
@Override
public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Integer indent) throws AlgebricksException {
- addIndent(indent).append("outer-unnest " + op.getVariable());
+ addIndent(indent).append("outer-unnest ").append(String.valueOf(op.getVariable()));
if (op.getPositionalVariable() != null) {
buffer.append(" at " + op.getPositionalVariable());
}
buffer.append(" <- " + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+ buffer.append(printLeftOuterMissingValue(op.getMissingValue()));
return null;
}
@Override
public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
- AlgebricksStringBuilderWriter plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+ AlgebricksStringBuilderWriter plan = printAbstractUnnestMapOperator(op, indent, "unnest-map", null);
appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
appendLimitInformation(plan, op.getOutputLimit());
return null;
@@ -325,14 +331,16 @@
@Override
public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
throws AlgebricksException {
- printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map", op.getMissingValue());
return null;
}
private AlgebricksStringBuilderWriter printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
- String opSignature) throws AlgebricksException {
- AlgebricksStringBuilderWriter plan = addIndent(indent).append(opSignature + " " + op.getVariables() + " <- "
- + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+ String opSignature, IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
+ AlgebricksStringBuilderWriter plan =
+ addIndent(indent).append(opSignature).append(' ').append(String.valueOf(op.getVariables()))
+ .append(" <- ").append(op.getExpressionRef().getValue().accept(exprVisitor, indent)).append(
+ leftOuterMissingValue != null ? printLeftOuterMissingValue(leftOuterMissingValue) : "");
appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
return plan;
}
@@ -495,8 +503,8 @@
@Override
public Void visitForwardOperator(ForwardOperator op, Integer indent) throws AlgebricksException {
- addIndent(indent)
- .append("forward: range-map = " + op.getSideDataExpression().getValue().accept(exprVisitor, indent));
+ addIndent(indent).append(
+ "forward: shared-variable = " + op.getSideDataExpression().getValue().accept(exprVisitor, indent));
return null;
}
@@ -665,4 +673,8 @@
}
}
}
+
+ private static String printLeftOuterMissingValue(IAlgebricksConstantValue leftOuterMissingValue) {
+ return leftOuterMissingValue.isNull() ? " (or <- " + leftOuterMissingValue + ")" : "";
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index ed5299c..2728e46 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -94,6 +95,7 @@
private static final String EXPRESSIONS_FIELD = "expressions";
private static final String EXPRESSION_FIELD = "expression";
private static final String CONDITION_FIELD = "condition";
+ private static final String MISSING_VALUE_FIELD = "missing-value";
private final Map<AbstractLogicalOperator, String> operatorIdentity = new HashMap<>();
private final IdCounter idCounter = new IdCounter();
@@ -301,6 +303,9 @@
try {
jsonGenerator.writeStringField(OPERATOR_FIELD, "left-outer-join");
writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
+ if (op.getMissingValue().isNull()) {
+ writeNullField(MISSING_VALUE_FIELD);
+ }
return null;
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -467,14 +472,21 @@
@Override
public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void indent) throws AlgebricksException {
- writeUnnestNonMapOperator(op, "outer-unnest", indent);
- return null;
+ try {
+ writeUnnestNonMapOperator(op, "outer-unnest", indent);
+ if (op.getMissingValue().isNull()) {
+ writeNullField(MISSING_VALUE_FIELD);
+ }
+ return null;
+ } catch (IOException e) {
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
+ }
}
@Override
public Void visitUnnestMapOperator(UnnestMapOperator op, Void indent) throws AlgebricksException {
try {
- writeUnnestMapOperator(op, indent, "unnest-map");
+ writeUnnestMapOperator(op, indent, "unnest-map", null);
writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
return null;
} catch (IOException e) {
@@ -484,7 +496,7 @@
@Override
public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void indent) throws AlgebricksException {
- writeUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ writeUnnestMapOperator(op, indent, "left-outer-unnest-map", op.getMissingValue());
return null;
}
@@ -782,8 +794,8 @@
}
}
- private void writeUnnestMapOperator(AbstractUnnestMapOperator op, Void indent, String opName)
- throws AlgebricksException {
+ private void writeUnnestMapOperator(AbstractUnnestMapOperator op, Void indent, String opName,
+ IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
try {
jsonGenerator.writeStringField(OPERATOR_FIELD, opName);
List<LogicalVariable> variables = op.getVariables();
@@ -792,6 +804,9 @@
}
writeArrayFieldOfExpression(EXPRESSIONS_FIELD, op.getExpressionRef(), indent);
writeFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
+ if (leftOuterMissingValue != null && leftOuterMissingValue.isNull()) {
+ writeNullField(MISSING_VALUE_FIELD);
+ }
} catch (IOException e) {
throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
@@ -963,6 +978,13 @@
jsonGenerator.writeEndObject();
}
+ /////////////// other fields ///////////////
+
+ /** Writes "fieldName": null */
+ private void writeNullField(String fieldName) throws IOException {
+ jsonGenerator.writeNullField(fieldName);
+ }
+
private void flushContentToWriter() throws AlgebricksException {
try {
jsonGenerator.flush();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java
new file mode 100644
index 0000000..07f1f7f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+
+public abstract class LeftOuterTypePropagationPolicy extends TypePropagationPolicy {
+
+ public static final TypePropagationPolicy MISSABLE = new LeftOuterTypePropagationPolicy() {
+
+ protected Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList,
+ List<List<LogicalVariable>> correlatedNullableVariableLists) {
+ boolean makeMissable =
+ !inCorrelatedVariableList(var, correlatedMissableVariableLists, nonMissableVariableList);
+ return makeMissable ? ntc.makeMissableType(varType) : varType;
+ }
+ };
+
+ public static final TypePropagationPolicy NULLABLE = new LeftOuterTypePropagationPolicy() {
+
+ protected Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList,
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList,
+ List<List<LogicalVariable>> correlatedNullableVariableLists) {
+ boolean makeNullable =
+ !inCorrelatedVariableList(var, correlatedNullableVariableLists, nonNullableVariableList);
+ return makeNullable ? ntc.makeNullableType(varType) : varType;
+ }
+ };
+
+ @Override
+ public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists,
+ ITypeEnvPointer... typeEnvs) throws AlgebricksException {
+ int n = typeEnvs.length;
+ // Searches from the inner branch to the outer branch.
+ // TODO(buyingyi): A split operator could lead to the case that the type for a variable could be
+ // found in both inner and outer branches. Fix computeOutputTypeEnvironment() in ProjectOperator
+ // and investigate why many test queries fail if only live variables' types are propagated.
+ for (int i = n - 1; i >= 0; i--) {
+ Object varType = typeEnvs[i].getTypeEnv().getVarType(var, nonMissableVariableList,
+ correlatedMissableVariableLists, nonNullableVariableList, correlatedNullableVariableLists);
+ if (varType == null) {
+ continue;
+ }
+ if (i == 0) { // outer branch
+ return varType;
+ }
+ // inner branch
+ return computeInnerBranchVarType(var, varType, ntc, nonMissableVariableList,
+ correlatedMissableVariableLists, nonNullableVariableList, correlatedNullableVariableLists);
+ }
+ return null;
+ }
+
+ protected abstract Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+ List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists);
+
+ protected static boolean inCorrelatedVariableList(LogicalVariable var,
+ List<List<LogicalVariable>> correlatedOptionalVariableLists,
+ List<LogicalVariable> nonOptionalVariableList) {
+ if (!nonOptionalVariableList.isEmpty()) {
+ for (List<LogicalVariable> correlatedVariables : correlatedOptionalVariableLists) {
+ if (correlatedVariables.contains(var)
+ && !OperatorPropertiesUtil.disjoint(correlatedVariables, nonOptionalVariableList)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+};
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
index c37c674..9cd5fe1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
@@ -32,7 +32,9 @@
@Override
public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists, ITypeEnvPointer... typeEnvs)
+ List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList,
+ List<List<LogicalVariable>> correlatedNullableVariableLists, ITypeEnvPointer... typeEnvs)
throws AlgebricksException {
for (ITypeEnvPointer p : typeEnvs) {
IVariableTypeEnvironment env = p.getTypeEnv();
@@ -40,67 +42,22 @@
throw new AlgebricksException(
"Null environment for pointer " + p + " in getVarType for var=" + var);
}
- Object t = env.getVarType(var, nonMissableVariableList, correlatedMissableVariableLists);
+ Object t = env.getVarType(var, nonMissableVariableList, correlatedMissableVariableLists,
+ nonNullableVariableList, correlatedNullableVariableLists);
if (t != null) {
- if (ntc != null && ntc.canBeMissing(t)) {
- for (List<LogicalVariable> list : correlatedMissableVariableLists) {
- if (list.contains(var)) {
- for (LogicalVariable v : list) {
- if (nonMissableVariableList.contains(v)) {
- return ntc.getNonOptionalType(t);
- }
- }
- }
- }
+ boolean makeNonMissable = !nonMissableVariableList.isEmpty() && ntc.canBeMissing(t)
+ && nonMissableVariableList.contains(var);
+ boolean makeNonNullable = !nonNullableVariableList.isEmpty() && ntc.canBeNull(t)
+ && nonNullableVariableList.contains(var);
+ if (makeNonMissable && makeNonNullable) {
+ return ntc.getNonOptionalType(t);
+ } else if (makeNonMissable) {
+ return ntc.getNonMissableType(t);
+ } else if (makeNonNullable) {
+ return ntc.getNonNullableType(t);
+ } else {
+ return t;
}
- return t;
- }
- }
- return null;
- }
- };
-
- public static final TypePropagationPolicy LEFT_OUTER = new TypePropagationPolicy() {
-
- @Override
- public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
- List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists, ITypeEnvPointer... typeEnvs)
- throws AlgebricksException {
- int n = typeEnvs.length;
- // Searches from the inner branch to the outer branch.
- // TODO(buyingyi): A split operator could lead to the case that the type for a variable could be
- // found in both inner and outer branches. Fix computeOutputTypeEnvironment() in ProjectOperator
- // and investigate why many test queries fail if only live variables' types are propagated.
- for (int i = n - 1; i >= 0; i--) {
- Object t = typeEnvs[i].getTypeEnv().getVarType(var, nonMissableVariableList,
- correlatedMissableVariableLists);
- if (t == null) {
- continue;
- }
- if (i == 0) { // outer branch
- return t;
- }
-
- // inner branch
- boolean nonMissingVarIsProduced = false;
- for (LogicalVariable v : nonMissableVariableList) {
- boolean toBreak = false;
- if (v == var) {
- nonMissingVarIsProduced = true;
- toBreak = true;
- } else if (typeEnvs[i].getTypeEnv().getVarType(v) != null) {
- nonMissingVarIsProduced = true;
- toBreak = true;
- }
- if (toBreak) {
- break;
- }
- }
- if (nonMissingVarIsProduced) {
- return t;
- } else {
- return ntc.makeMissableType(t);
}
}
return null;
@@ -109,5 +66,6 @@
public abstract Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+ List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists,
ITypeEnvPointer... typeEnvs) throws AlgebricksException;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
index 547737c..3bd4bc3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
@@ -38,7 +38,8 @@
}
@Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
return getVarType(var);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
deleted file mode 100644
index 1a85768..0000000
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
+++ /dev/null
@@ -1,79 +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.
- */
-package org.apache.hyracks.algebricks.core.algebra.typing;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-public class PropagateOperatorInputsTypeEnvironment extends AbstractTypeEnvironment {
-
- private final List<LogicalVariable> nonNullVariables = new ArrayList<LogicalVariable>();
- private final List<List<LogicalVariable>> correlatedNullableVariableLists = new ArrayList<List<LogicalVariable>>();
- private final ILogicalOperator op;
- private final ITypingContext ctx;
-
- public PropagateOperatorInputsTypeEnvironment(ILogicalOperator op, ITypingContext ctx,
- IExpressionTypeComputer expressionTypeComputer, IMetadataProvider<?, ?> metadataProvider) {
- super(expressionTypeComputer, metadataProvider);
- this.op = op;
- this.ctx = ctx;
- }
-
- public List<LogicalVariable> getNonNullVariables() {
- return nonNullVariables;
- }
-
- @Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariableList,
- List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
- nonNullVariableList.addAll(nonNullVariables);
- return getVarTypeFullList(var, nonNullVariableList, correlatedNullableVariableLists);
- }
-
- private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonNullVariableList,
- List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
- Object t = varTypeMap.get(var);
- if (t != null) {
- return t;
- }
- for (Mutable<ILogicalOperator> r : op.getInputs()) {
- ILogicalOperator c = r.getValue();
- IVariableTypeEnvironment env = ctx.getOutputTypeEnvironment(c);
- Object t2 = env.getVarType(var, nonNullVariableList, correlatedNullableVariableLists);
- if (t2 != null) {
- return t2;
- }
- }
- return null;
- }
-
- @Override
- public Object getVarType(LogicalVariable var) throws AlgebricksException {
- return getVarTypeFullList(var, nonNullVariables, correlatedNullableVariableLists);
- }
-
-}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
index 27aa902..96b55d5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
@@ -27,6 +27,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
public class PropagatingTypeEnvironment extends AbstractTypeEnvironment {
@@ -40,6 +41,10 @@
private final List<List<LogicalVariable>> correlatedMissableVariableLists = new ArrayList<>();
+ private final List<LogicalVariable> nonNullableVariables = new ArrayList<>();
+
+ private final List<List<LogicalVariable>> correlatedNullableVariableLists = new ArrayList<>();
+
public PropagatingTypeEnvironment(IExpressionTypeComputer expressionTypeComputer,
IMissableTypeComputer missableTypeComputer, IMetadataProvider<?, ?> metadataProvider,
TypePropagationPolicy policy, ITypeEnvPointer[] envPointers) {
@@ -51,7 +56,7 @@
@Override
public Object getVarType(LogicalVariable var) throws AlgebricksException {
- return getVarTypeFullList(var, nonMissableVariables, correlatedMissableVariableLists);
+ return getVarTypeFullList(var, null, null, null, null);
}
public List<LogicalVariable> getNonMissableVariables() {
@@ -62,31 +67,42 @@
return correlatedMissableVariableLists;
}
- @Override
- public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists) throws AlgebricksException {
- for (LogicalVariable v : nonMissableVariables) {
- if (!nonMissableVariableList.contains(v)) {
- nonMissableVariableList.add(v);
- }
- }
- Object t = getVarTypeFullList(var, nonMissableVariableList, correlatedMissableVariableLists);
- for (List<LogicalVariable> list : correlatedMissableVariableLists) {
- if (!correlatedMissableVariableLists.contains(list)) {
- correlatedMissableVariableLists.add(list);
- }
- }
- return t;
+ public List<LogicalVariable> getNonNullableVariables() {
+ return nonNullableVariables;
}
- private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonMissableVariableList,
- List<List<LogicalVariable>> correlatedMissableVariableLists) throws AlgebricksException {
+ public List<List<LogicalVariable>> getCorrelatedNullableVariableLists() {
+ return correlatedNullableVariableLists;
+ }
+
+ @Override
+ public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+ List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
+ List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
+ return getVarTypeFullList(var, nonMissableVariables, correlatedMissableVariableLists, nonNullableVariables,
+ correlatedNullableVariableLists);
+ }
+
+ private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonMissableVariableListExtra,
+ List<List<LogicalVariable>> correlatedMissableVariableListsExtra,
+ List<LogicalVariable> nonNullableVariableListExtra,
+ List<List<LogicalVariable>> correlatedNullableVariableListsExtra) throws AlgebricksException {
Object t = varTypeMap.get(var);
if (t != null) {
return t;
}
- return policy.getVarType(var, missableTypeComputer, nonMissableVariableList, correlatedMissableVariableLists,
- envPointers);
+
+ List<LogicalVariable> nonMissable =
+ OperatorPropertiesUtil.unionAll(nonMissableVariables, nonMissableVariableListExtra);
+ List<LogicalVariable> nonNullable =
+ OperatorPropertiesUtil.unionAll(nonNullableVariables, nonNullableVariableListExtra);
+ List<List<LogicalVariable>> correlatedMissable =
+ OperatorPropertiesUtil.unionAll(correlatedMissableVariableLists, correlatedMissableVariableListsExtra);
+ List<List<LogicalVariable>> correlatedNullable =
+ OperatorPropertiesUtil.unionAll(correlatedNullableVariableLists, correlatedNullableVariableListsExtra);
+
+ return policy.getVarType(var, missableTypeComputer, nonMissable, correlatedMissable, nonNullable,
+ correlatedNullable, envPointers);
}
@Override
@@ -100,6 +116,14 @@
missableVarList.add(v2);
}
}
+ if (nonNullableVariables.remove(v1)) {
+ nonNullableVariables.add(v2);
+ }
+ for (List<LogicalVariable> nullableVarList : correlatedNullableVariableLists) {
+ if (nullableVarList.remove(v1)) {
+ nullableVarList.add(v2);
+ }
+ }
return result;
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index 0df9851..26be09c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -202,6 +202,16 @@
}
}
+ public static void substituteOpInInput(ILogicalOperator workingOp, ILogicalOperator oldOp,
+ Mutable<ILogicalOperator> newOpRef) {
+ for (int i = 0; i < workingOp.getInputs().size(); i++) {
+ if (workingOp.getInputs().get(i).getValue().equals(oldOp)) {
+ workingOp.getInputs().set(i, newOpRef);
+ break;
+ }
+ }
+ }
+
public static ILogicalPlan deepCopy(ILogicalPlan plan, ILogicalOperator dataSource) throws AlgebricksException {
List<Mutable<ILogicalOperator>> roots = plan.getRoots();
List<Mutable<ILogicalOperator>> newRoots = clonePipeline(roots);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index c0d72bb..c66ee40 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -18,8 +18,11 @@
*/
package org.apache.hyracks.algebricks.core.algebra.util;
+import java.util.ArrayList;
import java.util.Collection;
+import java.util.Collections;
import java.util.HashSet;
+import java.util.List;
import java.util.Set;
import org.apache.commons.lang3.mutable.Mutable;
@@ -32,13 +35,13 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.CardinalityInferenceVisitor;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
@@ -221,33 +224,6 @@
}
}
- public static boolean isMissingTest(AbstractLogicalOperator op) {
- if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
- return false;
- }
- AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
- if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
- return false;
- }
- ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
- if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return false;
- }
- AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
- if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
- return false;
- }
- ILogicalExpression a1 = f1.getArguments().get(0).getValue();
- if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
- return false;
- }
- AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
- if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
- return false;
- }
- return true;
- }
-
public static void typePlan(ILogicalPlan p, IOptimizationContext context) throws AlgebricksException {
for (Mutable<ILogicalOperator> r : p.getRoots()) {
typeOpRec(r, context);
@@ -355,4 +331,22 @@
LogicalOperatorTag opTag = op.getOperatorTag();
return opTag == LogicalOperatorTag.REPLICATE || opTag == LogicalOperatorTag.SPLIT;
}
+
+ public static <T> List<T> unionAll(List<T> list1, List<T> list2) {
+ if (list2 == null || list2.isEmpty()) {
+ return list1 == null ? Collections.emptyList() : list1;
+ } else if (list1 == null || list1.isEmpty()) {
+ return list2;
+ } else {
+ List<T> result = new ArrayList<>(list1.size() + list2.size());
+ result.addAll(list1);
+ result.addAll(list2);
+ return result;
+ }
+ }
+
+ public static FunctionIdentifier getIsMissingNullFunction(IAlgebricksConstantValue value) {
+ return value.isMissing() ? AlgebricksBuiltinFunctions.IS_MISSING
+ : value.isNull() ? AlgebricksBuiltinFunctions.IS_NULL : null;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 20632f1..86c6066 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -30,11 +30,11 @@
public static final boolean SORT_PARALLEL_DEFAULT = true;
public static final boolean INDEX_ONLY_DEFAULT = true;
public static final boolean SANITYCHECK_DEFAULT = false;
- public static final boolean EXTERNAL_FIELD_PUSHDOWN_DEFAULT = false;
+ public static final boolean EXTERNAL_FIELD_PUSHDOWN_DEFAULT = true;
public static final boolean SUBPLAN_MERGE_DEFAULT = true;
public static final boolean SUBPLAN_NESTEDPUSHDOWN_DEFAULT = true;
public static final boolean MIN_MEMORY_ALLOCATION_DEFAULT = true;
- public static final boolean ARRAY_INDEX_DEFAULT = false;
+ public static final boolean ARRAY_INDEX_DEFAULT = true;
public static final int EXTERNAL_SCAN_BUFFER_SIZE =
StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 2ca4a36..7c7d5a8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -60,6 +60,7 @@
private final ITypeTraitProvider typeTraitProvider;
private final IMetadataProvider<?, ?> metadataProvider;
private final IMissingWriterFactory missingWriterFactory;
+ private final IMissingWriterFactory nullWriterFactory;
private final IUnnestingPositionWriterFactory unnestingPositionWriterFactory;
private final INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
private final Object appContext;
@@ -85,7 +86,8 @@
IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
IBinaryBooleanInspectorFactory booleanInspectorFactory,
IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
- IMissingWriterFactory missingWriterFactory, IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
+ IMissingWriterFactory missingWriterFactory, IMissingWriterFactory nullWriterFactory,
+ IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
IExpressionRuntimeProvider expressionRuntimeProvider, IExpressionTypeComputer expressionTypeComputer,
ITypingContext typingContext, IExpressionEvalSizeComputer expressionEvalSizeComputer,
@@ -107,6 +109,7 @@
this.clusterLocations = clusterLocations;
this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
this.missingWriterFactory = missingWriterFactory;
+ this.nullWriterFactory = nullWriterFactory;
this.unnestingPositionWriterFactory = unnestingPositionWriterFactory;
this.expressionRuntimeProvider = expressionRuntimeProvider;
this.expressionTypeComputer = expressionTypeComputer;
@@ -199,6 +202,10 @@
return missingWriterFactory;
}
+ public IMissingWriterFactory getNullWriterFactory() {
+ return nullWriterFactory;
+ }
+
public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory() {
return unnestingPositionWriterFactory;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
index 5142ce7..42364c8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
@@ -18,11 +18,13 @@
*/
package org.apache.hyracks.algebricks.core.jobgen.impl;
+import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
@@ -38,10 +40,12 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -213,4 +217,25 @@
}
return projectionList;
}
+
+ public static IMissingWriterFactory[] createMissingWriterFactories(JobGenContext context,
+ IAlgebricksConstantValue missingOrNullValue, int size) throws AlgebricksException {
+ IMissingWriterFactory missingWriterFactory = getMissingWriterFactory(context, missingOrNullValue);
+ IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[size];
+ Arrays.fill(nonMatchWriterFactories, missingWriterFactory);
+ return nonMatchWriterFactories;
+ }
+
+ public static IMissingWriterFactory getMissingWriterFactory(JobGenContext context,
+ IAlgebricksConstantValue missingOrNullValue) throws AlgebricksException {
+ IMissingWriterFactory missingWriterFactory;
+ if (missingOrNullValue.isMissing()) {
+ missingWriterFactory = context.getMissingWriterFactory();
+ } else if (missingOrNullValue.isNull()) {
+ missingWriterFactory = context.getNullWriterFactory();
+ } else {
+ throw new AlgebricksException(ErrorCode.ILLEGAL_STATE);
+ }
+ return missingWriterFactory;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 62bf671..9c9b21d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -87,8 +87,7 @@
}
public int getMaxFramesForJoinLeftInput() {
- int frameSize = getFrameSize();
- return getInt(MAX_FRAMES_FOR_JOIN_LEFT_INPUT, (int) (140L * 1024 * MB / frameSize));
+ return getInt(MAX_FRAMES_FOR_JOIN_LEFT_INPUT, -1);
}
public void setMaxFramesForJoinLeftInput(int frameLimit) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 01f7403..ee7f7aa 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -166,7 +166,9 @@
@Override
public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Boolean showDetails) throws AlgebricksException {
stringBuilder.setLength(0);
- stringBuilder.append("left outer join (").append(op.getCondition().getValue().toString()).append(")");
+ stringBuilder.append("left outer join ").append("(").append(op.getCondition().getValue().toString())
+ .append(")");
+ stringBuilder.append(op.getMissingValue().isNull() ? " (or null) " : "");
appendSchema(op, showDetails);
appendAnnotations(op, showDetails);
appendPhysicalOperatorInfo(op, showDetails);
@@ -351,6 +353,7 @@
if (op.getPositionalVariable() != null) {
stringBuilder.append(" at ").append(op.getPositionalVariable());
}
+ stringBuilder.append(op.getMissingValue().isNull() ? " (or null) " : "");
stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
appendSchema(op, showDetails);
appendAnnotations(op, showDetails);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
index f047cec..46e41f1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
@@ -283,14 +283,14 @@
computeAndSetTypeEnvironmentForOperator(ets);
ets.recomputeSchema();
- SelectOperator op1 = new SelectOperator(newMutable(ConstantExpression.TRUE), false, null);
+ SelectOperator op1 = new SelectOperator(newMutable(ConstantExpression.TRUE));
op1.getInputs().add(newMutable(ets));
computeAndSetTypeEnvironmentForOperator(op1);
op1.recomputeSchema();
op1.getInputs().clear();
- SelectOperator op2 = new SelectOperator(newMutable(ConstantExpression.FALSE), false, null);
+ SelectOperator op2 = new SelectOperator(newMutable(ConstantExpression.FALSE));
op2.getInputs().add(newMutable(ets));
op2.recomputeSchema();
// no type env
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
index 50c7703..c0773b2 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -105,9 +105,9 @@
firstExpression = expression;
} else {
// New select operator
- SelectOperator newSelectOperator =
- new SelectOperator(new MutableObject<>(expression), originalSelectOperator.getRetainMissing(),
- originalSelectOperator.getMissingPlaceholderVariable());
+ SelectOperator newSelectOperator = new SelectOperator(new MutableObject<>(expression),
+ originalSelectOperator.getRetainMissingAsValue(),
+ originalSelectOperator.getMissingPlaceholderVariable());
newSelectOperator.setSourceLocation(sourceLoc);
// Put the new operator at the bottom (child of current operator)
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
index c615a94..71a8486 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
@@ -104,6 +104,7 @@
} else if (AlgebricksBuiltinFunctions.isComparisonFunction(fi) || isComparisonFunction(fi)) {
for (Mutable<ILogicalExpression> exprRef : fexp.getArguments()) {
if (exprRef.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+ boolean argModified = false;
SourceLocation exprRefSourceLoc = exprRef.getValue().getSourceLocation();
LogicalVariable newVar = context.newVar();
AssignOperator newAssign = new AssignOperator(newVar,
@@ -123,7 +124,7 @@
// place assign on left branch
newAssign.getInputs().add(new MutableObject<ILogicalOperator>(leftBranch));
leftBranchRef.setValue(newAssign);
- modified = true;
+ argModified = true;
} else {
Mutable<ILogicalOperator> rightBranchRef = joinOp.getInputs().get(1);
ILogicalOperator rightBranch = rightBranchRef.getValue();
@@ -133,17 +134,18 @@
// place assign on right branch
newAssign.getInputs().add(new MutableObject<ILogicalOperator>(rightBranch));
rightBranchRef.setValue(newAssign);
- modified = true;
+ argModified = true;
}
}
- if (modified) {
+ if (argModified) {
// Replace original expr with variable reference.
VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
newVarRef.setSourceLocation(exprRefSourceLoc);
exprRef.setValue(newVarRef);
context.computeAndSetTypeEnvironmentForOperator(newAssign);
context.computeAndSetTypeEnvironmentForOperator(joinOp);
+ modified = true;
}
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
new file mode 100644
index 0000000..e1b2e5d
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PopulateResultMetadataRule.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.result.IResultMetadata;
+
+/**
+ * Populates {@link DistributeResultOperator}'s {@link IResultMetadata} instance with output type information.
+ */
+public final class PopulateResultMetadataRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT) {
+ return false;
+ }
+ DistributeResultOperator dop = (DistributeResultOperator) op;
+ IResultMetadata resultMetadata = dop.getResultMetadata();
+ if (resultMetadata.getOutputTypes() != null) {
+ return false;
+ }
+ List<Mutable<ILogicalExpression>> exprList = dop.getExpressions();
+ List<Object> exprTypeList = new ArrayList<>(exprList.size());
+ IVariableTypeEnvironment typeEnv = context.getOutputTypeEnvironment(dop.getInputs().get(0).getValue());
+ for (Mutable<ILogicalExpression> exprRef : exprList) {
+ exprTypeList.add(typeEnv.getType(exprRef.getValue()));
+ }
+ resultMetadata.setOutputTypes(exprTypeList);
+ return true;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
index afbbc4f..410b2ec 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -80,7 +80,7 @@
}
// pull up
ILogicalExpression pulledCond = makeCondition(otherPredicates, context);
- SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond), false, null);
+ SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
ILogicalExpression newJoinCond = makeCondition(eqVarVarComps, context);
join.getCondition().setValue(newJoinCond);
select.getInputs().add(new MutableObject<ILogicalOperator>(join));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
index 5f66c2f..5477662 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -38,9 +38,11 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
@@ -98,10 +100,10 @@
ILogicalOperator joinBranchRight = joinBranchRightRef.getValue();
VariableUtilities.getLiveVariables(joinBranchLeft, joinLiveVarsLeft);
VariableUtilities.getLiveVariables(joinBranchRight, joinLiveVarsRight);
- Mutable<ILogicalOperator> opIterRef = opRef2;
+ Mutable<ILogicalOperator> opIterRef;
ILogicalOperator opIter = op2;
while (opIter != join) {
- LogicalOperatorTag tag = ((AbstractLogicalOperator) opIter).getOperatorTag();
+ LogicalOperatorTag tag = opIter.getOperatorTag();
if (tag == LogicalOperatorTag.PROJECT) {
notPushedStack.addFirst(opIter);
} else {
@@ -129,11 +131,11 @@
boolean intersectsAllBranches = true;
boolean[] intersectsBranch = new boolean[join.getInputs().size()];
- LinkedList<LogicalVariable> selectVars = new LinkedList<LogicalVariable>();
+ LinkedList<LogicalVariable> selectVars = new LinkedList<>();
select.getCondition().getValue().getUsedVariables(selectVars);
int i = 0;
for (Mutable<ILogicalOperator> branch : join.getInputs()) {
- LinkedList<LogicalVariable> branchVars = new LinkedList<LogicalVariable>();
+ LinkedList<LogicalVariable> branchVars = new LinkedList<>();
VariableUtilities.getLiveVariables(branch.getValue(), branchVars);
if (i == 0) {
branchVars.addAll(liveInOpsToPushLeft);
@@ -181,7 +183,9 @@
if (j > 0 && isLoj) {
// if a LOJ and the select condition is not-missing filtering,
// we rewrite LOJ to IJ for this case.
- if (containsNotMissingFiltering(selectCondition)) {
+ FunctionIdentifier isMissingNullFunction = OperatorPropertiesUtil
+ .getIsMissingNullFunction(((LeftOuterJoinOperator) join).getMissingValue());
+ if (containsNotMissingFiltering(selectCondition, isMissingNullFunction)) {
lojToInner = true;
}
// Do not push conditions into the right branch of a LOJ;
@@ -247,7 +251,7 @@
ILogicalOperator op = iter.previous();
List<Mutable<ILogicalOperator>> opInpList = op.getInputs();
opInpList.clear();
- opInpList.add(new MutableObject<ILogicalOperator>(topOp));
+ opInpList.add(new MutableObject<>(topOp));
topOp = op;
context.computeAndSetTypeEnvironmentForOperator(op);
}
@@ -276,7 +280,7 @@
if (!bAddedToConj) {
AbstractFunctionCallExpression newCond = new ScalarFunctionCallExpression(
context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND),
- select.getCondition(), new MutableObject<ILogicalExpression>(join.getCondition().getValue()));
+ select.getCondition(), new MutableObject<>(join.getCondition().getValue()));
join.getCondition().setValue(newCond);
}
}
@@ -284,9 +288,9 @@
private static void copySelectToBranch(SelectOperator select, Mutable<ILogicalOperator> branch,
IOptimizationContext context) throws AlgebricksException {
- ILogicalOperator newSelect = new SelectOperator(select.getCondition(), select.getRetainMissing(),
+ ILogicalOperator newSelect = new SelectOperator(select.getCondition(), select.getRetainMissingAsValue(),
select.getMissingPlaceholderVariable());
- Mutable<ILogicalOperator> newRef = new MutableObject<ILogicalOperator>(branch.getValue());
+ Mutable<ILogicalOperator> newRef = new MutableObject<>(branch.getValue());
newSelect.getInputs().add(newRef);
branch.setValue(newSelect);
context.computeAndSetTypeEnvironmentForOperator(newSelect);
@@ -296,16 +300,17 @@
* Whether the expression contains a not-missing filtering
*
* @param expr
+ * @param isMissingNullFunId
* @return true if the expression contains a not-missing filtering function call; false otherwise.
*/
- private boolean containsNotMissingFiltering(ILogicalExpression expr) {
+ private boolean containsNotMissingFiltering(ILogicalExpression expr, FunctionIdentifier isMissingNullFunId) {
if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
return false;
}
ScalarFunctionCallExpression func = (ScalarFunctionCallExpression) expr;
if (func.getFunctionIdentifier() == AlgebricksBuiltinFunctions.AND) {
for (Mutable<ILogicalExpression> argumentRef : func.getArguments()) {
- if (containsNotMissingFiltering(argumentRef.getValue())) {
+ if (containsNotMissingFiltering(argumentRef.getValue(), isMissingNullFunId)) {
return true;
}
}
@@ -319,34 +324,6 @@
return false;
}
ScalarFunctionCallExpression func2 = (ScalarFunctionCallExpression) arg;
- if (func2.getFunctionIdentifier() != AlgebricksBuiltinFunctions.IS_MISSING) {
- return false;
- }
- return true;
- }
-
- /**
- * Whether the expression contains a missing filtering
- *
- * @param expr
- * @return true if the expression contains a missing filtering function call; false otherwise.
- */
- private boolean containsMissingFiltering(ILogicalExpression expr) {
- if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return false;
- }
- ScalarFunctionCallExpression func = (ScalarFunctionCallExpression) expr;
- if (func.getFunctionIdentifier() == AlgebricksBuiltinFunctions.AND) {
- for (Mutable<ILogicalExpression> argumentRef : func.getArguments()) {
- if (containsMissingFiltering(argumentRef.getValue())) {
- return true;
- }
- }
- return false;
- }
- if (func.getFunctionIdentifier() != AlgebricksBuiltinFunctions.IS_MISSING) {
- return false;
- }
- return true;
+ return func2.getFunctionIdentifier().equals(isMissingNullFunId);
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index b95971f..a6fe495 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -428,11 +428,11 @@
return new IndexBulkloadPOperator(primaryKeys, secondaryKeys, additionalFilteringKeys,
opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex());
} else {
- LogicalVariable upsertIndicatorVar = null;
+ LogicalVariable operationVar = null;
List<LogicalVariable> prevSecondaryKeys = null;
LogicalVariable prevAdditionalFilteringKey = null;
if (opInsDel.getOperation() == Kind.UPSERT) {
- upsertIndicatorVar = getKey(opInsDel.getUpsertIndicatorExpr().getValue());
+ operationVar = getKey(opInsDel.getOperationExpr().getValue());
prevSecondaryKeys = new ArrayList<>();
getKeys(opInsDel.getPrevSecondaryKeyExprs(), prevSecondaryKeys);
if (opInsDel.getPrevAdditionalFilteringExpression() != null) {
@@ -442,8 +442,8 @@
}
}
return new IndexInsertDeleteUpsertPOperator(primaryKeys, secondaryKeys, additionalFilteringKeys,
- opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex(), upsertIndicatorVar,
- prevSecondaryKeys, prevAdditionalFilteringKey,
+ opInsDel.getFilterExpression(), opInsDel.getBeforeOpFilterExpression(),
+ opInsDel.getDataSourceIndex(), operationVar, prevSecondaryKeys, prevAdditionalFilteringKey,
opInsDel.getNumberOfAdditionalNonFilteringFields());
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
index c1e613b..391d03a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
@@ -65,6 +65,8 @@
import org.apache.hyracks.api.exceptions.SourceLocation;
/**
+ * NOTE: This rule is currently not used
+ *
* The rule searches for SUBPLAN operator with a optional PROJECT operator and
* an AGGREGATE followed by a join operator.
*
@@ -258,8 +260,7 @@
ScalarFunctionCallExpression nonNullTest =
new ScalarFunctionCallExpression(finfoNot, new MutableObject<ILogicalExpression>(isNullTest));
nonNullTest.setSourceLocation(sourceLoc);
- SelectOperator selectNonNull =
- new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest), false, null);
+ SelectOperator selectNonNull = new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest));
selectNonNull.setSourceLocation(sourceLoc);
GroupByOperator g = new GroupByOperator();
g.setSourceLocation(sourceLoc);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
index e37be1b..d7f8d38 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
@@ -22,17 +22,26 @@
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.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+/**
+ * NOTE: This rule is currently not used
+ *
+ */
public class IntroduceLeftOuterJoinForSubplanRule implements IAlgebraicRewriteRule {
@Override
@@ -65,7 +74,7 @@
AbstractLogicalOperator op1 = (AbstractLogicalOperator) subplanRoot.getValue();
Mutable<ILogicalOperator> opUnder = subplan.getInputs().get(0);
- if (OperatorPropertiesUtil.isMissingTest((AbstractLogicalOperator) opUnder.getValue())) {
+ if (isMissingTest((AbstractLogicalOperator) opUnder.getValue())) {
return false;
}
@@ -87,7 +96,8 @@
}
}
ntsRef.setValue(opUnder.getValue());
- LeftOuterJoinOperator loj = new LeftOuterJoinOperator(join.getCondition());
+ LeftOuterJoinOperator loj =
+ new LeftOuterJoinOperator(join.getCondition(), ConstantExpression.MISSING.getValue());
loj.setSourceLocation(join.getSourceLocation());
loj.getInputs().add(leftRef);
loj.getInputs().add(rightRef);
@@ -124,4 +134,30 @@
return getNtsAtEndOfPipeline(op.getInputs().get(0));
}
+ public static boolean isMissingTest(AbstractLogicalOperator op) {
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return false;
+ }
+ ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
+ if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
+ if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+ return false;
+ }
+ ILogicalExpression a1 = f1.getArguments().get(0).getValue();
+ if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ return false;
+ }
+ AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+ if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+ return false;
+ }
+ return true;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
index d3ecf69..428b710 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
@@ -25,19 +25,25 @@
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
/**
+ * NOTE: This rule is currently not used
+ *
* Looks for a nested group-by plan ending in
* subplan {
* ...
@@ -82,7 +88,7 @@
if (op1.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
SubplanOperator subplan = (SubplanOperator) op1;
AbstractLogicalOperator op2 = (AbstractLogicalOperator) subplan.getInputs().get(0).getValue();
- if (OperatorPropertiesUtil.isMissingTest(op2)) {
+ if (isMissingTest(op2)) {
if (subplan.getNestedPlans().size() == 1) {
ILogicalPlan p1 = subplan.getNestedPlans().get(0);
if (p1.getRoots().size() == 1) {
@@ -122,4 +128,31 @@
return true;
}
+
+ private static boolean isMissingTest(AbstractLogicalOperator op) {
+ if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ return false;
+ }
+ AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+ return false;
+ }
+ ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
+ if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
+ if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+ return false;
+ }
+ ILogicalExpression a1 = f1.getArguments().get(0).getValue();
+ if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ return false;
+ }
+ AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+ if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+ return false;
+ }
+ return true;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
index 9ee5a48..1b42bb6 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
@@ -18,8 +18,6 @@
*/
package org.apache.hyracks.algebricks.runtime.operators.std;
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -38,6 +36,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
public class UnnestRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
@@ -83,14 +82,7 @@
@Override
public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
throws HyracksDataException {
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- DataOutput output = new DataOutputStream(bos);
- if (missingWriterFactory != null) {
- IMissingWriter missingWriter = missingWriterFactory.createMissingWriter();
- missingWriter.writeMissing(output);
- }
- byte[] missingBytes = bos.toByteArray();
- int missingBytesLen = bos.size();
+ ByteArrayAccessibleOutputStream missingBytes = leftOuter ? writeMissingBytes() : null;
IEvaluatorContext evalCtx = new EvaluatorContext(ctx);
return new AbstractOneInputOneOutputOneFramePushRuntime() {
private IPointable p = VoidPointable.FACTORY.createPointable();
@@ -154,7 +146,7 @@
}
if (unnestColIsProjected) {
if (missing) {
- tupleBuilder.addField(missingBytes, 0, missingBytesLen);
+ tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
} else {
tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
}
@@ -166,7 +158,7 @@
if (positionWriter != null) {
// Write the positional variable
if (missing) {
- tupleBuilder.addField(missingBytes, 0, missingBytesLen);
+ tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
} else {
positionWriter.write(tupleBuilder.getDataOutput(), positionIndex);
tupleBuilder.addFieldEndOffset();
@@ -181,4 +173,11 @@
}
};
}
+
+ private ByteArrayAccessibleOutputStream writeMissingBytes() throws HyracksDataException {
+ ByteArrayAccessibleOutputStream baos = new ByteArrayAccessibleOutputStream();
+ IMissingWriter missingWriter = missingWriterFactory.createMissingWriter();
+ missingWriter.writeMissing(new DataOutputStream(baos));
+ return baos;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
index eb68e79..4773dc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
@@ -53,12 +53,7 @@
public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(final ICCServiceContext ccServiceCtx,
Set<JobFlag> jobFlags) throws HyracksException {
final JobActivityGraphBuilder builder = new JobActivityGraphBuilder(spec, jobFlags);
- PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
- @Override
- public void visit(IConnectorDescriptor conn) throws HyracksException {
- builder.addConnector(conn);
- }
- });
+ PlanUtils.visit(spec, builder::addConnector);
PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
@Override
public void visit(IOperatorDescriptor op) {
@@ -78,12 +73,7 @@
acg.setConnectorPolicyAssignmentPolicy(spec.getConnectorPolicyAssignmentPolicy());
acg.setUseConnectorPolicyForScheduling(spec.isUseConnectorPolicyForScheduling());
final Set<Constraint> constraints = new HashSet<>();
- final IConstraintAcceptor acceptor = new IConstraintAcceptor() {
- @Override
- public void addConstraint(Constraint constraint) {
- constraints.add(constraint);
- }
- };
+ final IConstraintAcceptor acceptor = constraints::add;
PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
@Override
public void visit(IOperatorDescriptor op) {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java
index 41fed25..4d324f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksJobletContext.java
@@ -33,6 +33,8 @@
long getJobStartTime();
+ String getJobStartTimeZoneId();
+
ICounterContext getCounterContext();
Object getGlobalJobData();
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluator.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluator.java
index 5fbc1ab..20381f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluator.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluator.java
@@ -21,9 +21,6 @@
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
-/*
- * Compares two tuples to make sure that records, whose comparison keys are NULL do not pass comparator filter
- */
public interface IPredicateEvaluator {
- public boolean evaluate(IFrameTupleAccessor fta0, int tupId0, IFrameTupleAccessor fta1, int tupId1);
+ boolean evaluate(IFrameTupleAccessor fta, int tupId);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactory.java
index eb94514..8748587 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactory.java
@@ -22,9 +22,8 @@
import java.io.Serializable;
/*
- * Provides PredicateEvaluator for equi-join related operators
+ * Provides PredicateEvaluator
*/
-
public interface IPredicateEvaluatorFactory extends Serializable {
- public IPredicateEvaluator createPredicateEvaluator();
+ IPredicateEvaluator createPredicateEvaluator();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactoryProvider.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactoryProvider.java
index 3eefc29..bc3e5c8 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactoryProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IPredicateEvaluatorFactoryProvider.java
@@ -24,7 +24,6 @@
/*
* Provides PredicateEvaluatorFactory based on (equi-join) keys
*/
-
public interface IPredicateEvaluatorFactoryProvider extends Serializable {
- public IPredicateEvaluatorFactory getPredicateEvaluatorFactory(int[] keys0, int[] keys1);
+ IPredicateEvaluatorFactory getPredicateEvaluatorFactory(int[] keys);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/MappedFileSplit.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/MappedFileSplit.java
index 0a0e026..c7be4ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/MappedFileSplit.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/io/MappedFileSplit.java
@@ -59,7 +59,7 @@
@Override
public FileReference getFileReference(IIOManager ioManager) throws HyracksDataException {
if (cached == null) {
- cached = new FileReference(ioManager.getIODevices().get(ioDevice), getPath());
+ cached = super.getFileReference(ioManager);
}
return cached;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java
index 32f784b..b483158 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java
@@ -53,7 +53,14 @@
String getKeyStorePassword();
/**
- * Gets a trust store file to be used for validating certificates of secured connections.
+ * Gets the trust store to be used for validating certificates of secured connections
+ *
+ * @return the trust store to be used
+ */
+ KeyStore getTrustStore();
+
+ /**
+ * Gets a trust store file to be used if {@link INetworkSecurityConfig#getTrustStore()} returns null.
*
* @return the trust store file
*/
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultMetadata.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultMetadata.java
index 12696af..56dd1a4 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultMetadata.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/result/IResultMetadata.java
@@ -19,6 +19,11 @@
package org.apache.hyracks.api.result;
import java.io.Serializable;
+import java.util.List;
public interface IResultMetadata extends Serializable {
+
+ List<Object> getOutputTypes();
+
+ void setOutputTypes(List<Object> typeInfo);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
index 7c304b9..7147542 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -197,4 +197,14 @@
}
return current;
}
+
+ /**
+ * Returns the message of the throwable if of type IFormattedException, otherwise, .toString() is returned
+ *
+ * @param e throwable
+ * @return error message
+ */
+ public static String getMessageOrToString(Throwable e) {
+ return e instanceof IFormattedException ? e.getMessage() : e.toString();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
index b677132..765f9d7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
@@ -18,9 +18,25 @@
*/
package org.apache.hyracks.api.util;
+import java.util.function.Consumer;
+
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
@FunctionalInterface
public interface HyracksThrowingConsumer<V> {
- void process(V value) throws HyracksDataException;
+ void accept(V value) throws HyracksDataException;
+
+ @SuppressWarnings("Duplicates")
+ static <T> Consumer<T> asUnchecked(HyracksThrowingConsumer<T> consumer) {
+ return input -> {
+ try {
+ consumer.accept(input);
+ } catch (HyracksDataException e) {
+ throw new UncheckedExecutionException(e);
+ }
+ };
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java
index 6ad53ab..ae49cb6 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/IoUtil.java
@@ -19,13 +19,18 @@
package org.apache.hyracks.api.util;
import java.io.File;
+import java.io.FileFilter;
import java.io.FileNotFoundException;
+import java.io.FilenameFilter;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Objects;
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -152,4 +157,36 @@
}
}
}
+
+ /**
+ * Gets a collection of files matching {@code filter} by searching {@code root} directory and
+ * all of its subdirectories
+ *
+ * @param root
+ * @param filter
+ * @return a collection of matching files
+ */
+ public static Collection<File> getMatchingFiles(Path root, FilenameFilter filter) {
+ if (!Files.isDirectory(root)) {
+ throw new IllegalArgumentException("Parameter 'root' is not a directory: " + root);
+ }
+ Objects.requireNonNull(filter);
+ Collection<File> files = new ArrayList<>();
+ FileFilter dirOrMatchingFileFilter = file -> file.isDirectory() || filter.accept(file, file.getName());
+ collectDirFiles(root.toFile(), dirOrMatchingFileFilter, files);
+ return files;
+ }
+
+ private static void collectDirFiles(File dir, FileFilter filter, Collection<File> files) {
+ File[] matchingFiles = dir.listFiles(filter);
+ if (matchingFiles != null) {
+ for (File file : matchingFiles) {
+ if (file.isDirectory()) {
+ collectDirFiles(file, filter, files);
+ } else {
+ files.add(file);
+ }
+ }
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index 712d4ee..57e22e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -49,7 +49,11 @@
</dependency>
<dependency>
<groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
+ <artifactId>netty-buffer</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
index f54a62c..e4c98ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
@@ -184,6 +184,10 @@
public synchronized void failNode(String nodeId) throws HyracksException {
NodeControllerState state = nodeRegistry.get(nodeId);
+ if (state == null) {
+ LOGGER.info("node {} is not registered; no need to fail it", nodeId);
+ return;
+ }
Set<JobId> affectedJobIds = state.getActiveJobIds();
// Removes the node from node map.
nodeRegistry.remove(nodeId);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
index ad55398..3574acd 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
@@ -519,7 +519,7 @@
node.getNodeController().startTasks(deploymentId, jobId, jagBytes, taskDescriptors,
connectorPolicies, jobRun.getFlags(),
ccs.createOrGetJobParameterByteStore(jobId).getParameterMap(), deployedJobSpecId,
- jobRun.getStartTime());
+ jobRun.getStartTime(), jobRun.getStartTimeZoneId());
}
}
} catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
index d782ac5..9850ee2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
@@ -21,6 +21,7 @@
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
+import java.time.ZoneId;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@@ -305,6 +306,7 @@
// Executes a job when the required capacity for the job is met.
private void executeJob(JobRun run) throws HyracksException {
run.setStartTime(System.currentTimeMillis());
+ run.setStartTimeZoneId(ZoneId.systemDefault().getId());
JobId jobId = run.getJobId();
activeRunMap.put(jobId, run);
run.setStatus(JobStatus.RUNNING, null);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
index 5b98260..0cc09b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
@@ -86,6 +86,8 @@
private long startTime;
+ private String startTimeZoneId;
+
private long endTime;
private JobStatus status;
@@ -200,6 +202,14 @@
this.startTime = startTime;
}
+ public String getStartTimeZoneId() {
+ return startTimeZoneId;
+ }
+
+ public void setStartTimeZoneId(String startTimeZoneId) {
+ this.startTimeZoneId = startTimeZoneId;
+ }
+
public long getEndTime() {
return endTime;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
index a754baf..97f6edb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
@@ -42,8 +42,8 @@
public interface INodeController {
void startTasks(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
List<TaskAttemptDescriptor> taskDescriptors, Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies,
- Set<JobFlag> flags, Map<byte[], byte[]> jobParameters, DeployedJobSpecId deployedJobSpecId, long startTime)
- throws Exception;
+ Set<JobFlag> flags, Map<byte[], byte[]> jobParameters, DeployedJobSpecId deployedJobSpecId, long startTime,
+ String startTimeZoneId) throws Exception;
void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
index 27a8b02..1939f9e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
@@ -796,11 +796,13 @@
private final Map<byte[], byte[]> jobParameters;
private final DeployedJobSpecId deployedJobSpecId;
private final long jobStartTime;
+ private final String jobStartTimeZoneId;
public StartTasksFunction(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
List<TaskAttemptDescriptor> taskDescriptors,
Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, Set<JobFlag> flags,
- Map<byte[], byte[]> jobParameters, DeployedJobSpecId deployedJobSpecId, long jobStartTime) {
+ Map<byte[], byte[]> jobParameters, DeployedJobSpecId deployedJobSpecId, long jobStartTime,
+ String jobStartTimeZoneId) {
this.deploymentId = deploymentId;
this.jobId = jobId;
this.planBytes = planBytes;
@@ -810,6 +812,7 @@
this.jobParameters = jobParameters;
this.deployedJobSpecId = deployedJobSpecId;
this.jobStartTime = jobStartTime;
+ this.jobStartTimeZoneId = jobStartTimeZoneId;
}
@Override
@@ -853,6 +856,10 @@
return jobStartTime;
}
+ public String getJobStartTimeZoneId() {
+ return jobStartTimeZoneId;
+ }
+
public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
DataInputStream dis = new DataInputStream(bais);
@@ -923,9 +930,10 @@
}
long jobStartTime = dis.readLong();
+ String jobStartTimeZoneId = dis.readUTF();
return new StartTasksFunction(deploymentId, jobId, planBytes, taskDescriptors, connectorPolicies, flags,
- jobParameters, deployedJobSpecId, jobStartTime);
+ jobParameters, deployedJobSpecId, jobStartTime, jobStartTimeZoneId);
}
public static void serialize(OutputStream out, Object object) throws Exception {
@@ -981,6 +989,10 @@
//write job start time
dos.writeLong(fn.jobStartTime);
+
+ //write job start timezone
+ dos.writeUTF(fn.jobStartTimeZoneId);
+
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index ff5dd33..79e176f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -67,9 +67,9 @@
public void startTasks(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
List<TaskAttemptDescriptor> taskDescriptors, Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies,
Set<JobFlag> flags, Map<byte[], byte[]> jobParameters, DeployedJobSpecId deployedJobSpecId,
- long jobStartTime) throws Exception {
+ long jobStartTime, String jobStartTimeZoneId) throws Exception {
StartTasksFunction stf = new StartTasksFunction(deploymentId, jobId, planBytes, taskDescriptors,
- connectorPolicies, flags, jobParameters, deployedJobSpecId, jobStartTime);
+ connectorPolicies, flags, jobParameters, deployedJobSpecId, jobStartTime, jobStartTimeZoneId);
ipcHandle.send(-1, stf, null);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
index 02cd184..f179c36 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Joblet.java
@@ -105,11 +105,13 @@
private final long jobStartTime;
+ private final String jobStartTimeZoneId;
+
private final long maxWarnings;
public Joblet(NodeControllerService nodeController, DeploymentId deploymentId, JobId jobId,
INCServiceContext serviceCtx, ActivityClusterGraph acg,
- IJobletEventListenerFactory jobletEventListenerFactory, long jobStartTime) {
+ IJobletEventListenerFactory jobletEventListenerFactory, long jobStartTime, String jobStartTimeZoneId) {
this.nodeController = nodeController;
this.serviceCtx = serviceCtx;
this.deploymentId = deploymentId;
@@ -136,6 +138,7 @@
IGlobalJobDataFactory gjdf = acg.getGlobalJobDataFactory();
globalJobData = gjdf != null ? gjdf.createGlobalJobData(this) : null;
this.jobStartTime = jobStartTime;
+ this.jobStartTimeZoneId = jobStartTimeZoneId;
this.maxWarnings = acg.getMaxWarnings();
}
@@ -162,6 +165,11 @@
return jobStartTime;
}
+ @Override
+ public String getJobStartTimeZoneId() {
+ return jobStartTimeZoneId;
+ }
+
public void addTask(Task task) {
taskMap.put(task.getTaskAttemptId(), task);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
index 08a18f9..0721f66 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
@@ -74,7 +74,8 @@
ncs.getWorkQueue()
.schedule(new StartTasksWork(ncs, stf.getDeploymentId(), stf.getJobId(), stf.getPlanBytes(),
stf.getTaskDescriptors(), stf.getConnectorPolicies(), stf.getFlags(),
- stf.getJobParameters(), stf.getDeployedJobSpecId(), stf.getJobStartTime()));
+ stf.getJobParameters(), stf.getDeployedJobSpecId(), stf.getJobStartTime(),
+ stf.getJobStartTimeZoneId()));
return;
case ABORT_TASKS:
CCNCFunctions.AbortTasksFunction atf = (CCNCFunctions.AbortTasksFunction) fn;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index c774317..8c9cbfb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -710,7 +710,7 @@
@Override
public void run() {
try {
- tracer.instant("CurrentTime", traceCategory, Tracer.Scope.p, Tracer.dateTimeStamp());
+ tracer.instant("CurrentTime", traceCategory, ITracer.Scope.p, Tracer::dateTimeStamp);
} catch (Exception e) {
LOGGER.log(Level.WARN, "Exception tracing current time", e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 551d8fa..7d84fe9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -97,10 +97,13 @@
private final long jobStartTime;
+ private final String jobStartTimeZoneId;
+
public StartTasksWork(NodeControllerService ncs, DeploymentId deploymentId, JobId jobId, byte[] acgBytes,
List<TaskAttemptDescriptor> taskDescriptors,
Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, Set<JobFlag> flags,
- Map<byte[], byte[]> jobParameters, DeployedJobSpecId deployedJobSpecId, long jobStartTime) {
+ Map<byte[], byte[]> jobParameters, DeployedJobSpecId deployedJobSpecId, long jobStartTime,
+ String jobStartTimeZoneId) {
this.ncs = ncs;
this.deploymentId = deploymentId;
this.jobId = jobId;
@@ -111,6 +114,7 @@
this.flags = flags;
this.jobParameters = jobParameters;
this.jobStartTime = jobStartTime;
+ this.jobStartTimeZoneId = jobStartTimeZoneId;
}
@Override
@@ -215,7 +219,7 @@
}
listenerFactory.updateListenerJobParameters(ncs.createOrGetJobParameterByteStore(jobId));
}
- ji = new Joblet(ncs, deploymentId, jobId, appCtx, acg, listenerFactory, jobStartTime);
+ ji = new Joblet(ncs, deploymentId, jobId, appCtx, acg, listenerFactory, jobStartTime, jobStartTimeZoneId);
jobletMap.put(jobId, ji);
}
return ji;
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java
index 289e441..7c72d5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/FloatPointable.java
@@ -62,7 +62,7 @@
}
}
- private static int getIntBits(byte[] bytes, int start) {
+ public static int getIntBits(byte[] bytes, int start) {
return IntegerPointable.getInteger(bytes, start);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
index 828de18..49f6221 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -463,7 +463,8 @@
return false;
}
- builder.reset(out, Math.min(utfLen - byteIdx, (int) (codePointLength * 1.0 * byteIdx / codePointIdx)));
+ int estimateOutBytes = byteIdx == 0 ? codePointLength : (int) (codePointLength * 1.0 * byteIdx / codePointIdx);
+ builder.reset(out, Math.min(utfLen - byteIdx, estimateOutBytes));
codePointIdx = 0;
while (byteIdx < utfLen && codePointIdx < codePointLength) {
builder.appendCodePoint(src.codePointAt(src.getMetaDataLength() + byteIdx));
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java
index 452710e..d188f16 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java
@@ -85,8 +85,9 @@
for (int i = 0; i < diff; i++) {
out.writeByte(0);
}
- for (int i = ary.getLength() - 1; i >= actualDataStart + diff; i--) {
- ary.getByteArray()[i] = ary.getByteArray()[i - diff];
+ int firstCharIdx = startOffset + estimateMetaLen;
+ for (int dest = ary.getLength() - 1, src = dest - diff; src >= firstCharIdx; dest--, src--) {
+ ary.getByteArray()[dest] = ary.getByteArray()[src];
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
index dcf9a10..f088c7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
@@ -362,4 +362,37 @@
assertEquals(0, expected.compareTo(result));
}
+ @Test
+ public void testStringBuilder() throws Exception {
+ UTF8StringBuilder builder = new UTF8StringBuilder();
+ GrowableArray array = new GrowableArray();
+ UTF8StringPointable stringPointable = new UTF8StringPointable();
+ String writtenString;
+ int startIdx;
+
+ array.append(STRING_UTF8_MIX.getByteArray(), STRING_UTF8_MIX.getStartOffset(), STRING_UTF8_MIX.getLength());
+ String chunk = "ABC";
+ String originalString = chunk.repeat(699051);
+
+ // test grow path
+ startIdx = array.getLength();
+ builder.reset(array, 2);
+ builder.appendString(originalString);
+ builder.finish();
+ stringPointable.set(array.getByteArray(), startIdx, array.getLength());
+ writtenString = stringPointable.toString();
+ assertEquals(originalString, writtenString);
+
+ // test shrink path
+ array.reset();
+ array.append(STRING_UTF8_MIX.getByteArray(), STRING_UTF8_MIX.getStartOffset(), STRING_UTF8_MIX.getLength());
+ startIdx = array.getLength();
+ builder.reset(array, 699051);
+ builder.appendString(chunk);
+ builder.finish();
+ stringPointable.set(array.getByteArray(), startIdx, array.getLength());
+ writtenString = stringPointable.toString();
+ assertEquals(chunk, writtenString);
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
index 59ab2f4..e1acaee 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
@@ -116,7 +116,7 @@
try {
flush(writer);
} finally {
- tracer.durationE(tid, traceCategory, args);
+ tracer.durationE(traceCategory, tid, args);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/util/FrameUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/util/FrameUtils.java
index ec45437..39e223b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/util/FrameUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/util/FrameUtils.java
@@ -157,7 +157,7 @@
flushedBytes = frameTupleAppender.getBuffer().capacity();
long tid = tracer.durationB(name, cat, args);
frameTupleAppender.write(writer, true);
- tracer.durationE(tid, cat, args);
+ tracer.durationE(cat, tid, args);
if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
throw HyracksDataException.create(ErrorCode.TUPLE_CANNOT_FIT_INTO_EMPTY_FRAME,
tupleAccessor.getTupleLength(tIndex));
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinUtil.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinUtil.java
index 398dee9..7cee242 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinUtil.java
@@ -28,16 +28,21 @@
private HybridHashJoinUtil() {
}
+ public enum SIDE {
+ BUILD,
+ PROBE
+ }
+
/**
* Prints out the detailed information for partitions: in-memory and spilled partitions.
* This method exists for a debug purpose.
*/
- public String printPartitionInfo(BitSet spilledStatus, OptimizedHybridHashJoin.SIDE whichSide, int numOfPartitions,
- int[] probePSizeInTups, int[] buildPSizeInTups, RunFileWriter[] probeRFWriters,
- RunFileWriter[] buildRFWriters, IPartitionedTupleBufferManager bufferManager) {
+ public String printPartitionInfo(BitSet spilledStatus, SIDE whichSide, int numOfPartitions, int[] probePSizeInTups,
+ int[] buildPSizeInTups, RunFileWriter[] probeRFWriters, RunFileWriter[] buildRFWriters,
+ IPartitionedTupleBufferManager bufferManager) {
StringBuilder buf = new StringBuilder();
buf.append(">>> " + this + " " + Thread.currentThread().getId() + " printInfo():" + "\n");
- if (whichSide == OptimizedHybridHashJoin.SIDE.BUILD) {
+ if (whichSide == SIDE.BUILD) {
buf.append("BUILD side" + "\n");
} else {
buf.append("PROBE side" + "\n");
@@ -49,7 +54,7 @@
int spilledPartByteSize = 0;
for (int pid = spilledStatus.nextSetBit(0); pid >= 0 && pid < numOfPartitions; pid =
spilledStatus.nextSetBit(pid + 1)) {
- if (whichSide == OptimizedHybridHashJoin.SIDE.BUILD) {
+ if (whichSide == SIDE.BUILD) {
spilledTupleCount += buildPSizeInTups[pid];
spilledPartByteSize += buildRFWriters[pid].getFileSize();
buf.append("part:\t" + pid + "\t#tuple:\t" + buildPSizeInTups[pid] + "\tsize(MB):\t"
@@ -70,7 +75,7 @@
int inMemoryPartByteSize = 0;
for (int pid = spilledStatus.nextClearBit(0); pid >= 0 && pid < numOfPartitions; pid =
spilledStatus.nextClearBit(pid + 1)) {
- if (whichSide == OptimizedHybridHashJoin.SIDE.BUILD) {
+ if (whichSide == SIDE.BUILD) {
inMemoryTupleCount += buildPSizeInTups[pid];
inMemoryPartByteSize += bufferManager.getPhysicalSize(pid);
} else {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
index cb63b6a..4c2019e 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -28,7 +28,6 @@
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksFrameMgrContext;
import org.apache.hyracks.api.dataflow.value.IMissingWriter;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluator;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparator;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -59,7 +58,6 @@
private final ISerializableTable table;
private final TuplePointer storedTuplePointer;
private final boolean reverseOutputOrder; //Should we reverse the order of tuples, we are writing in output
- private final IPredicateEvaluator predEvaluator;
private final TupleInFrameListAccessor tupleAccessor;
// To release frames
private final ISimpleFrameBufferManager bufferManager;
@@ -70,17 +68,16 @@
public InMemoryHashJoin(IHyracksFrameMgrContext ctx, FrameTupleAccessor accessorProbe,
ITuplePartitionComputer tpcProbe, FrameTupleAccessor accessorBuild, RecordDescriptor rDBuild,
ITuplePartitionComputer tpcBuild, boolean isLeftOuter, IMissingWriter[] missingWritersBuild,
- ISerializableTable table, IPredicateEvaluator predEval, ISimpleFrameBufferManager bufferManager)
- throws HyracksDataException {
+ ISerializableTable table, ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
this(ctx, accessorProbe, tpcProbe, accessorBuild, rDBuild, tpcBuild, isLeftOuter, missingWritersBuild, table,
- predEval, false, bufferManager);
+ false, bufferManager);
}
public InMemoryHashJoin(IHyracksFrameMgrContext ctx, FrameTupleAccessor accessorProbe,
ITuplePartitionComputer tpcProbe, FrameTupleAccessor accessorBuild, RecordDescriptor rDBuild,
ITuplePartitionComputer tpcBuild, boolean isLeftOuter, IMissingWriter[] missingWritersBuild,
- ISerializableTable table, IPredicateEvaluator predEval, boolean reverse,
- ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
+ ISerializableTable table, boolean reverse, ISimpleFrameBufferManager bufferManager)
+ throws HyracksDataException {
this.table = table;
storedTuplePointer = new TuplePointer();
buffers = new ArrayList<>();
@@ -89,7 +86,6 @@
this.accessorProbe = accessorProbe;
this.tpcProbe = tpcProbe;
appender = new FrameTupleAppender(new VSizeFrame(ctx));
- predEvaluator = predEval;
this.isLeftOuter = isLeftOuter;
if (isLeftOuter) {
int fieldCountOuter = accessorBuild.getFieldCount();
@@ -178,11 +174,8 @@
accessorBuild.reset(buffers.get(bIndex));
int c = tpComparator.compare(accessorProbe, tid, accessorBuild, tIndex);
if (c == 0) {
- boolean predEval = evaluatePredicate(tid, tIndex);
- if (predEval) {
- matchFound = true;
- appendToResult(tid, tIndex, writer);
- }
+ matchFound = true;
+ appendToResult(tid, tIndex, writer);
}
}
}
@@ -228,14 +221,6 @@
table.close();
}
- private boolean evaluatePredicate(int tIx1, int tIx2) {
- if (reverseOutputOrder) { //Role Reversal Optimization is triggered
- return (predEvaluator == null) || predEvaluator.evaluate(accessorBuild, tIx2, accessorProbe, tIx1);
- } else {
- return (predEvaluator == null) || predEvaluator.evaluate(accessorProbe, tIx1, accessorBuild, tIx2);
- }
- }
-
private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException {
if (reverseOutputOrder) {
FrameUtils.appendConcatToWriter(writer, appender, accessorBuild, buildSidetIx, accessorProbe, probeSidetIx);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index 33976a8..f89ccb0 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -32,8 +32,6 @@
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import org.apache.hyracks.api.dataflow.value.IMissingWriter;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluator;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -63,7 +61,6 @@
private final IBinaryHashFunctionFactory[] hashFunctionFactories0;
private final IBinaryHashFunctionFactory[] hashFunctionFactories1;
private final ITuplePairComparatorFactory comparatorFactory;
- private final IPredicateEvaluatorFactory predEvaluatorFactory;
private final boolean isLeftOuter;
private final IMissingWriterFactory[] nonMatchWriterFactories;
private final int tableSize;
@@ -73,23 +70,21 @@
public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
IBinaryHashFunctionFactory[] hashFunctionFactories0, IBinaryHashFunctionFactory[] hashFunctionFactories1,
ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int tableSize,
- IPredicateEvaluatorFactory predEvalFactory, int memSizeInFrames) {
- this(spec, keys0, keys1, hashFunctionFactories0, hashFunctionFactories1, comparatorFactory, predEvalFactory,
- recordDescriptor, false, null, tableSize, memSizeInFrames);
+ int memSizeInFrames) {
+ this(spec, keys0, keys1, hashFunctionFactories0, hashFunctionFactories1, comparatorFactory, recordDescriptor,
+ false, null, tableSize, memSizeInFrames);
}
public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
IBinaryHashFunctionFactory[] hashFunctionFactories0, IBinaryHashFunctionFactory[] hashFunctionFactories1,
- ITuplePairComparatorFactory comparatorFactory, IPredicateEvaluatorFactory predEvalFactory,
- RecordDescriptor recordDescriptor, boolean isLeftOuter, IMissingWriterFactory[] missingWriterFactories1,
- int tableSize, int memSizeInFrames) {
+ ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, boolean isLeftOuter,
+ IMissingWriterFactory[] missingWriterFactories1, int tableSize, int memSizeInFrames) {
super(spec, 2, 1);
this.keys0 = keys0;
this.keys1 = keys1;
this.hashFunctionFactories0 = hashFunctionFactories0;
this.hashFunctionFactories1 = hashFunctionFactories1;
this.comparatorFactory = comparatorFactory;
- this.predEvaluatorFactory = predEvalFactory;
this.outRecDescs[0] = recordDescriptor;
this.isLeftOuter = isLeftOuter;
this.nonMatchWriterFactories = missingWriterFactories1;
@@ -159,8 +154,6 @@
} else {
nullWriters1 = null;
}
- final IPredicateEvaluator predEvaluator =
- (predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
final int memSizeInBytes = memSizeInFrames * jobletCtx.getInitialFrameSize();
final IDeallocatableFramePool framePool = new DeallocatableFramePool(jobletCtx, memSizeInBytes);
@@ -178,8 +171,7 @@
state = new HashBuildTaskState(jobletCtx.getJobId(), new TaskId(getActivityId(), partition));
ISerializableTable table = new SerializableHashTable(tableSize, jobletCtx, bufferManager);
state.joiner = new InMemoryHashJoin(jobletCtx, new FrameTupleAccessor(rd0), hpc0,
- new FrameTupleAccessor(rd1), rd1, hpc1, isLeftOuter, nullWriters1, table, predEvaluator,
- bufferManager);
+ new FrameTupleAccessor(rd1), rd1, hpc1, isLeftOuter, nullWriters1, table, bufferManager);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
index 03ff72f..414ae25 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksJobletContext;
import org.apache.hyracks.api.dataflow.value.IMissingWriter;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluator;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparator;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -63,22 +62,20 @@
private final RunFileWriter runFileWriter;
private final boolean isLeftOuter;
private final ArrayTupleBuilder missingTupleBuilder;
- private final IPredicateEvaluator predEvaluator;
- // Added for handling correct calling for predicate-evaluator upon recursive calls
+ // Added for handling correct calling of recursive calls
// (in OptimizedHybridHashJoin) that cause role-reversal
private final boolean isReversed;
private final BufferInfo tempInfo = new BufferInfo(null, -1, -1);
private final BitSet outerMatchLOJ;
public NestedLoopJoin(IHyracksJobletContext jobletContext, FrameTupleAccessor accessorOuter,
- FrameTupleAccessor accessorInner, int memBudgetInFrames, IPredicateEvaluator predEval, boolean isLeftOuter,
+ FrameTupleAccessor accessorInner, int memBudgetInFrames, boolean isLeftOuter,
IMissingWriter[] missingWriters) throws HyracksDataException {
- this(jobletContext, accessorOuter, accessorInner, memBudgetInFrames, predEval, isLeftOuter, missingWriters,
- false);
+ this(jobletContext, accessorOuter, accessorInner, memBudgetInFrames, isLeftOuter, missingWriters, false);
}
public NestedLoopJoin(IHyracksJobletContext jobletContext, FrameTupleAccessor accessorOuter,
- FrameTupleAccessor accessorInner, int memBudgetInFrames, IPredicateEvaluator predEval, boolean isLeftOuter,
+ FrameTupleAccessor accessorInner, int memBudgetInFrames, boolean isLeftOuter,
IMissingWriter[] missingWriters, boolean isReversed) throws HyracksDataException {
this.accessorInner = accessorInner;
this.accessorOuter = accessorOuter;
@@ -97,7 +94,6 @@
new VariableFramePool(jobletContext, outerBufferMngrMemBudgetInBytes), FrameFreeSlotPolicyFactory
.createFreeSlotPolicy(EnumFreeSlotPolicy.LAST_FIT, outerBufferMngrMemBudgetInFrames));
- this.predEvaluator = predEval;
this.isLeftOuter = isLeftOuter;
if (isLeftOuter) {
if (isReversed) {
@@ -202,8 +198,7 @@
boolean matchFound = false;
for (int j = 0; j < innerTupleCount; ++j) {
int c = tpComparator.compare(accessorOuter, i, accessorInner, j);
- boolean prdEval = evaluatePredicate(i, j);
- if (c == 0 && prdEval) {
+ if (c == 0) {
matchFound = true;
appendToResults(i, j, writer);
}
@@ -214,14 +209,6 @@
}
}
- private boolean evaluatePredicate(int tIx1, int tIx2) {
- if (isReversed) { //Role Reversal Optimization is triggered
- return ((predEvaluator == null) || predEvaluator.evaluate(accessorInner, tIx2, accessorOuter, tIx1));
- } else {
- return ((predEvaluator == null) || predEvaluator.evaluate(accessorOuter, tIx1, accessorInner, tIx2));
- }
- }
-
private void appendToResults(int outerTupleId, int innerTupleId, IFrameWriter writer) throws HyracksDataException {
if (isReversed) {
appendResultToFrame(accessorInner, innerTupleId, accessorOuter, outerTupleId, writer);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index 1de8094..24e1b45 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -29,8 +29,6 @@
import org.apache.hyracks.api.dataflow.TaskId;
import org.apache.hyracks.api.dataflow.value.IMissingWriter;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluator;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -52,25 +50,16 @@
private static final long serialVersionUID = 1L;
private final ITuplePairComparatorFactory comparatorFactory;
private final int memSize;
- private final IPredicateEvaluatorFactory predEvaluatorFactory;
private final boolean isLeftOuter;
private final IMissingWriterFactory[] nullWriterFactories1;
public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
boolean isLeftOuter, IMissingWriterFactory[] nullWriterFactories1) {
- this(spec, comparatorFactory, recordDescriptor, memSize, null, isLeftOuter, nullWriterFactories1);
- }
-
- public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
- ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
- IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter,
- IMissingWriterFactory[] nullWriterFactories1) {
super(spec, 2, 1);
this.comparatorFactory = comparatorFactory;
this.outRecDescs[0] = recordDescriptor;
this.memSize = memSize;
- this.predEvaluatorFactory = predEvalFactory;
this.isLeftOuter = isLeftOuter;
this.nullWriterFactories1 = nullWriterFactories1;
}
@@ -117,8 +106,6 @@
final IHyracksJobletContext jobletCtx = ctx.getJobletContext();
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
- final IPredicateEvaluator predEvaluator =
- (predEvaluatorFactory != null) ? predEvaluatorFactory.createPredicateEvaluator() : null;
final IMissingWriter[] nullWriters1 = isLeftOuter ? new IMissingWriter[nullWriterFactories1.length] : null;
if (isLeftOuter) {
@@ -134,8 +121,7 @@
public void open() throws HyracksDataException {
state = new JoinCacheTaskState(jobletCtx.getJobId(), new TaskId(getActivityId(), partition));
state.joiner = new NestedLoopJoin(jobletCtx, new FrameTupleAccessor(rd0),
- new FrameTupleAccessor(rd1), memSize, predEvaluator, isLeftOuter, nullWriters1);
-
+ new FrameTupleAccessor(rd1), memSize, isLeftOuter, nullWriters1);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index f5eb4f7..7a9bb25 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -59,11 +59,6 @@
// Used for special probe BigObject which can not be held into the Join memory
private FrameTupleAppender bigFrameAppender;
- public enum SIDE {
- BUILD,
- PROBE
- }
-
private final IHyracksJobletContext jobletCtx;
private final String buildRelName;
@@ -74,7 +69,8 @@
private final RecordDescriptor probeRd;
private final RunFileWriter[] buildRFWriters; //writing spilled build partitions
private final RunFileWriter[] probeRFWriters; //writing spilled probe partitions
- private final IPredicateEvaluator predEvaluator;
+ private final IPredicateEvaluator buildPredEval;
+ private final IPredicateEvaluator probePredEval;
private final boolean isLeftOuter;
private final IMissingWriter[] nonMatchWriters;
private final BitSet spilledStatus; //0=resident, 1=spilled
@@ -98,8 +94,8 @@
public OptimizedHybridHashJoin(IHyracksJobletContext jobletCtx, int memSizeInFrames, int numOfPartitions,
String probeRelName, String buildRelName, RecordDescriptor probeRd, RecordDescriptor buildRd,
- ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc, IPredicateEvaluator predEval,
- boolean isLeftOuter, IMissingWriterFactory[] nullWriterFactories1) {
+ ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc, IPredicateEvaluator probePredEval,
+ IPredicateEvaluator buildPredEval, boolean isLeftOuter, IMissingWriterFactory[] nullWriterFactories1) {
this.jobletCtx = jobletCtx;
this.memSizeInFrames = memSizeInFrames;
this.buildRd = buildRd;
@@ -113,8 +109,12 @@
this.probeRFWriters = new RunFileWriter[numOfPartitions];
this.accessorBuild = new FrameTupleAccessor(buildRd);
this.accessorProbe = new FrameTupleAccessor(probeRd);
- this.predEvaluator = predEval;
this.isLeftOuter = isLeftOuter;
+ if (isLeftOuter && probePredEval != null) {
+ throw new IllegalStateException();
+ }
+ this.buildPredEval = buildPredEval;
+ this.probePredEval = probePredEval;
this.isReversed = false;
this.spilledStatus = new BitSet(numOfPartitions);
this.nonMatchWriters = isLeftOuter ? new IMissingWriter[nullWriterFactories1.length] : null;
@@ -141,11 +141,12 @@
accessorBuild.reset(buffer);
int tupleCount = accessorBuild.getTupleCount();
for (int i = 0; i < tupleCount; ++i) {
- int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
- processTupleBuildPhase(i, pid);
- buildPSizeInTups[pid]++;
+ if (buildPredEval == null || buildPredEval.evaluate(accessorBuild, i)) {
+ int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
+ processTupleBuildPhase(i, pid);
+ buildPSizeInTups[pid]++;
+ }
}
-
}
private void processTupleBuildPhase(int tid, int pid) throws HyracksDataException {
@@ -217,8 +218,8 @@
ISerializableTable table = new SerializableHashTable(inMemTupCount, jobletCtx, bufferManagerForHashTable);
this.inMemJoiner = new InMemoryHashJoin(jobletCtx, new FrameTupleAccessor(probeRd), probeHpc,
- new FrameTupleAccessor(buildRd), buildRd, buildHpc, isLeftOuter, nonMatchWriters, table, predEvaluator,
- isReversed, bufferManagerForHashTable);
+ new FrameTupleAccessor(buildRd), buildRd, buildHpc, isLeftOuter, nonMatchWriters, table, isReversed,
+ bufferManagerForHashTable);
buildHashTable();
}
@@ -473,22 +474,28 @@
public void probe(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
accessorProbe.reset(buffer);
int tupleCount = accessorProbe.getTupleCount();
-
- if (isBuildRelAllInMemory()) {
- inMemJoiner.join(buffer, writer);
- return;
- }
inMemJoiner.resetAccessorProbe(accessorProbe);
- for (int i = 0; i < tupleCount; ++i) {
- int pid = probeHpc.partition(accessorProbe, i, numOfPartitions);
-
- if (buildPSizeInTups[pid] > 0 || isLeftOuter) { //Tuple has potential match from previous phase
- if (spilledStatus.get(pid)) { //pid is Spilled
- processTupleProbePhase(i, pid);
- } else { //pid is Resident
+ if (isBuildRelAllInMemory()) {
+ for (int i = 0; i < tupleCount; ++i) {
+ // NOTE: probePredEval is guaranteed to be 'null' for outer join and in case of role reversal
+ if (probePredEval == null || probePredEval.evaluate(accessorProbe, i)) {
inMemJoiner.join(i, writer);
}
- probePSizeInTups[pid]++;
+ }
+ } else {
+ for (int i = 0; i < tupleCount; ++i) {
+ // NOTE: probePredEval is guaranteed to be 'null' for outer join and in case of role reversal
+ if (probePredEval == null || probePredEval.evaluate(accessorProbe, i)) {
+ int pid = probeHpc.partition(accessorProbe, i, numOfPartitions);
+ if (buildPSizeInTups[pid] > 0 || isLeftOuter) { //Tuple has potential match from previous phase
+ if (spilledStatus.get(pid)) { //pid is Spilled
+ processTupleProbePhase(i, pid);
+ } else { //pid is Resident
+ inMemJoiner.join(i, writer);
+ }
+ probePSizeInTups[pid]++;
+ }
+ }
}
}
}
@@ -600,7 +607,10 @@
return bufferManager.getPhysicalSize(pid);
}
- public void setIsReversed(boolean b) {
- this.isReversed = b;
+ public void setIsReversed(boolean reversed) {
+ if (reversed && (buildPredEval != null || probePredEval != null)) {
+ throw new IllegalStateException();
+ }
+ this.isReversed = reversed;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index bb79981..555e8fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -131,7 +131,8 @@
private final IBinaryHashFunctionFamily[] buildHashFunctionFactories;
private final ITuplePairComparatorFactory tuplePairComparatorFactoryProbe2Build; //For HHJ & NLJ in probe
private final ITuplePairComparatorFactory tuplePairComparatorFactoryBuild2Probe; //For HHJ & NLJ in probe
- private final IPredicateEvaluatorFactory predEvaluatorFactory;
+ private final IPredicateEvaluatorFactory probePredEvalFactory;
+ private final IPredicateEvaluatorFactory buildPredEvalFactory;
private final boolean isLeftOuter;
private final IMissingWriterFactory[] nonMatchWriterFactories;
@@ -148,8 +149,9 @@
IBinaryHashFunctionFamily[] propHashFunctionFactories,
IBinaryHashFunctionFamily[] buildHashFunctionFactories, RecordDescriptor recordDescriptor,
ITuplePairComparatorFactory tupPaircomparatorFactory01,
- ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory,
- boolean isLeftOuter, IMissingWriterFactory[] nonMatchWriterFactories) {
+ ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvalFactory0,
+ IPredicateEvaluatorFactory predEvalFactory1, boolean isLeftOuter,
+ IMissingWriterFactory[] nonMatchWriterFactories) {
super(spec, 2, 1);
this.memSizeInFrames = memSizeInFrames;
this.inputsize0 = inputsize0;
@@ -161,7 +163,8 @@
this.tuplePairComparatorFactoryProbe2Build = tupPaircomparatorFactory01;
this.tuplePairComparatorFactoryBuild2Probe = tupPaircomparatorFactory10;
outRecDescs[0] = recordDescriptor;
- this.predEvaluatorFactory = predEvaluatorFactory;
+ this.probePredEvalFactory = predEvalFactory0;
+ this.buildPredEvalFactory = predEvalFactory1;
this.isLeftOuter = isLeftOuter;
this.nonMatchWriterFactories = nonMatchWriterFactories;
}
@@ -171,10 +174,11 @@
IBinaryHashFunctionFamily[] propHashFunctionFactories,
IBinaryHashFunctionFamily[] buildHashFunctionFactories, RecordDescriptor recordDescriptor,
ITuplePairComparatorFactory tupPaircomparatorFactory01,
- ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory) {
+ ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvalFactory0,
+ IPredicateEvaluatorFactory predEvalFactory1) {
this(spec, memSizeInFrames, inputsize0, factor, keys0, keys1, propHashFunctionFactories,
buildHashFunctionFactories, recordDescriptor, tupPaircomparatorFactory01, tupPaircomparatorFactory10,
- predEvaluatorFactory, false, null);
+ predEvalFactory0, predEvalFactory1, false, null);
}
@Override
@@ -199,20 +203,21 @@
//memorySize is the memory for join (we have already excluded the 2 buffers for in/out)
private static int getNumberOfPartitions(int memorySize, int buildSize, double factor, int nPartitions)
throws HyracksDataException {
- int numberOfPartitions = 0;
if (memorySize <= 2) {
throw new HyracksDataException("Not enough memory is available for Hybrid Hash Join.");
}
- if (memorySize > buildSize * factor) {
- // We will switch to in-Mem HJ eventually: create two big partitions.
- // We set 2 (not 1) to avoid a corner case where the only partition may be spilled to the disk.
- // This may happen since this formula doesn't consider the hash table size. If this is the case,
- // we will do a nested loop join after some iterations. But, this is not effective.
- return 2;
+ int minimumNumberOfPartitions = Math.min(20, memorySize);
+ if (buildSize < 0 || memorySize > buildSize * factor) {
+
+ return minimumNumberOfPartitions;
}
- numberOfPartitions = (int) (Math.ceil((buildSize * factor / nPartitions - memorySize) / (memorySize - 1)));
- numberOfPartitions = Math.max(2, numberOfPartitions);
- if (numberOfPartitions > memorySize) {
+ // Two frames are already excluded from the memorySize for taking the input and output into account. That
+ // makes the denominator in the following formula to be different than the denominator in original Hybrid Hash
+ // Join which is memorySize - 1. This formula gives the total number of partitions, the spilled partitions
+ // and the memory-resident partition ( + 1 in formula is for taking the memory-resident partition into account).
+ int numberOfPartitions = (int) (Math.ceil((buildSize * factor / nPartitions - memorySize) / (memorySize))) + 1;
+ numberOfPartitions = Math.max(minimumNumberOfPartitions, numberOfPartitions);
+ if (numberOfPartitions > memorySize) { // Considers applying Grace Hash Join instead of Hybrid Hash Join.
numberOfPartitions = (int) Math.ceil(Math.sqrt(buildSize * factor / nPartitions));
return Math.max(2, Math.min(numberOfPartitions, memorySize));
}
@@ -267,8 +272,10 @@
final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(probeAid, 0);
- final IPredicateEvaluator predEvaluator =
- (predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
+ final IPredicateEvaluator buildPredEval =
+ (buildPredEvalFactory == null ? null : buildPredEvalFactory.createPredicateEvaluator());
+ final IPredicateEvaluator probePredEval = (probePredEvalFactory == null || isLeftOuter ? null
+ : probePredEvalFactory.createPredicateEvaluator());
return new AbstractUnaryInputSinkOperatorNodePushable() {
private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(
@@ -292,7 +299,7 @@
getNumberOfPartitions(state.memForJoin, inputsize0, fudgeFactor, nPartitions);
state.hybridHJ = new OptimizedHybridHashJoin(ctx.getJobletContext(), state.memForJoin,
state.numOfPartitions, PROBE_REL, BUILD_REL, probeRd, buildRd, probeHpc, buildHpc,
- predEvaluator, isLeftOuter, nonMatchWriterFactories);
+ probePredEval, buildPredEval, isLeftOuter, nonMatchWriterFactories);
state.hybridHJ.initBuild();
if (LOGGER.isTraceEnabled()) {
@@ -365,8 +372,6 @@
final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final ITuplePairComparator probComp = tuplePairComparatorFactoryProbe2Build.createTuplePairComparator(ctx);
final ITuplePairComparator buildComp = tuplePairComparatorFactoryBuild2Probe.createTuplePairComparator(ctx);
- final IPredicateEvaluator predEvaluator =
- predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator();
final IMissingWriter[] nonMatchWriter =
isLeftOuter ? new IMissingWriter[nonMatchWriterFactories.length] : null;
@@ -592,7 +597,7 @@
OptimizedHybridHashJoin rHHj;
int n = getNumberOfPartitions(state.memForJoin, tableSize, fudgeFactor, nPartitions);
rHHj = new OptimizedHybridHashJoin(jobletCtx, state.memForJoin, n, PROBE_REL, BUILD_REL, probeRd,
- buildRd, probeHpc, buildHpc, predEvaluator, isLeftOuter, nonMatchWriterFactories); //checked-confirmed
+ buildRd, probeHpc, buildHpc, null, null, isLeftOuter, nonMatchWriterFactories); //checked-confirmed
rHHj.setIsReversed(isReversed);
try {
@@ -747,7 +752,7 @@
ISerializableTable table = new SerializableHashTable(tabSize, jobletCtx, bufferManager);
InMemoryHashJoin joiner = new InMemoryHashJoin(jobletCtx, new FrameTupleAccessor(probeRDesc),
hpcRepProbe, new FrameTupleAccessor(buildRDesc), buildRDesc, hpcRepBuild, isLeftOuter,
- nonMatchWriter, table, predEvaluator, isReversed, bufferManager);
+ nonMatchWriter, table, isReversed, bufferManager);
joiner.setComparator(comp);
try {
bReader.open();
@@ -804,8 +809,7 @@
boolean isReversed = outerRd == buildRd && innerRd == probeRd;
ITuplePairComparator nljComptorOuterInner = isReversed ? buildComp : probComp;
NestedLoopJoin nlj = new NestedLoopJoin(jobletCtx, new FrameTupleAccessor(outerRd),
- new FrameTupleAccessor(innerRd), memorySize, predEvaluator, isLeftOuter, nonMatchWriter,
- isReversed);
+ new FrameTupleAccessor(innerRd), memorySize, isLeftOuter, nonMatchWriter, isReversed);
nlj.setComparator(nljComptorOuterInner);
IFrame cacheBuff = new VSizeFrame(jobletCtx);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
index 46d62a7..08f15b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
@@ -272,12 +272,7 @@
isSorted = true;
}
- private static final Comparator<IResetableComparable> entryComparator = new Comparator<IResetableComparable>() {
- @Override
- public int compare(IResetableComparable o1, IResetableComparable o2) {
- return o1.compareTo(o2);
- }
- };
+ private static final Comparator<IResetableComparable> entryComparator = Comparable::compareTo;
@Override
public void close() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index ab3db7d..20e8eae 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -143,7 +143,7 @@
new IndexDataflowHelperFactory(storageManager, btreeSplitProvider);
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, lowKeyFields,
highKeyFields, true, true, dataflowHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 44b982b..26ddbd6 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -170,7 +170,7 @@
new IndexDataflowHelperFactory(storageManager, secondarySplitProvider);
BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
secondaryLowKeyFields, secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
@@ -188,7 +188,7 @@
new IndexDataflowHelperFactory(storageManager, primarySplitProvider);
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java
index 7ba81e1..3aa3d23 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/RuntimeContext.java
@@ -48,12 +48,7 @@
private ILocalResourceRepository localResourceRepository;
private IResourceLifecycleManager<IIndex> lcManager;
private ResourceIdFactory resourceIdFactory;
- private ThreadFactory threadFactory = new ThreadFactory() {
- @Override
- public Thread newThread(Runnable r) {
- return new Thread(r);
- }
- };
+ private ThreadFactory threadFactory = Thread::new;
public RuntimeContext(INCServiceContext appCtx) throws HyracksDataException {
fileMapManager = new FileMapManager();
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index 283b352..eac332d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -185,7 +185,7 @@
// scan primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
// sort based on secondary keys
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
index 336204c..9474b12 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -76,7 +76,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
DataSetConstants.primaryRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false,
- false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -98,12 +98,12 @@
@Override
protected IResourceFactory createPrimaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.primaryTypeTraits,
- DataSetConstants.primaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.primaryComparatorFactories, pageManagerFactory, null, null);
}
@Override
protected IResourceFactory createSecondaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.secondaryTypeTraits,
- DataSetConstants.secondaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.secondaryComparatorFactories, pageManagerFactory, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
index 870cfe3..af043ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -81,7 +81,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
DataSetConstants.primaryRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false,
- false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -103,12 +103,12 @@
@Override
protected IResourceFactory createPrimaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.primaryTypeTraits,
- DataSetConstants.primaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.primaryComparatorFactories, pageManagerFactory, null, null);
}
@Override
protected IResourceFactory createSecondaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.secondaryTypeTraits,
- DataSetConstants.secondaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.secondaryComparatorFactories, pageManagerFactory, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
index 5cffd95..a1f53bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -67,12 +67,12 @@
@Override
protected IResourceFactory createPrimaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.primaryTypeTraits,
- DataSetConstants.primaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.primaryComparatorFactories, pageManagerFactory, null, null);
}
@Override
protected IResourceFactory createSecondaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.secondaryTypeTraits,
- DataSetConstants.secondaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.secondaryComparatorFactories, pageManagerFactory, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
index b2060b6..93ab78d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -83,9 +83,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+ secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -95,9 +96,10 @@
int[] primaryHighKeyFields = { 1 };
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -121,12 +123,12 @@
@Override
protected IResourceFactory createPrimaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.primaryTypeTraits,
- DataSetConstants.primaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.primaryComparatorFactories, pageManagerFactory, null, null);
}
@Override
protected IResourceFactory createSecondaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.secondaryTypeTraits,
- DataSetConstants.secondaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.secondaryComparatorFactories, pageManagerFactory, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
index e73a11e..a5a0bda 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -82,9 +82,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+ secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -94,9 +95,10 @@
// coming from secondary index
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
@@ -117,12 +119,12 @@
@Override
protected IResourceFactory createPrimaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.primaryTypeTraits,
- DataSetConstants.primaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.primaryComparatorFactories, pageManagerFactory, null, null);
}
@Override
protected IResourceFactory createSecondaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.secondaryTypeTraits,
- DataSetConstants.secondaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.secondaryComparatorFactories, pageManagerFactory, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
index a333dc4..3691cd4 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
@@ -82,9 +82,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+ secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -94,9 +95,10 @@
int[] primaryHighKeyFields = { 1 };
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -120,12 +122,12 @@
@Override
protected IResourceFactory createPrimaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.primaryTypeTraits,
- DataSetConstants.primaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.primaryComparatorFactories, pageManagerFactory, null, null);
}
@Override
protected IResourceFactory createSecondaryResourceFactory() {
return new BTreeResourceFactory(storageManager, DataSetConstants.secondaryTypeTraits,
- DataSetConstants.secondaryComparatorFactories, pageManagerFactory);
+ DataSetConstants.secondaryComparatorFactories, pageManagerFactory, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index d46f5f5..bcd71a3 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -49,6 +49,6 @@
getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
MERGE_POLICY_PROPERTIES, DURABLE, bloomFilterKeyFields,
LSMTreeOperatorTestHelper.DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, btreefields,
- NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null);
+ NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
index 9d3ba5c..da1441c 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
@@ -89,10 +89,10 @@
int[] lowKeyFields = { 0 };
int[] highKeyFields = { 1 };
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
- new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryAndFilterRecDesc, lowKeyFields,
- highKeyFields, true, true, primaryHelperFactory, false, false,
- NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null, true);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+ DataSetConstants.primaryAndFilterRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory,
+ false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null,
+ true, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -147,7 +147,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
DataSetConstants.primaryAndFilterRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory,
false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, minFilterFields,
- maxFilterFields, true);
+ maxFilterFields, true, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
index 97a4746..bed4794 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
@@ -90,10 +90,10 @@
int[] secondaryHighKeyFields = { 1 };
// search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
- new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryWithFilterRecDesc,
- secondaryLowKeyFields, secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false,
- NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null, true);
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+ DataSetConstants.secondaryWithFilterRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+ secondaryHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
+ NoOpOperationCallbackFactory.INSTANCE, null, null, true, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -103,10 +103,11 @@
int[] maxFilterFields = { 3 };
// search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
- DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
- primaryHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
- NoOpOperationCallbackFactory.INSTANCE, minFilterFields, maxFilterFields, false);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, primaryHelperFactory, false, false,
+ NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, minFilterFields,
+ maxFilterFields, false, NoopMissingWriterFactory.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index 3b3fc1e..1725ea3 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -54,6 +54,6 @@
NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory, getVirtualBufferCacheProvider(),
SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE,
btreeComparatorFactories, valueProviderFactories, rtreePolicyType, linearizerCmpFactory, null,
- btreeFields, IS_POINT_MBR, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
+ btreeFields, IS_POINT_MBR, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
index 0bb483a..28b9e88 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -54,6 +54,6 @@
NoOpIOOperationCallbackFactory.INSTANCE, NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory,
getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
MERGE_POLICY_PROPERTIES, DURABLE, valueProviderFactories, rtreePolicyType, linearizerCmpFactory,
- btreeFields, IS_POINT_MBR, btreeComparatorFactories);
+ btreeFields, IS_POINT_MBR, btreeComparatorFactories, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
index 04179c2..6a31962 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
@@ -95,7 +95,7 @@
int[] keyFields = { 0, 1, 2, 3 };
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec,
secondaryWithFilterRecDesc, keyFields, true, true, secondaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index 2d092f1..f849bc8 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -228,7 +228,7 @@
protected void createPrimaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
btreeFactory = new BTreeResourceFactory(storageManager, primaryTypeTraits, primaryComparatorFactories,
- pageManagerFactory);
+ pageManagerFactory, null, null);
IIndexBuilderFactory indexBuilderFactory =
new IndexBuilderFactory(storageManager, primarySplitProvider, btreeFactory, false);
IndexCreateOperatorDescriptor primaryCreateOp = new IndexCreateOperatorDescriptor(spec, indexBuilderFactory);
@@ -320,7 +320,7 @@
// scan primary index
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
// load secondary index
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
index 43d685e..1981c1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
@@ -38,6 +38,6 @@
int[] btreeFields, ITypeTraits[] secondaryTypeTraits,
IBinaryComparatorFactory[] secondaryComparatorFactories, IMetadataPageManagerFactory pageManagerFactory) {
return new RTreeResourceFactory(storageManager, secondaryTypeTraits, secondaryComparatorFactories,
- pageManagerFactory, valueProviderFactories, rtreePolicyType);
+ pageManagerFactory, valueProviderFactories, rtreePolicyType, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
index 784cd10..0fcf892 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
@@ -88,7 +88,7 @@
int[] keyFields = { 0, 1, 2, 3 };
RTreeSearchOperatorDescriptor secondarySearchOp =
new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
- false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
// fifth field from the tuples coming from secondary index
int[] primaryLowKeyFields = { 4 };
@@ -97,7 +97,7 @@
// search primary index
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
index ff44d10..3cfef19 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
@@ -86,7 +86,7 @@
int[] keyFields = null;
RTreeSearchOperatorDescriptor secondarySearchOp =
new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
- false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index e234322..8ea0701 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -86,7 +86,7 @@
int[] keyFields = { 0, 1, 2, 3 };
RTreeSearchOperatorDescriptor secondarySearchOp =
new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
- false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
// fifth field from the tuples coming from secondary index
int[] primaryLowKeyFields = { 4 };
@@ -95,7 +95,7 @@
// search primary index
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
- NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+ NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/OptimizedHybridHashJoinTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/OptimizedHybridHashJoinTest.java
index 93739df..9215856 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/OptimizedHybridHashJoinTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/OptimizedHybridHashJoinTest.java
@@ -26,7 +26,6 @@
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksJobletContext;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
-import org.apache.hyracks.api.dataflow.value.IPredicateEvaluator;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparator;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -61,7 +60,6 @@
static RecordDescriptor buildRd;
static ITuplePartitionComputer probeHpc;
static ITuplePartitionComputer buildHpc;
- static IPredicateEvaluator predEval;
int memSizeInFrames = -1;
int numOfPartitions = -1;
boolean isLeftOuter = false;
@@ -151,7 +149,7 @@
private void testJoin(int memSizeInFrames, int numOfPartitions, VSizeFrame frame) throws HyracksDataException {
hhj = new OptimizedHybridHashJoin(ctx, memSizeInFrames, numOfPartitions, probeRelName, buildRelName, probeRd,
- buildRd, probeHpc, buildHpc, predEval, isLeftOuter, null);
+ buildRd, probeHpc, buildHpc, null, null, isLeftOuter, null);
hhj.initBuild();
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index a8bb1d5..296b682 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -133,7 +133,7 @@
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 128,
- null, 128);
+ 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
ResultSetId rsId = new ResultSetId(1);
@@ -176,12 +176,12 @@
new DelimitedDataTupleParserFactory(custValueParserFactories, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
- OptimizedHybridHashJoinOperatorDescriptor join =
- new OptimizedHybridHashJoinOperatorDescriptor(spec, 32, 20, 1.2, new int[] { 1 }, new int[] { 0 },
- new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
- new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
- custOrderJoinDesc, new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0),
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1), null, false, null);
+ OptimizedHybridHashJoinOperatorDescriptor join = new OptimizedHybridHashJoinOperatorDescriptor(spec, 32, 20,
+ 1.2, new int[] { 1 }, new int[] { 0 },
+ new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
+ new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE }, custOrderJoinDesc,
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0),
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1), null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
@@ -234,8 +234,8 @@
new int[] { 1 },
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1), null, custOrderJoinDesc,
- true, nonMatchWriterFactories, 128, 128);
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1), custOrderJoinDesc, true,
+ nonMatchWriterFactories, 128, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
ResultSetId rsId = new ResultSetId(1);
@@ -288,7 +288,7 @@
new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
custOrderJoinDesc, new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null, true,
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null, null, true,
nonMatchWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
@@ -343,7 +343,7 @@
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 128,
- null, 128);
+ 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
ResultSetId rsId = new ResultSetId(1);
@@ -396,12 +396,12 @@
new DelimitedDataTupleParserFactory(custValueParserFactories, '|'), custDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
- OptimizedHybridHashJoinOperatorDescriptor join =
- new OptimizedHybridHashJoinOperatorDescriptor(spec, 5, 20, 1.2, new int[] { 1 }, new int[] { 0 },
- new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
- new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
- custOrderJoinDesc, new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0),
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1), null, false, null);
+ OptimizedHybridHashJoinOperatorDescriptor join = new OptimizedHybridHashJoinOperatorDescriptor(spec, 5, 20, 1.2,
+ new int[] { 1 }, new int[] { 0 },
+ new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE },
+ new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE }, custOrderJoinDesc,
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0),
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1), null, null, false, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
@@ -460,7 +460,7 @@
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 128,
- null, 128);
+ 128);
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
ResultSetId rsId = new ResultSetId(1);
@@ -524,7 +524,7 @@
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), custOrderJoinDesc, 128,
- null, 128);
+ 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
ResultSetId rsId = new ResultSetId(1);
@@ -581,7 +581,7 @@
new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
custOrderJoinDesc, new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null);
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
@@ -629,7 +629,7 @@
new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
custOrderJoinDesc, new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null);
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
@@ -678,7 +678,7 @@
new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
custOrderJoinDesc, new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null);
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null, null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java
index 3bbeca8..a351c85 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java
@@ -185,7 +185,7 @@
new IBinaryHashFunctionFactory[] {
PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
- Common.custOrderJoinDesc, tableSize, null, memSize * frameSize);
+ Common.custOrderJoinDesc, tableSize, memSize * frameSize);
} else if ("hybrid".equalsIgnoreCase(algo)) {
join = new OptimizedHybridHashJoinOperatorDescriptor(spec, memSize, graceInputSize, graceFactor,
@@ -194,7 +194,7 @@
new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
Common.custOrderJoinDesc,
new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 0, 1),
- new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null);
+ new JoinComparatorFactory(UTF8StringBinaryComparatorFactory.INSTANCE, 1, 0), null, null);
} else {
System.err.println("unknown algorithm:" + algo);
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
index 6033094..985515c 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
@@ -63,6 +63,7 @@
<ignoredDependencies>
<ignoredDependency>org.apache.hadoop:hadoop*::</ignoredDependency>
<ignoredDependency>org.apache.logging.log4j:log4j*::</ignoredDependency>
+ <ignoredDependency>org.slf4j:slf4j*::</ignoredDependency>
</ignoredDependencies>
</configuration>
<executions>
@@ -99,6 +100,10 @@
<artifactId>hadoop-common</artifactId>
<exclusions>
<exclusion>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>nimbus-jose-jwt</artifactId>
+ </exclusion>
+ <exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>jsp-api</artifactId>
</exclusion>
@@ -124,6 +129,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -135,9 +148,21 @@
<scope>test</scope>
<exclusions>
<exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-all</artifactId>
+ </exclusion>
+ <exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -152,6 +177,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -211,6 +244,11 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-simple</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-1.2-api</artifactId>
</dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java
index bc187f8..b9d68f7 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/scheduler/Scheduler.java
@@ -49,16 +49,21 @@
* class works for Hadoop old API.
*/
public class Scheduler {
+ /**
+ * Empty input splits
+ */
+ public static final InputSplit[] EMPTY_INPUT_SPLITS = {};
+
private static final Logger LOGGER = LogManager.getLogger();
/** a list of NCs */
private String[] NCs;
/** a map from ip to NCs */
- private Map<String, List<String>> ipToNcMapping = new HashMap<String, List<String>>();
+ private Map<String, List<String>> ipToNcMapping = new HashMap<>();
/** a map from the NC name to the index */
- private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
+ private Map<String, Integer> ncNameToIndex = new HashMap<>();
/** a map from NC name to the NodeControllerInfo */
private Map<String, NodeControllerInfo> ncNameToNcInfos;
@@ -108,8 +113,7 @@
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
- * the mapping from nc names to nc infos
+ * @param ncNameToNcInfos the mapping from nc names to nc infos
* @throws HyracksException
*/
public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
@@ -121,10 +125,8 @@
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
- * the mapping from nc names to nc infos
- * @param topology
- * the hyracks cluster toplogy
+ * @param ncNameToNcInfos the mapping from nc names to nc infos
+ * @param topology the hyracks cluster toplogy
* @throws HyracksException
*/
public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, ClusterTopology topology)
@@ -137,8 +139,7 @@
/**
* The constructor of the scheduler.
*
- * @param ncNameToNcInfos
- * the mapping from nc names to nc infos
+ * @param ncNameToNcInfos the mapping from nc names to nc infos
* @throws HyracksException
*/
public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder ncCollectionBuilder)
@@ -156,14 +157,14 @@
* @throws HyracksDataException
*/
public String[] getLocationConstraints(InputSplit[] splits) throws HyracksException {
- if (splits == null) {
+ if (splits == null || splits == EMPTY_INPUT_SPLITS) {
/** deal the case when the splits array is null */
return new String[] {};
}
int[] workloads = new int[NCs.length];
Arrays.fill(workloads, 0);
String[] locations = new String[splits.length];
- Map<String, IntWritable> locationToNumOfSplits = new HashMap<String, IntWritable>();
+ Map<String, IntWritable> locationToNumOfSplits = new HashMap<>();
/**
* upper bound number of slots that a machine can get
*/
@@ -217,16 +218,11 @@
/**
* Schedule non-local slots to each machine
*
- * @param splits
- * The HDFS file splits.
- * @param workloads
- * The current capacity of each machine.
- * @param locations
- * The result schedule.
- * @param slotLimit
- * The maximum slots of each machine.
- * @param scheduled
- * Indicate which slot is scheduled.
+ * @param splits The HDFS file splits.
+ * @param workloads The current capacity of each machine.
+ * @param locations The result schedule.
+ * @param slotLimit The maximum slots of each machine.
+ * @param scheduled Indicate which slot is scheduled.
*/
private void scheduleNonLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slotLimit,
boolean[] scheduled) throws IOException, UnknownHostException {
@@ -259,18 +255,12 @@
/**
* Schedule data-local slots to each machine.
*
- * @param splits
- * The HDFS file splits.
- * @param workloads
- * The current capacity of each machine.
- * @param locations
- * The result schedule.
- * @param slots
- * The maximum slots of each machine.
- * @param random
- * The random generator.
- * @param scheduled
- * Indicate which slot is scheduled.
+ * @param splits The HDFS file splits.
+ * @param workloads The current capacity of each machine.
+ * @param locations The result schedule.
+ * @param slots The maximum slots of each machine.
+ * @param random The random generator.
+ * @param scheduled Indicate which slot is scheduled.
* @throws IOException
* @throws UnknownHostException
*/
@@ -278,7 +268,7 @@
boolean[] scheduled, final Map<String, IntWritable> locationToNumSplits)
throws IOException, UnknownHostException {
/** scheduling candidates will be ordered inversely according to their popularity */
- PriorityQueue<String> scheduleCadndiates = new PriorityQueue<String>(3, new Comparator<String>() {
+ PriorityQueue<String> scheduleCadndiates = new PriorityQueue<>(3, new Comparator<>() {
@Override
public int compare(String s1, String s2) {
@@ -346,10 +336,8 @@
/**
* Scan the splits once and build a popularity map
*
- * @param splits
- * the split array
- * @param locationToNumOfSplits
- * the map to be built
+ * @param splits the split array
+ * @param locationToNumOfSplits the map to be built
* @throws IOException
*/
private void buildPopularityMap(InputSplit[] splits, Map<String, IntWritable> locationToNumOfSplits)
diff --git a/hyracks-fullstack/hyracks/hyracks-http/pom.xml b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
index cc15214..466a447 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
@@ -45,7 +45,27 @@
<dependencies>
<dependency>
<groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
+ <artifactId>netty-buffer</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-handler</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
@@ -62,7 +82,7 @@
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore-nio</artifactId>
- <version>4.4.10</version>
+ <version>4.4.15</version>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServlet.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServlet.java
index 515b95c..f3e099d 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServlet.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/api/IServlet.java
@@ -23,6 +23,8 @@
import org.apache.hyracks.http.server.HttpServer;
+import io.netty.handler.codec.http.HttpMethod;
+
/**
* Represents a component that handles IServlet requests
*/
@@ -62,4 +64,9 @@
*/
default void init() throws IOException {
}
+
+ /**
+ * @return {@code true} if the servlet ignores query parameters.
+ */
+ boolean ignoresQueryParameters(HttpMethod method);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
index ed567f5..ff23ac2 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
@@ -205,4 +205,8 @@
return this.getClass().getSimpleName() + Arrays.toString(paths);
}
+ @Override
+ public boolean ignoresQueryParameters(HttpMethod method) {
+ return false;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java
index d4f57ea..c3c7dbc 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/BaseRequest.java
@@ -43,9 +43,10 @@
protected final HttpScheme scheme;
protected final InetSocketAddress localAddress;
- public static IServletRequest create(ChannelHandlerContext ctx, FullHttpRequest request, HttpScheme scheme) {
- QueryStringDecoder decoder = new QueryStringDecoder(request.uri());
- Map<? extends CharSequence, List<String>> param = decoder.parameters();
+ public static IServletRequest create(ChannelHandlerContext ctx, FullHttpRequest request, HttpScheme scheme,
+ boolean ignoreQueryParameters) {
+ Map<? extends CharSequence, List<String>> param =
+ ignoreQueryParameters ? Collections.emptyMap() : new QueryStringDecoder(request.uri()).parameters();
InetSocketAddress remoteAddress = (InetSocketAddress) ctx.channel().remoteAddress();
InetSocketAddress localAddress = (InetSocketAddress) ctx.channel().localAddress();
return new BaseRequest(request, localAddress, remoteAddress, param, scheme);
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java
index de6ed72..fa7ef66 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/FormUrlEncodedRequest.java
@@ -36,13 +36,16 @@
public class FormUrlEncodedRequest extends BaseRequest implements IServletRequest {
- public static IServletRequest create(ChannelHandlerContext ctx, FullHttpRequest request, HttpScheme scheme) {
+ public static IServletRequest create(ChannelHandlerContext ctx, FullHttpRequest request, HttpScheme scheme,
+ boolean ignoreQueryParameters) {
Charset charset = HttpUtil.getRequestCharset(request);
Map<String, List<String>> parameters = new LinkedHashMap<>();
URLEncodedUtils.parse(request.content().toString(charset), charset).forEach(
pair -> parameters.computeIfAbsent(pair.getName(), a -> new ArrayList<>()).add(pair.getValue()));
- new QueryStringDecoder(request.uri()).parameters()
- .forEach((name, value) -> parameters.computeIfAbsent(name, a -> new ArrayList<>()).addAll(value));
+ if (!ignoreQueryParameters) {
+ new QueryStringDecoder(request.uri()).parameters()
+ .forEach((name, value) -> parameters.computeIfAbsent(name, a -> new ArrayList<>()).addAll(value));
+ }
InetSocketAddress remoteAddress = (InetSocketAddress) ctx.channel().remoteAddress();
InetSocketAddress localAddress = (InetSocketAddress) ctx.channel().localAddress();
return new FormUrlEncodedRequest(request, localAddress, remoteAddress, parameters, scheme);
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
index a7ace7a..b5adf2f 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerHandler.java
@@ -142,7 +142,8 @@
IServletRequest servletRequest;
try {
HttpScheme scheme = HttpUtil.getScheme(server, request);
- servletRequest = createServletRequest(ctx, request, scheme);
+ boolean ignoreParam = servlet.ignoresQueryParameters(request.method());
+ servletRequest = createServletRequest(ctx, request, scheme, ignoreParam);
} catch (IllegalArgumentException e) {
LOGGER.log(Level.WARN, "Failure Decoding Request", e);
respond(ctx, request, HttpResponseStatus.BAD_REQUEST);
@@ -171,8 +172,8 @@
}
protected IServletRequest createServletRequest(ChannelHandlerContext ctx, FullHttpRequest request,
- HttpScheme scheme) {
- return HttpUtil.toServletRequest(ctx, request, scheme);
+ HttpScheme scheme, boolean ignoreQueryParameters) {
+ return HttpUtil.toServletRequest(ctx, request, scheme, ignoreQueryParameters);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
index 835cd54..e433c02 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
@@ -38,7 +38,9 @@
import java.util.stream.Stream;
import org.apache.commons.lang3.StringUtils;
+import org.apache.http.NameValuePair;
import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.utils.URLEncodedUtils;
import org.apache.hyracks.http.api.IServletRequest;
import org.apache.hyracks.http.api.IServletResponse;
import org.apache.hyracks.http.server.BaseRequest;
@@ -71,10 +73,15 @@
}
public static IServletRequest toServletRequest(ChannelHandlerContext ctx, FullHttpRequest request,
- HttpScheme scheme) {
+ HttpScheme scheme, boolean ignoreQueryParameters) {
return ContentType.APPLICATION_X_WWW_FORM_URLENCODED.equals(getContentTypeOnly(request))
- && !HttpMethod.GET.equals(request.method()) ? FormUrlEncodedRequest.create(ctx, request, scheme)
- : BaseRequest.create(ctx, request, scheme);
+ && !HttpMethod.GET.equals(request.method())
+ ? FormUrlEncodedRequest.create(ctx, request, scheme, ignoreQueryParameters)
+ : BaseRequest.create(ctx, request, scheme, ignoreQueryParameters);
+ }
+
+ public static boolean ignoreQueryParameters(HttpMethod method) {
+ return HttpMethod.POST.equals(method) || HttpMethod.DELETE.equals(method) || HttpMethod.PUT.equals(method);
}
public static String getContentTypeOnly(IServletRequest request) {
@@ -110,6 +117,10 @@
response.setHeader(HttpHeaderNames.CONTENT_TYPE, type);
}
+ public static void setServerHeader(IServletResponse response, String value) throws IOException {
+ response.setHeader(HttpHeaderNames.SERVER, value);
+ }
+
public static Map<String, String> getRequestHeaders(IServletRequest request) {
Map<String, String> headers = new HashMap<>();
request.getHttpRequest().headers().forEach(entry -> headers.put(entry.getKey(), entry.getValue()));
@@ -243,6 +254,23 @@
? HttpScheme.HTTPS : HttpScheme.HTTP;
}
+ /**
+ * @return the first parameter value of the supplied parameter name, or {@link Optional#empty()}
+ */
+ public static Optional<String> extractQueryParameter(String uri, String parameterName) {
+ return extractQueryParameters(uri, parameterName).map(values -> values[0]);
+ }
+
+ /**
+ * @return all parameter values of the supplied parameter name, or {@link Optional#empty()}
+ */
+ public static Optional<String[]> extractQueryParameters(String uri, String parameterName) {
+ String[] values = URLEncodedUtils.parse(uri, StandardCharsets.UTF_8, '?', '&').stream()
+ .filter(pair -> pair.getName().equals(parameterName)).map(NameValuePair::getValue)
+ .toArray(String[]::new);
+ return values.length == 0 ? Optional.empty() : Optional.of(values);
+ }
+
public static class ContentType {
public static final String ADM = "adm";
public static final String JSON = "json";
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java
index 25ea787..2170c15 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java
@@ -19,55 +19,113 @@
package org.apache.hyracks.ipc.security;
import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
import org.apache.hyracks.api.network.INetworkSecurityConfig;
public class NetworkSecurityConfig implements INetworkSecurityConfig {
- private static final long serialVersionUID = -1914030130038989199L;
+ private static final long serialVersionUID = 2L;
+ private static final char[] INTEGRITY_PASSWORD = NetworkSecurityConfig.class.getName().toCharArray();
private final boolean sslEnabled;
private final File keyStoreFile;
private final File trustStoreFile;
private final String keyStorePassword;
- private final transient KeyStore keyStore;
+ private transient KeyStore keyStore;
+ private transient KeyStore trustStore;
private NetworkSecurityConfig(boolean sslEnabled, String keyStoreFile, String keyStorePassword,
- String trustStoreFile, KeyStore keyStore) {
+ String trustStoreFile, KeyStore keyStore, KeyStore trustStore) {
this.sslEnabled = sslEnabled;
this.keyStoreFile = keyStoreFile != null ? new File(keyStoreFile) : null;
this.keyStorePassword = keyStorePassword;
this.trustStoreFile = trustStoreFile != null ? new File(trustStoreFile) : null;
this.keyStore = keyStore;
+ this.trustStore = trustStore;
}
public static NetworkSecurityConfig of(boolean sslEnabled, String keyStoreFile, String keyStorePassword,
String trustStoreFile) {
- return new NetworkSecurityConfig(sslEnabled, keyStoreFile, keyStorePassword, trustStoreFile, null);
+ return new NetworkSecurityConfig(sslEnabled, keyStoreFile, keyStorePassword, trustStoreFile, null, null);
}
public static NetworkSecurityConfig of(boolean sslEnabled, KeyStore keyStore, String keyStorePassword,
- String trustStoreFile) {
- return new NetworkSecurityConfig(sslEnabled, null, keyStorePassword, trustStoreFile, keyStore);
+ KeyStore trustStore) {
+ return new NetworkSecurityConfig(sslEnabled, null, keyStorePassword, null, keyStore, trustStore);
}
+ @Override
public boolean isSslEnabled() {
return sslEnabled;
}
+ @Override
public File getKeyStoreFile() {
return keyStoreFile;
}
+ @Override
public String getKeyStorePassword() {
return keyStorePassword;
}
+ @Override
public KeyStore getKeyStore() {
return keyStore;
}
+ @Override
+ public KeyStore getTrustStore() {
+ return trustStore;
+ }
+
+ @Override
public File getTrustStoreFile() {
return trustStoreFile;
}
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ out.defaultWriteObject();
+ writeStore(keyStore, out);
+ writeStore(trustStore, out);
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ keyStore = readStore(in);
+ trustStore = readStore(in);
+ }
+
+ private void writeStore(KeyStore keyStore, ObjectOutputStream out) throws IOException {
+ if (keyStore == null) {
+ out.writeUTF("");
+ return;
+ }
+ out.writeUTF(keyStore.getType());
+ try {
+ keyStore.store(out, INTEGRITY_PASSWORD);
+ } catch (KeyStoreException | NoSuchAlgorithmException | CertificateException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ private KeyStore readStore(ObjectInputStream in) throws IOException {
+ String keyStoreType = in.readUTF();
+ if (keyStoreType.isEmpty()) {
+ return null;
+ }
+ try {
+ KeyStore store = KeyStore.getInstance(keyStoreType);
+ store.load(in, INTEGRITY_PASSWORD);
+ return store;
+ } catch (KeyStoreException | NoSuchAlgorithmException | CertificateException e) {
+ throw new IllegalStateException(e);
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
index b7c0d0f..42dacf5 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
@@ -87,7 +87,10 @@
KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(defaultAlgorithm);
TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(defaultAlgorithm);
keyManagerFactory.init(engineKeyStore, password);
- final KeyStore trustStore = loadTrustStoreFromFile(password, config);
+ KeyStore trustStore = config.getTrustStore();
+ if (trustStore == null) {
+ trustStore = loadTrustStoreFromFile(password, config);
+ }
trustManagerFactory.init(trustStore);
SSLContext ctx = SSLContext.getInstance(TSL_VERSION);
ctx.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
index 3cc4604..16bcc7c 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
@@ -36,7 +36,7 @@
<dependency>
<groupId>org.apache.maven.plugin-tools</groupId>
<artifactId>maven-plugin-annotations</artifactId>
- <version>3.6.0</version>
+ <scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
@@ -45,7 +45,6 @@
<dependency>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-remote-resources-plugin</artifactId>
- <version>1.6.0</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
@@ -54,7 +53,7 @@
<dependency>
<groupId>org.freemarker</groupId>
<artifactId>freemarker</artifactId>
- <version>2.3.29</version>
+ <version>2.3.31</version>
</dependency>
<dependency>
<groupId>commons-io</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
index 6bc92b8..347c19a 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/GenerateFileMojo.java
@@ -18,11 +18,15 @@
*/
package org.apache.hyracks.maven.license;
+import static org.apache.hyracks.maven.license.GenerateFileMojo.EmbeddedArtifact.LICENSE;
+import static org.apache.hyracks.maven.license.GenerateFileMojo.EmbeddedArtifact.NOTICE;
import static org.apache.hyracks.maven.license.ProjectFlag.ALTERNATE_LICENSE_FILE;
import static org.apache.hyracks.maven.license.ProjectFlag.ALTERNATE_NOTICE_FILE;
import static org.apache.hyracks.maven.license.ProjectFlag.IGNORE_MISSING_EMBEDDED_LICENSE;
import static org.apache.hyracks.maven.license.ProjectFlag.IGNORE_MISSING_EMBEDDED_NOTICE;
import static org.apache.hyracks.maven.license.ProjectFlag.IGNORE_NOTICE_OVERRIDE;
+import static org.apache.hyracks.maven.license.ProjectFlag.ON_MULTIPLE_EMBEDDED_LICENSE;
+import static org.apache.hyracks.maven.license.ProjectFlag.ON_MULTIPLE_EMBEDDED_NOTICE;
import java.io.File;
import java.io.FileOutputStream;
@@ -334,26 +338,55 @@
}
}
+ enum EmbeddedArtifact {
+ NOTICE,
+ LICENSE
+ }
+
private void resolveNoticeFiles() throws MojoExecutionException, IOException {
- // TODO(mblow): this will match *any* NOTICE[.(txt|md)] file located within the artifact-
- // this seems way too liberal
- resolveArtifactFiles("NOTICE", IGNORE_MISSING_EMBEDDED_NOTICE, ALTERNATE_NOTICE_FILE,
- entry -> entry.getName().matches("(.*/|^)" + "NOTICE" + "(.(txt|md))?"), Project::setNoticeText,
- text -> stripFoundationAssertionFromNotices ? FOUNDATION_PATTERN.matcher(text).replaceAll("") : text);
+ resolveArtifactFiles(NOTICE);
}
private void resolveLicenseFiles() throws MojoExecutionException, IOException {
- // TODO(mblow): this will match *any* LICENSE[.(txt|md)] file located within the artifact-
- // this seems way too liberal
- resolveArtifactFiles("LICENSE", IGNORE_MISSING_EMBEDDED_LICENSE, ALTERNATE_LICENSE_FILE,
- entry -> entry.getName().matches("(.*/|^)" + "LICENSE" + "(.(txt|md))?"), Project::setLicenseText,
- UnaryOperator.identity());
+ resolveArtifactFiles(LICENSE);
}
- private void resolveArtifactFiles(final String name, final ProjectFlag ignoreFlag,
- final ProjectFlag alternateFilenameFlag, final Predicate<JarEntry> filter,
- final BiConsumer<Project, String> consumer, final UnaryOperator<String> contentTransformer)
- throws MojoExecutionException, IOException {
+ private void resolveArtifactFiles(final EmbeddedArtifact artifact) throws MojoExecutionException, IOException {
+ final String name;
+ final ProjectFlag ignoreFlag;
+ final ProjectFlag alternateFilenameFlag;
+ final ProjectFlag onMultipleFlag;
+ final Predicate<JarEntry> filter;
+ final BiConsumer<Project, String> consumer;
+ final UnaryOperator<String> contentTransformer;
+
+ switch (artifact) {
+ case NOTICE:
+ name = "NOTICE";
+ ignoreFlag = IGNORE_MISSING_EMBEDDED_NOTICE;
+ alternateFilenameFlag = ALTERNATE_NOTICE_FILE;
+ onMultipleFlag = ON_MULTIPLE_EMBEDDED_NOTICE;
+ // TODO(mblow): this will match *any* NOTICE[.(txt|md)] file located within the artifact-
+ // this seems way too liberal
+ filter = entry -> entry.getName().matches("(.*/|^)" + "NOTICE" + "(.(txt|md))?");
+ consumer = Project::setNoticeText;
+ contentTransformer = UnaryOperator.identity();
+ break;
+ case LICENSE:
+ name = "LICENSE";
+ ignoreFlag = IGNORE_MISSING_EMBEDDED_LICENSE;
+ alternateFilenameFlag = ALTERNATE_LICENSE_FILE;
+ onMultipleFlag = ON_MULTIPLE_EMBEDDED_LICENSE;
+ // TODO(mblow): this will match *any* LICENSE[.(txt|md)] file located within the artifact-
+ // this seems way too liberal
+ filter = entry -> entry.getName().matches("(.*/|^)" + "LICENSE" + "(.(txt|md))?");
+ consumer = Project::setLicenseText;
+ contentTransformer = stripFoundationAssertionFromNotices
+ ? text -> FOUNDATION_PATTERN.matcher(text).replaceAll("") : UnaryOperator.identity();
+ break;
+ default:
+ throw new IllegalStateException("NYI: " + artifact);
+ }
for (Project p : getProjects()) {
File artifactFile = new File(p.getArtifactPath());
if (!artifactFile.exists()) {
@@ -371,12 +404,45 @@
warnUnlessFlag(p, ignoreFlag, "No " + name + " file found for " + p.gav());
} else {
if (matches.size() > 1) {
- getLog().warn("Multiple " + name + " files found for " + p.gav() + ": " + matches.keySet()
- + "; taking first");
+ // TODO(mblow): duplicate elimination on matches content
+ warnUnlessFlag(p, onMultipleFlag,
+ "Multiple " + name + " files found for " + p.gav() + ": " + matches.keySet() + "!");
+ String onMultiple = (String) getProjectFlag(p.gav(), onMultipleFlag);
+ if (onMultiple == null) {
+ onMultiple = "concat";
+ }
+ switch (onMultiple.toLowerCase()) {
+ case "concat":
+ getLog().info("...concatenating all " + matches.size() + " matches");
+ StringBuilder content = new StringBuilder();
+ for (Map.Entry<String, JarEntry> match : matches.entrySet()) {
+ resolveContent(p, jarFile, match.getValue(), contentTransformer, (p1, text) -> {
+ content.append("------------ BEGIN <").append(match.getKey())
+ .append("> ------------\n");
+ content.append(text);
+ if (content.charAt(content.length() - 1) != '\n') {
+ content.append('\n');
+ }
+ content.append("------------ END <").append(match.getKey())
+ .append("> ------------\n");
+ }, name);
+ }
+ consumer.accept(p, content.toString());
+ break;
+ case "first":
+ Map.Entry<String, JarEntry> first = matches.entrySet().iterator().next();
+ getLog().info("...taking first match: " + first.getKey());
+ resolveContent(p, jarFile, first.getValue(), contentTransformer, consumer, name);
+ break;
+ default:
+ throw new IllegalArgumentException("unknown value for " + onMultipleFlag.propName()
+ + ": " + onMultiple.toLowerCase());
+ }
} else {
- getLog().info(p.gav() + " has " + name + " file: " + matches.keySet());
+ Map.Entry<String, JarEntry> match = matches.entrySet().iterator().next();
+ getLog().info(p.gav() + " has " + name + " file: " + match.getKey());
+ resolveContent(p, jarFile, match.getValue(), contentTransformer, consumer, name);
}
- resolveContent(p, jarFile, matches.values().iterator().next(), contentTransformer, consumer, name);
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
index 89993fd..2206621 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/LicenseMojo.java
@@ -41,13 +41,19 @@
import java.util.stream.Collectors;
import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.mutable.MutableBoolean;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.maven.license.project.LicensedProjects;
import org.apache.hyracks.maven.license.project.Project;
import org.apache.maven.artifact.Artifact;
+import org.apache.maven.artifact.DefaultArtifact;
+import org.apache.maven.artifact.handler.ArtifactHandler;
+import org.apache.maven.artifact.handler.DefaultArtifactHandler;
import org.apache.maven.artifact.repository.ArtifactRepository;
+import org.apache.maven.artifact.resolver.ArtifactResolutionRequest;
+import org.apache.maven.artifact.resolver.ArtifactResolutionResult;
import org.apache.maven.artifact.resolver.ArtifactResolver;
import org.apache.maven.execution.MavenSession;
import org.apache.maven.model.License;
@@ -121,6 +127,9 @@
@Parameter
protected boolean failOnWarning;
+ @Parameter
+ protected List<String> extraDependencies = new ArrayList<>();
+
private Map<String, MavenProject> projectCache = new HashMap<>();
private Map<String, Model> supplementModels;
@@ -393,24 +402,47 @@
private void gatherProjectDependencies(MavenProject project,
Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException {
- final Set dependencyArtifacts = project.getArtifacts();
+ final Set<Artifact> dependencyArtifacts = project.getArtifacts();
if (dependencyArtifacts != null) {
- for (Object depArtifactObj : dependencyArtifacts) {
- final Artifact depArtifact = (Artifact) depArtifactObj;
- if (!excludedScopes.contains(depArtifact.getScope())) {
- MavenProject dep = resolveDependency(depArtifact);
- dep.setArtifact(depArtifact);
- dependencyGavMap.put(toGav(dep), dep);
- List<Pair<String, String>> licenseUrls = new ArrayList<>();
- for (Object license : dep.getLicenses()) {
- final License license1 = (License) license;
- String url = license1.getUrl() != null ? license1.getUrl()
- : (license1.getName() != null ? license1.getName() : "LICENSE_EMPTY_NAME_URL");
- licenseUrls.add(new ImmutablePair<>(url, license1.getName()));
- }
- dependencyLicenseMap.put(dep, licenseUrls);
+ for (Artifact depArtifact : dependencyArtifacts) {
+ processArtifact(depArtifact, dependencyLicenseMap, dependencyGavMap);
+ }
+ }
+ for (String gav : extraDependencies) {
+ ArtifactHandler handler = new DefaultArtifactHandler("jar");
+ String[] gavParts = StringUtils.split(gav, ':');
+ Artifact manualDep = new DefaultArtifact(gavParts[0], gavParts[1], gavParts[2], Artifact.SCOPE_COMPILE,
+ "jar", null, handler);
+ processArtifact(manualDep, dependencyLicenseMap, dependencyGavMap);
+ }
+ }
+
+ private void processArtifact(Artifact depArtifact,
+ Map<MavenProject, List<Pair<String, String>>> dependencyLicenseMap,
+ Map<String, MavenProject> dependencyGavMap) throws ProjectBuildingException {
+ if (!excludedScopes.contains(depArtifact.getScope())) {
+ MavenProject dep = resolveDependency(depArtifact);
+ if (!depArtifact.isResolved()) {
+ ArtifactResolutionRequest arr = new ArtifactResolutionRequest();
+ arr.setLocalRepository(localRepository);
+ arr.setRemoteRepositories(remoteRepositories);
+ arr.setArtifact(depArtifact);
+ ArtifactResolutionResult result = artifactResolver.resolve(arr);
+ if (!result.isSuccess()) {
+ throw new ProjectBuildingException(project.getId(),
+ "Unable to resolve " + depArtifact + ": " + result.getExceptions(), (Throwable) null);
}
}
+ dep.setArtifact(depArtifact);
+ dependencyGavMap.put(toGav(dep), dep);
+ List<Pair<String, String>> licenseUrls = new ArrayList<>();
+ for (Object license : dep.getLicenses()) {
+ final License license1 = (License) license;
+ String url = license1.getUrl() != null ? license1.getUrl()
+ : (license1.getName() != null ? license1.getName() : "LICENSE_EMPTY_NAME_URL");
+ licenseUrls.add(new ImmutablePair<>(url, license1.getName()));
+ }
+ dependencyLicenseMap.put(dep, licenseUrls);
}
}
@@ -430,7 +462,7 @@
.get(SupplementalModelHelper.generateSupplementMapKey(depObj.getGroupId(), depObj.getArtifactId()));
registerVerified(depProj, supplement);
if (supplement != null) {
- Model merged = SupplementalModelHelper.mergeModels(assembler, depProj.getModel(), supplement);
+ Model merged = SupplementalModelHelper.mergeModels(assembler, depProj.getModel(), supplement).clone();
Set<String> origLicenses =
depProj.getModel().getLicenses().stream().map(License::getUrl).collect(Collectors.toSet());
Set<String> newLicenses =
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
index dd8fdf7..d61dde1 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
@@ -20,8 +20,8 @@
import static org.apache.hyracks.maven.license.LicenseUtil.toGav;
+import java.util.Arrays;
import java.util.Properties;
-import java.util.stream.Stream;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
@@ -33,6 +33,8 @@
IGNORE_MISSING_EMBEDDED_NOTICE,
IGNORE_LICENSE_OVERRIDE,
IGNORE_NOTICE_OVERRIDE,
+ ON_MULTIPLE_EMBEDDED_LICENSE,
+ ON_MULTIPLE_EMBEDDED_NOTICE,
ALTERNATE_LICENSE_FILE,
ALTERNATE_NOTICE_FILE;
@@ -50,7 +52,7 @@
case IGNORE_MISSING_EMBEDDED_NOTICE:
case IGNORE_LICENSE_OVERRIDE:
case IGNORE_NOTICE_OVERRIDE:
- if (Stream.of(StringUtils.split(value, ",")).anyMatch(depObj.getVersion()::equals)) {
+ if (Arrays.asList(StringUtils.split(value, ",")).contains(depObj.getVersion())) {
licenseMojo.getProjectFlags().put(Pair.of(toGav(depObj), this), Boolean.TRUE);
} else {
licenseMojo.getLog().info(propName() + " defined on versions that *do not* match: " + value
@@ -59,6 +61,9 @@
break;
case ALTERNATE_LICENSE_FILE:
case ALTERNATE_NOTICE_FILE:
+ case ON_MULTIPLE_EMBEDDED_NOTICE:
+ case ON_MULTIPLE_EMBEDDED_LICENSE:
+ boolean found = false;
for (String spec : StringUtils.split(value, ",")) {
String[] specSplit = StringUtils.split(spec, ":");
if (specSplit.length != 2) {
@@ -66,8 +71,13 @@
}
if (specSplit[0].equals(depObj.getVersion())) {
licenseMojo.getProjectFlags().put(Pair.of(toGav(depObj), this), specSplit[1]);
+ found = true;
}
}
+ if (!found) {
+ licenseMojo.getLog().info(propName() + " defined on versions that *do not* match: " + value
+ + " for " + toGav(depObj));
+ }
break;
default:
throw new IllegalStateException("NYI: " + this);
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java
index 8ac8b12..f7aee35 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/SourcePointerResolver.java
@@ -296,5 +296,15 @@
public void setMirroredRepositories(List<ArtifactRepository> list) {
// unused
}
+
+ @java.lang.Override
+ public boolean isBlocked() {
+ return false;
+ }
+
+ @java.lang.Override
+ public void setBlocked(boolean blocked) {
+ // unused
+ }
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
index 8c327f7..17fe593 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeFieldPrefixTupleReference;
import org.apache.hyracks.storage.am.btree.impls.FieldPrefixSlotManager;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriter;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameCompressor;
import org.apache.hyracks.storage.common.MultiComparator;
@@ -46,10 +47,17 @@
private final ITypeTraits[] typeTraits;
- public FieldPrefixCompressor(ITypeTraits[] typeTraits, float ratioThreshold, int occurrenceThreshold) {
+ private final ITypeTraits nullTypeTraits;
+
+ private final INullIntrospector nullIntrospector;
+
+ public FieldPrefixCompressor(ITypeTraits[] typeTraits, float ratioThreshold, int occurrenceThreshold,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
this.typeTraits = typeTraits;
this.ratioThreshold = ratioThreshold;
this.occurrenceThreshold = occurrenceThreshold;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
@@ -163,7 +171,8 @@
int prefixTupleIndex = 0;
uncompressedTupleCount = 0;
- BTreeTypeAwareTupleWriter tupleWriter = new BTreeTypeAwareTupleWriter(typeTraits, false);
+ BTreeTypeAwareTupleWriter tupleWriter =
+ new BTreeTypeAwareTupleWriter(typeTraits, false, nullTypeTraits, nullIntrospector);
BTreeFieldPrefixTupleReference tupleToWrite =
new BTreeFieldPrefixTupleReference(tupleWriter.createTupleReference());
tupleToWrite.setFieldCount(fieldCount);
@@ -340,7 +349,8 @@
KeyPartition kp = new KeyPartition(maxCmps);
keyPartitions.add(kp);
- BTreeTypeAwareTupleWriter tupleWriter = new BTreeTypeAwareTupleWriter(typeTraits, false);
+ BTreeTypeAwareTupleWriter tupleWriter =
+ new BTreeTypeAwareTupleWriter(typeTraits, false, nullTypeTraits, nullIntrospector);
BTreeFieldPrefixTupleReference prevTuple =
new BTreeFieldPrefixTupleReference(tupleWriter.createTupleReference());
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java
index 4cf145b..4589bed1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResource.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.common.IIndex;
import org.apache.hyracks.storage.common.IResource;
@@ -34,20 +35,25 @@
public class BTreeResource implements IResource {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private String path;
private final IStorageManager storageManager;
private final ITypeTraits[] typeTraits;
private final IBinaryComparatorFactory[] comparatorFactories;
private final IPageManagerFactory pageManagerFactory;
+ private final ITypeTraits nullTypeTraits;
+ private final INullIntrospector nullIntrospector;
public BTreeResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory) {
+ IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
this.path = path;
this.storageManager = storageManager;
this.typeTraits = typeTraits;
this.comparatorFactories = comparatorFactories;
this.pageManagerFactory = pageManagerFactory;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
@@ -56,7 +62,8 @@
IIOManager ioManager = ctx.getIoManager();
FileReference resourceRef = ioManager.resolve(path);
return BTreeUtils.createBTree(bufferCache, typeTraits, comparatorFactories, BTreeLeafFrameType.REGULAR_NSM,
- resourceRef, pageManagerFactory.createPageManager(bufferCache), false);
+ resourceRef, pageManagerFactory.createPageManager(bufferCache), false, nullTypeTraits,
+ nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResourceFactory.java
index 7a21495..d817327 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeResourceFactory.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.common.IResource;
import org.apache.hyracks.storage.common.IResourceFactory;
@@ -28,24 +29,29 @@
public class BTreeResourceFactory implements IResourceFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private final IStorageManager storageManager;
private final ITypeTraits[] typeTraits;
private final IBinaryComparatorFactory[] comparatorFactories;
private final IPageManagerFactory pageManagerFactory;
+ private final ITypeTraits nullTypeTraits;
+ private final INullIntrospector nullIntrospector;
public BTreeResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
- IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory) {
+ IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
this.storageManager = storageManager;
this.typeTraits = typeTraits;
this.comparatorFactories = comparatorFactories;
this.pageManagerFactory = pageManagerFactory;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
public IResource createResource(FileReference fileRef) {
return new BTreeResource(fileRef.getRelativePath(), storageManager, typeTraits, comparatorFactories,
- pageManagerFactory);
+ pageManagerFactory, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 128929c..2455625 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -46,6 +46,7 @@
protected final IMissingWriterFactory missingWriterFactory;
protected final ISearchOperationCallbackFactory searchCallbackFactory;
protected final boolean appendIndexFilter;
+ protected final IMissingWriterFactory nonFilterWriterFactory;
protected boolean appendOpCallbackProceedResult;
protected byte[] searchCallbackProceedResultFalseValue;
protected byte[] searchCallbackProceedResultTrueValue;
@@ -56,10 +57,11 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+ int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory) {
this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, null, -1, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
}
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -67,8 +69,9 @@
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
- ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
+ IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -82,6 +85,7 @@
this.minFilterFieldIndexes = minFilterFieldIndexes;
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.appendIndexFilter = appendIndexFilter;
+ this.nonFilterWriterFactory = nonFilterWriterFactory;
this.outRecDescs[0] = outRecDesc;
this.tupleFilterFactory = tupleFilterFactory;
this.outputLimit = outputLimit;
@@ -97,8 +101,8 @@
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
- tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
- searchCallbackProceedResultTrueValue);
+ nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 77ef1dc..fb331bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -46,26 +46,27 @@
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
- ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
- throws HyracksDataException {
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
+ ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
- missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1, false, null, null);
+ nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
+ false, null, null);
}
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
- ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
- throws HyracksDataException {
+ IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
- tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
- searchCallbackProceedResultTrueValue);
+ retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
+ searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
if (lowKeyFields != null && lowKeyFields.length > 0) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index e6637db..f8402e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -20,6 +20,7 @@
package org.apache.hyracks.storage.am.btree.dataflow;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -38,9 +39,10 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
ISearchOperationCallbackFactory searchOpCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory,
- boolean appendIndexFilter) {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, dataflowHelperFactory,
- retainInput, false, null, searchOpCallbackProvider, null, null, appendIndexFilter);
+ retainInput, false, null, searchOpCallbackProvider, null, null, appendIndexFilter,
+ nonFilterWriterFactory);
this.tupleUpdaterFactory = tupleUpdaterFactory;
}
@@ -50,6 +52,7 @@
return new BTreeUpdateSearchOperatorNodePushable(ctx, partition,
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
- searchCallbackFactory, tupleUpdaterFactory.createTupleUpdater(), appendIndexFilter);
+ searchCallbackFactory, tupleUpdaterFactory.createTupleUpdater(), appendIndexFilter,
+ nonFilterWriterFactory);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index a296672..a102e39 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -41,10 +41,11 @@
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- ITupleUpdater tupleUpdater, boolean appendIndexFilter) throws HyracksDataException {
+ ITupleUpdater tupleUpdater, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory)
+ throws HyracksDataException {
super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, null, null,
indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory,
- appendIndexFilter);
+ appendIndexFilter, nonFilterWriterFactory);
this.tupleUpdater = tupleUpdater;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index 6584df3..10eab69 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -75,8 +75,10 @@
this.slotManager = new FieldPrefixSlotManager();
ITypeTraits[] typeTraits = tupleWriter.getTypeTraits();
- this.framePrefixTuple = new FieldPrefixPrefixTupleReference(typeTraits);
- this.compressor = new FieldPrefixCompressor(typeTraits, 0.001f, 2);
+ ITypeTraits nullTypeTraits = tupleWriter.getNullTypeTraits();
+ this.framePrefixTuple = new FieldPrefixPrefixTupleReference(typeTraits, nullTypeTraits);
+ this.compressor =
+ new FieldPrefixCompressor(typeTraits, 0.001f, 2, nullTypeTraits, tupleWriter.getNullIntrospector());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 78faaff..40b2f5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -42,7 +42,6 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
import org.apache.hyracks.storage.am.common.api.IBTreeIndexTupleReference;
import org.apache.hyracks.storage.am.common.api.IPageManager;
-import org.apache.hyracks.storage.am.common.api.ISplitKey;
import org.apache.hyracks.storage.am.common.api.ITreeIndexAccessor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -51,7 +50,6 @@
import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
@@ -64,17 +62,13 @@
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.NoOpIndexCursorStats;
-import org.apache.hyracks.storage.common.buffercache.BufferCache;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
-import org.apache.hyracks.util.JSONUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
public class BTree extends AbstractTreeIndex {
public static final float DEFAULT_FILL_FACTOR = 0.7f;
@@ -85,7 +79,7 @@
private final AtomicInteger smoCounter;
private final ReadWriteLock treeLatch;
- private final int maxTupleSize;
+ protected final int maxTupleSize;
public BTree(IBufferCache bufferCache, IPageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount,
@@ -886,13 +880,13 @@
}
@Override
- public BTreeRangeSearchCursor createSearchCursor(boolean exclusive) {
+ public ITreeIndexCursor createSearchCursor(boolean exclusive) {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
return new BTreeRangeSearchCursor(leafFrame, exclusive, (IIndexCursorStats) iap.getParameters()
.getOrDefault(HyracksConstants.INDEX_CURSOR_STATS, NoOpIndexCursorStats.INSTANCE));
}
- public BTreeRangeSearchCursor createPointCursor(boolean exclusive, boolean stateful) {
+ public ITreeIndexCursor createPointCursor(boolean exclusive, boolean stateful) {
return createSearchCursor(exclusive);
}
@@ -1004,220 +998,7 @@
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
boolean checkIfEmptyIndex, IPageWriteCallback callback) throws HyracksDataException {
- return new BTreeBulkLoader(fillFactor, verifyInput, callback);
- }
-
- public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
- protected final ISplitKey splitKey;
- protected final boolean verifyInput;
-
- public BTreeBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback)
- throws HyracksDataException {
- super(fillFactor, callback);
- this.verifyInput = verifyInput;
- splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
- splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
- }
-
- @Override
- public void add(ITupleReference tuple) throws HyracksDataException {
- try {
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- NodeFrontier leafFrontier = nodeFrontiers.get(0);
- int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
- int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-
- // try to free space by compression
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- leafFrame.compress();
- spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
- }
- //full, allocate new page
- if (spaceUsed + spaceNeeded > leafMaxBytes) {
- if (leafFrame.getTupleCount() == 0) {
- bufferCache.returnPage(leafFrontier.page, false);
- } else {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- if (verifyInput) {
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
- splitKey.setLeftPage(leafFrontier.pageId);
-
- propagateBulk(1, pagesToWrite);
-
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
-
- ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
-
- write(leafFrontier.page);
- for (ICachedPage c : pagesToWrite) {
- write(c);
- }
- pagesToWrite.clear();
- splitKey.setRightPage(leafFrontier.pageId);
- }
- if (tupleSize > maxTupleSize) {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- // calculate required number of pages.
- int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
- final int multiplier =
- (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
- if (multiplier > 1) {
- leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
- freePageManager.takeBlock(metaFrame, multiplier - 1));
- } else {
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- }
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
- } else {
- final long dpid = BufferedFileHandle.getDiskPageId(getFileId(), leafFrontier.pageId);
- leafFrontier.page = bufferCache.confiscatePage(dpid);
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- }
- } else {
- if (verifyInput && leafFrame.getTupleCount() > 0) {
- leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
- verifyInputTuple(tuple, leafFrontier.lastTuple);
- }
- }
- ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
- } catch (HyracksDataException | RuntimeException e) {
- logState(tuple, e);
- handleException();
- throw e;
- }
- }
-
- protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
- // New tuple should be strictly greater than last tuple.
- int cmpResult = cmp.compare(tuple, prevTuple);
- if (cmpResult < 0) {
- throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
- }
- if (cmpResult == 0) {
- throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
- }
- }
-
- protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
- if (splitKey.getBuffer() == null) {
- return;
- }
-
- if (level >= nodeFrontiers.size()) {
- addLevel();
- }
-
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
-
- ITupleReference tuple = splitKey.getTuple();
- int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
- int spaceNeeded = tupleBytes + slotSize + 4;
- if (tupleBytes > interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize())) {
- throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
- interiorFrame.getMaxTupleSize(BTree.this.bufferCache.getPageSize()));
- }
-
- int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
- if (spaceUsed + spaceNeeded > interiorMaxBytes) {
-
- ISplitKey copyKey = splitKey.duplicate(leafFrame.getTupleWriter().createTupleReference());
- tuple = copyKey.getTuple();
-
- frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
- int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
- splitKey.initData(splitKeySize);
- tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(),
- splitKey.getBuffer().array(), 0);
- splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
-
- ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- pagesToWrite.add(frontier.page);
- splitKey.setLeftPage(finalPageId);
-
- propagateBulk(level + 1, pagesToWrite);
- frontier.page = bufferCache.confiscatePage(BufferCache.INVALID_DPID);
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) level);
- }
- ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
- }
-
- private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
- if (level >= nodeFrontiers.size()) {
- rootPage = nodeFrontiers.get(level - 1).pageId;
- releasedLatches = true;
- return;
- }
- if (level < 1) {
- ICachedPage lastLeaf = nodeFrontiers.get(level).page;
- int lastLeafPage = nodeFrontiers.get(level).pageId;
- lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), nodeFrontiers.get(level).pageId));
- write(lastLeaf);
- nodeFrontiers.get(level).page = null;
- persistFrontiers(level + 1, lastLeafPage);
- return;
- }
- NodeFrontier frontier = nodeFrontiers.get(level);
- interiorFrame.setPage(frontier.page);
- //just finalize = the layer right above the leaves has correct righthand pointers already
- if (rightPage < 0) {
- throw new HyracksDataException(
- "Error in index creation. Internal node appears to have no rightmost guide");
- }
- ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
- int finalPageId = freePageManager.takePage(metaFrame);
- frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(getFileId(), finalPageId));
- write(frontier.page);
- frontier.pageId = finalPageId;
- persistFrontiers(level + 1, finalPageId);
- }
-
- @Override
- public void end() throws HyracksDataException {
- try {
- persistFrontiers(0, -1);
- super.end();
- } catch (HyracksDataException | RuntimeException e) {
- handleException();
- throw e;
- }
- }
-
- @Override
- public void abort() throws HyracksDataException {
- super.handleException();
- }
-
- private void logState(ITupleReference tuple, Exception e) {
- try {
- ObjectNode state = JSONUtil.createObject();
- state.set("leafFrame", leafFrame.getState());
- state.set("interiorFrame", interiorFrame.getState());
- int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
- interiorFrame.getBytesRequiredToWriteTuple(tuple));
- state.put("tupleSize", tupleSize);
- state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
- state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
- state.put("leafMaxBytes", leafMaxBytes);
- state.put("maxTupleSize", maxTupleSize);
- LOGGER.error("failed to add tuple {}", state, e);
- } catch (Throwable t) {
- e.addSuppressed(t);
- }
- }
+ return new BTreeNSMBulkLoader(fillFactor, verifyInput, callback, this);
}
@SuppressWarnings("rawtypes")
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
new file mode 100644
index 0000000..04c84e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeNSMBulkLoader.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree.impls;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
+import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import org.apache.hyracks.storage.am.common.api.ISplitKey;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.JSONUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class BTreeNSMBulkLoader extends AbstractTreeIndexBulkLoader {
+ private static final Logger LOGGER = LogManager.getLogger();
+ protected final ISplitKey splitKey;
+ protected final boolean verifyInput;
+ private final int maxTupleSize;
+
+ public BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, verifyInput, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected BTreeNSMBulkLoader(float fillFactor, boolean verifyInput, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ super(fillFactor, callback, index, leafFrame);
+ this.verifyInput = verifyInput;
+ splitKey = new BTreeSplitKey(tupleWriter.createTupleReference());
+ splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+ maxTupleSize = ((BTree) index).maxTupleSize;
+ }
+
+ @Override
+ public void add(ITupleReference tuple) throws HyracksDataException {
+ try {
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+ int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+ // try to free space by compression
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ leafFrame.compress();
+ spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+ }
+ //full, allocate new page
+ if (spaceUsed + spaceNeeded > leafMaxBytes) {
+ if (leafFrame.getTupleCount() == 0) {
+ //The current page is empty. Return it.
+ bufferCache.returnPage(leafFrontier.page, false);
+ } else {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ if (verifyInput) {
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ //The current page is not empty. Write it.
+ writeFullLeafPage();
+ }
+ if (tupleSize > maxTupleSize) {
+ //We need a large page
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ // calculate required number of pages.
+ int headerSize = Math.max(leafFrame.getPageHeaderSize(), interiorFrame.getPageHeaderSize());
+ final int multiplier =
+ (int) Math.ceil((double) tupleSize / (bufferCache.getPageSize() - headerSize));
+ if (multiplier > 1) {
+ leafFrontier.page = bufferCache.confiscateLargePage(dpid, multiplier,
+ freePageManager.takeBlock(metaFrame, multiplier - 1));
+ } else {
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ }
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ ((IBTreeLeafFrame) leafFrame).setLargeFlag(true);
+ } else {
+ //allocate a new page
+ confiscateNewLeafPage();
+ }
+ } else {
+ if (verifyInput && leafFrame.getTupleCount() > 0) {
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ verifyInputTuple(tuple, leafFrontier.lastTuple);
+ }
+ }
+ ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
+ } catch (HyracksDataException | RuntimeException e) {
+ logState(tuple, e);
+ handleException();
+ throw e;
+ }
+ }
+
+ protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws HyracksDataException {
+ // New tuple should be strictly greater than last tuple.
+ int cmpResult = cmp.compare(tuple, prevTuple);
+ if (cmpResult < 0) {
+ throw HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
+ }
+ if (cmpResult == 0) {
+ throw HyracksDataException.create(ErrorCode.DUPLICATE_LOAD_INPUT);
+ }
+ }
+
+ protected void propagateBulk(int level, List<ICachedPage> pagesToWrite) throws HyracksDataException {
+ if (splitKey.getBuffer() == null) {
+ return;
+ }
+
+ if (level >= nodeFrontiers.size()) {
+ addLevel();
+ }
+
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+
+ ITupleReference tuple = splitKey.getTuple();
+ int tupleBytes = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount());
+ int spaceNeeded = tupleBytes + slotSize + 4;
+ if (tupleBytes > interiorFrame.getMaxTupleSize(bufferCache.getPageSize())) {
+ throw HyracksDataException.create(ErrorCode.RECORD_IS_TOO_LARGE, tupleBytes,
+ interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
+ }
+
+ int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
+ if (spaceUsed + spaceNeeded > interiorMaxBytes) {
+ ISplitKey copyKey = splitKey.duplicate(tupleWriter.createTupleReference());
+ tuple = copyKey.getTuple();
+
+ frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
+ int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+
+ ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ pagesToWrite.add(frontier.page);
+ splitKey.setLeftPage(finalPageId);
+
+ propagateBulk(level + 1, pagesToWrite);
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) level);
+ }
+ ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
+ }
+
+ private void persistFrontiers(int level, int rightPage) throws HyracksDataException {
+ if (level >= nodeFrontiers.size()) {
+ setRootPageId(nodeFrontiers.get(level - 1).pageId);
+ releasedLatches = true;
+ return;
+ }
+ if (level < 1) {
+ ICachedPage lastLeaf = nodeFrontiers.get(level).page;
+ int lastLeafPage = nodeFrontiers.get(level).pageId;
+ lastLeaf.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, nodeFrontiers.get(level).pageId));
+ writeLastLeaf(lastLeaf);
+ nodeFrontiers.get(level).page = null;
+ persistFrontiers(level + 1, lastLeafPage);
+ return;
+ }
+ NodeFrontier frontier = nodeFrontiers.get(level);
+ interiorFrame.setPage(frontier.page);
+ //just finalize = the layer right above the leaves has correct righthand pointers already
+ if (rightPage < 0) {
+ throw new HyracksDataException("Error in index creation. Internal node appears to have no rightmost guide");
+ }
+ ((IBTreeInteriorFrame) interiorFrame).setRightmostChildPageId(rightPage);
+ int finalPageId = freePageManager.takePage(metaFrame);
+ frontier.page.setDiskPageId(BufferedFileHandle.getDiskPageId(fileId, finalPageId));
+ write(frontier.page);
+ frontier.pageId = finalPageId;
+ persistFrontiers(level + 1, finalPageId);
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ try {
+ persistFrontiers(0, -1);
+ super.end();
+ } catch (HyracksDataException | RuntimeException e) {
+ handleException();
+ throw e;
+ }
+ }
+
+ @Override
+ public void abort() throws HyracksDataException {
+ super.handleException();
+ }
+
+ protected void writeFullLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+ final int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
+ splitKey.initData(splitKeySize);
+ tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer().array(),
+ 0);
+ splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer().array(), 0);
+ splitKey.setLeftPage(leafFrontier.pageId);
+
+ propagateBulk(1, pagesToWrite);
+
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+
+ ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
+
+ write(leafFrontier.page);
+ for (ICachedPage c : pagesToWrite) {
+ write(c);
+ }
+ pagesToWrite.clear();
+ splitKey.setRightPage(leafFrontier.pageId);
+ }
+
+ protected void writeLastLeaf(ICachedPage page) throws HyracksDataException {
+ write(page);
+ }
+
+ protected final void confiscateNewLeafPage() throws HyracksDataException {
+ final NodeFrontier leafFrontier = nodeFrontiers.get(0);
+ final long dpid = BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId);
+ leafFrontier.page = bufferCache.confiscatePage(dpid);
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ }
+
+ private void logState(ITupleReference tuple, Exception e) {
+ try {
+ ObjectNode state = JSONUtil.createObject();
+ state.set("leafFrame", leafFrame.getState());
+ state.set("interiorFrame", interiorFrame.getState());
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ state.put("tupleSize", tupleSize);
+ state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
+ state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
+ state.put("leafMaxBytes", leafMaxBytes);
+ state.put("maxTupleSize", maxTupleSize);
+ LOGGER.error("failed to add tuple {}", state, e);
+ } catch (Throwable t) {
+ e.addSuppressed(t);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
index b767537..2c728a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
@@ -27,8 +27,8 @@
public class FieldPrefixPrefixTupleReference extends TypeAwareTupleReference {
- public FieldPrefixPrefixTupleReference(ITypeTraits[] typeTraits) {
- super(typeTraits);
+ public FieldPrefixPrefixTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
+ super(typeTraits, nullTypeTraits);
}
// assumes tuple index refers to prefix tuples
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleReference.java
index 5b89a19..f4b1948 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleReference.java
@@ -23,14 +23,15 @@
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
+// TODO(ali): this is broken if updateAware is enabled
public class BTreeTypeAwareTupleReference extends TypeAwareTupleReference implements IBTreeIndexTupleReference {
public static final byte UPDATE_BIT_OFFSET = 6;
protected final boolean updateAware;
- public BTreeTypeAwareTupleReference(ITypeTraits[] typeTraits, boolean updateAware) {
- super(typeTraits);
+ public BTreeTypeAwareTupleReference(ITypeTraits[] typeTraits, boolean updateAware, ITypeTraits nullTypeTraits) {
+ super(typeTraits, nullTypeTraits);
this.updateAware = updateAware;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriter.java
index 083462b..87f46b2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriter.java
@@ -19,21 +19,23 @@
package org.apache.hyracks.storage.am.btree.tuples;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
-public class BTreeTypeAwareTupleWriter extends TypeAwareTupleWriter implements ITreeIndexTupleWriter {
+// TODO(ali): this is broken if updateAware is enabled
+public class BTreeTypeAwareTupleWriter extends TypeAwareTupleWriter {
protected final boolean updateAware;
protected boolean isUpdated;
- public BTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits, boolean updateAware) {
- super(typeTraits);
+ public BTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits, boolean updateAware, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
this.updateAware = updateAware;
}
@Override
public BTreeTypeAwareTupleReference createTupleReference() {
- return new BTreeTypeAwareTupleReference(typeTraits, updateAware);
+ return new BTreeTypeAwareTupleReference(typeTraits, updateAware, nullTypeTraits);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriterFactory.java
index 7089e79..e2aa759 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/tuples/BTreeTypeAwareTupleWriterFactory.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.btree.tuples;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
public class BTreeTypeAwareTupleWriterFactory extends TypeAwareTupleWriterFactory {
@@ -26,13 +27,14 @@
private static final long serialVersionUID = 1L;
protected final boolean updateAware;
- public BTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits, boolean updateAware) {
- super(typeTraits);
+ public BTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits, boolean updateAware, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
this.updateAware = updateAware;
}
@Override
public BTreeTypeAwareTupleWriter createTupleWriter() {
- return new BTreeTypeAwareTupleWriter(typeTraits, updateAware);
+ return new BTreeTypeAwareTupleWriter(typeTraits, updateAware, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java
index 0c06bc3..09340b6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/util/BTreeUtils.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.storage.am.btree.impls.BTree;
import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManager;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import org.apache.hyracks.storage.common.MultiComparator;
@@ -43,9 +44,10 @@
public static BTree createBTree(IBufferCache bufferCache, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType, FileReference file,
- IPageManager freePageManager, boolean updateAware) throws HyracksDataException {
+ IPageManager freePageManager, boolean updateAware, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
- new BTreeTypeAwareTupleWriterFactory(typeTraits, updateAware);
+ new BTreeTypeAwareTupleWriterFactory(typeTraits, updateAware, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
return new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
@@ -54,9 +56,10 @@
public static DiskBTree createDiskBTree(IBufferCache bufferCache, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType, FileReference file,
- IPageManager freePageManager, boolean updateAware) throws HyracksDataException {
+ IPageManager freePageManager, boolean updateAware, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
- new BTreeTypeAwareTupleWriterFactory(typeTraits, updateAware);
+ new BTreeTypeAwareTupleWriterFactory(typeTraits, updateAware, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
return new DiskBTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
@@ -65,9 +68,10 @@
public static BTree createBTree(IBufferCache bufferCache, IPageManager freePageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType, FileReference file,
- boolean updateAware) throws HyracksDataException {
+ boolean updateAware, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
- new BTreeTypeAwareTupleWriterFactory(typeTraits, updateAware);
+ new BTreeTypeAwareTupleWriterFactory(typeTraits, updateAware, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
return new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 0600add..b42337c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -295,7 +295,7 @@
recordDescProvider.getInputRecordDescriptor(new ActivityId(new OperatorDescriptorId(0), 0),
0),
keys, keys, lowKeyInclusive, highKeyInclusive, keys, keys, pair.getLeft(), false, false,
- null, pair.getRight(), false);
+ null, pair.getRight(), false, null);
writers.add(writer);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/INullIntrospector.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/INullIntrospector.java
new file mode 100644
index 0000000..e159612
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/INullIntrospector.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+/**
+ * An introspector that tells whether data is {@code NULL} or not.
+ */
+@FunctionalInterface
+public interface INullIntrospector extends Serializable, IJsonSerializable {
+
+ boolean isNull(byte[] bytes, int offset, int length);
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index a5bc206..eab8c96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -89,6 +89,7 @@
protected PermutingFrameTupleReference maxFilterKey;
protected final boolean appendIndexFilter;
protected ArrayTupleBuilder nonFilterTupleBuild;
+ protected IMissingWriter nonFilterWriter;
protected final ISearchOperationCallbackFactory searchCallbackFactory;
protected boolean failed = false;
protected IOperatorStats stats;
@@ -110,28 +111,31 @@
// no filter and limit pushdown
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
- ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
- throws HyracksDataException {
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
+ ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
- false, null, null);
+ retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, null, -1, false, null, null);
}
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
- ITupleFilterFactory tupleFactoryFactory, long outputLimit, boolean appendSearchCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
- throws HyracksDataException {
+ IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFactoryFactory, long outputLimit,
+ boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
this.ctx = ctx;
this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
this.retainInput = retainInput;
this.retainMissing = retainMissing;
this.appendIndexFilter = appendIndexFilter;
- if (this.retainMissing || this.appendIndexFilter) {
- this.nonMatchWriter = missingWriterFactory.createMissingWriter();
+ if (this.retainMissing) {
+ this.nonMatchWriter = nonMatchWriterFactory.createMissingWriter();
+ }
+ if (this.appendIndexFilter) {
+ this.nonFilterWriter = nonFilterWriterFactory.createMissingWriter();
}
this.inputRecDesc = inputRecDesc;
this.searchCallbackFactory = searchCallbackFactory;
@@ -197,7 +201,7 @@
if (appendIndexFilter) {
int numIndexFilterFields = index.getNumOfFilterFields();
nonFilterTupleBuild = new ArrayTupleBuilder(numIndexFilterFields);
- buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonMatchWriter);
+ buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonFilterWriter);
}
if (tupleFilterFactory != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index 81e528b..11368bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -19,9 +19,6 @@
package org.apache.hyracks.storage.am.common.impls;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,17 +27,9 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import org.apache.hyracks.storage.common.IIndexBulkLoader;
-import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.storage.common.buffercache.HaltOnFailureCallback;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
-import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
-import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
-import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
-import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
public abstract class AbstractTreeIndex implements ITreeIndex {
@@ -229,118 +218,6 @@
return fieldCount;
}
- public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
- protected final MultiComparator cmp;
- protected final int slotSize;
- protected final int leafMaxBytes;
- protected final int interiorMaxBytes;
- protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
- protected final ITreeIndexMetadataFrame metaFrame;
- protected final ITreeIndexTupleWriter tupleWriter;
- protected ITreeIndexFrame leafFrame;
- protected ITreeIndexFrame interiorFrame;
- // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
- // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
- // that tree are allowed. Currently, this is not enforced.
- protected boolean releasedLatches;
- private final IFIFOPageWriter pageWriter;
- protected List<ICachedPage> pagesToWrite;
- private final ICompressedPageWriter compressedPageWriter;
-
- public AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
- leafFrame = leafFrameFactory.createFrame();
- interiorFrame = interiorFrameFactory.createFrame();
- metaFrame = freePageManager.createMetadataFrame();
-
- pageWriter = bufferCache.createFIFOWriter(callback, this);
-
- if (!isEmptyTree(leafFrame)) {
- throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
- }
-
- this.cmp = MultiComparator.create(cmpFactories);
-
- leafFrame.setMultiComparator(cmp);
- interiorFrame.setMultiComparator(cmp);
-
- tupleWriter = leafFrame.getTupleWriter();
-
- NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());
- leafFrontier.pageId = freePageManager.takePage(metaFrame);
- leafFrontier.page =
- bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
-
- interiorFrame.setPage(leafFrontier.page);
- interiorFrame.initBuffer((byte) 0);
- interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
-
- leafFrame.setPage(leafFrontier.page);
- leafFrame.initBuffer((byte) 0);
- leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
- slotSize = leafFrame.getSlotSize();
-
- nodeFrontiers.add(leafFrontier);
- pagesToWrite = new ArrayList<>();
- compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
- }
-
- protected void handleException() {
- // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
- compressedPageWriter.abort();
- for (NodeFrontier nodeFrontier : nodeFrontiers) {
- if (nodeFrontier != null && nodeFrontier.page != null) {
- ICachedPage frontierPage = nodeFrontier.page;
- if (frontierPage.confiscated()) {
- bufferCache.returnPage(frontierPage, false);
- }
- }
- }
- for (ICachedPage pageToDiscard : pagesToWrite) {
- if (pageToDiscard != null) {
- bufferCache.returnPage(pageToDiscard, false);
- }
- }
- releasedLatches = true;
- }
-
- @Override
- public void end() throws HyracksDataException {
- if (hasFailed()) {
- throw HyracksDataException.create(getFailure());
- }
- freePageManager.setRootPageId(rootPage);
- }
-
- protected void addLevel() throws HyracksDataException {
- NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
- frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
- frontier.pageId = -1;
- frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
- interiorFrame.setPage(frontier.page);
- interiorFrame.initBuffer((byte) nodeFrontiers.size());
- nodeFrontiers.add(frontier);
- }
-
- public ITreeIndexFrame getLeafFrame() {
- return leafFrame;
- }
-
- public void setLeafFrame(ITreeIndexFrame leafFrame) {
- this.leafFrame = leafFrame;
- }
-
- public void write(ICachedPage cPage) throws HyracksDataException {
- compressedPageWriter.prepareWrite(cPage);
- pageWriter.write(cPage);
- }
-
- @Override
- public void force() throws HyracksDataException {
- bufferCache.force(fileId, false);
- }
-
- }
-
public IBinaryComparatorFactory[] getCmpFactories() {
return cmpFactories;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
new file mode 100644
index 0000000..45a88a7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/AbstractTreeIndexBulkLoader.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexMetadataFrame;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.storage.common.buffercache.IFIFOPageWriter;
+import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
+import org.apache.hyracks.storage.common.buffercache.PageWriteFailureCallback;
+import org.apache.hyracks.storage.common.compression.file.ICompressedPageWriter;
+import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+
+public abstract class AbstractTreeIndexBulkLoader extends PageWriteFailureCallback implements IIndexBulkLoader {
+ protected final IBufferCache bufferCache;
+ protected final IPageManager freePageManager;
+ protected final AbstractTreeIndex treeIndex;
+ protected final int fileId;
+ protected final MultiComparator cmp;
+ protected final int slotSize;
+ protected final int leafMaxBytes;
+ protected final int interiorMaxBytes;
+ protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<>();
+ protected final ITreeIndexMetadataFrame metaFrame;
+ protected final ITreeIndexTupleWriter tupleWriter;
+ protected ITreeIndexFrame leafFrame;
+ protected ITreeIndexFrame interiorFrame;
+ // Immutable bulk loaders write their root page at page -2, as needed e.g. by append-only file systems such as
+ // HDFS. Since loading this tree relies on the root page actually being at that point, no further inserts into
+ // that tree are allowed. Currently, this is not enforced.
+ protected boolean releasedLatches;
+ private final IFIFOPageWriter pageWriter;
+ protected List<ICachedPage> pagesToWrite;
+ private final ICompressedPageWriter compressedPageWriter;
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index)
+ throws HyracksDataException {
+ this(fillFactor, callback, index, index.getLeafFrameFactory().createFrame());
+ }
+
+ protected AbstractTreeIndexBulkLoader(float fillFactor, IPageWriteCallback callback, ITreeIndex index,
+ ITreeIndexFrame leafFrame) throws HyracksDataException {
+ this.bufferCache = index.getBufferCache();
+ this.freePageManager = index.getPageManager();
+ this.fileId = index.getFileId();
+ this.treeIndex = (AbstractTreeIndex) index;
+ this.leafFrame = leafFrame;
+ interiorFrame = treeIndex.getInteriorFrameFactory().createFrame();
+ metaFrame = freePageManager.createMetadataFrame();
+
+ pageWriter = bufferCache.createFIFOWriter(callback, this);
+
+ if (!treeIndex.isEmptyTree(leafFrame)) {
+ throw HyracksDataException.create(ErrorCode.CANNOT_BULK_LOAD_NON_EMPTY_TREE);
+ }
+
+ this.cmp = MultiComparator.create(treeIndex.getCmpFactories());
+
+ leafFrame.setMultiComparator(cmp);
+ interiorFrame.setMultiComparator(cmp);
+
+ tupleWriter = leafFrame.getTupleWriter();
+ NodeFrontier leafFrontier = new NodeFrontier(createTupleReference());
+ leafFrontier.pageId = freePageManager.takePage(metaFrame);
+ leafFrontier.page = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId));
+
+ interiorFrame.setPage(leafFrontier.page);
+ interiorFrame.initBuffer((byte) 0);
+ interiorMaxBytes = (int) (interiorFrame.getBuffer().capacity() * fillFactor);
+
+ leafFrame.setPage(leafFrontier.page);
+ leafFrame.initBuffer((byte) 0);
+ leafMaxBytes = (int) (leafFrame.getBuffer().capacity() * fillFactor);
+ slotSize = leafFrame.getSlotSize();
+
+ nodeFrontiers.add(leafFrontier);
+ pagesToWrite = new ArrayList<>();
+ compressedPageWriter = bufferCache.getCompressedPageWriter(fileId);
+ }
+
+ protected ITreeIndexTupleReference createTupleReference() {
+ return leafFrame.createTupleReference();
+ }
+
+ protected void handleException() {
+ // Unlatch and unpin pages that weren't in the queue to avoid leaking memory.
+ compressedPageWriter.abort();
+ for (NodeFrontier nodeFrontier : nodeFrontiers) {
+ if (nodeFrontier != null && nodeFrontier.page != null) {
+ ICachedPage frontierPage = nodeFrontier.page;
+ if (frontierPage.confiscated()) {
+ bufferCache.returnPage(frontierPage, false);
+ }
+ }
+ }
+ for (ICachedPage pageToDiscard : pagesToWrite) {
+ if (pageToDiscard != null) {
+ bufferCache.returnPage(pageToDiscard, false);
+ }
+ }
+ releasedLatches = true;
+ }
+
+ @Override
+ public void end() throws HyracksDataException {
+ if (hasFailed()) {
+ throw HyracksDataException.create(getFailure());
+ }
+ freePageManager.setRootPageId(treeIndex.getRootPageId());
+ }
+
+ protected void setRootPageId(int rootPage) {
+ treeIndex.rootPage = rootPage;
+ }
+
+ protected void addLevel() throws HyracksDataException {
+ NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
+ frontier.page = bufferCache.confiscatePage(IBufferCache.INVALID_DPID);
+ frontier.pageId = -1;
+ frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
+ interiorFrame.setPage(frontier.page);
+ interiorFrame.initBuffer((byte) nodeFrontiers.size());
+ nodeFrontiers.add(frontier);
+ }
+
+ public ITreeIndexFrame getLeafFrame() {
+ return leafFrame;
+ }
+
+ public void setLeafFrame(ITreeIndexFrame leafFrame) {
+ this.leafFrame = leafFrame;
+ }
+
+ public void write(ICachedPage cPage) throws HyracksDataException {
+ compressedPageWriter.prepareWrite(cPage);
+ pageWriter.write(cPage);
+ }
+
+ @Override
+ public void force() throws HyracksDataException {
+ bufferCache.force(fileId, false);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/SimpleTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/SimpleTupleWriter.java
index ca7217e..6273b68 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/SimpleTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/SimpleTupleWriter.java
@@ -69,6 +69,7 @@
int runner = targetOff;
int nullFlagsBytes = getNullFlagsBytes(tuple);
int fieldSlotsBytes = getFieldSlotsBytes(tuple);
+ // TODO(ali): fix if needed
for (int i = 0; i < nullFlagsBytes; i++) {
targetBuf[runner++] = (byte) 0;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
index 11f86c9..084c89f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
@@ -35,11 +35,13 @@
protected int dataStartOff;
protected final ITypeTraits[] typeTraits;
- protected VarLenIntDecoder encDec = VarLenIntEncoderDecoder.createDecoder();
+ protected final ITypeTraits nullTypeTraits; // can be null
+ protected final VarLenIntDecoder encDec = VarLenIntEncoderDecoder.createDecoder();
protected int[] decodedFieldSlots;
- public TypeAwareTupleReference(ITypeTraits[] typeTraits) {
+ public TypeAwareTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
this.typeTraits = typeTraits;
+ this.nullTypeTraits = nullTypeTraits;
this.fieldStartIndex = 0;
setFieldCount(typeTraits.length);
}
@@ -57,11 +59,12 @@
for (int i = fieldStartIndex; i < end; i++) {
if (!typeTraits[i].isFixedLength()) {
cumul += encDec.decode();
- decodedFieldSlots[field++] = cumul;
+ } else if (nullTypeTraits != null && isNull(buf, tupleStartOff, i)) {
+ cumul += nullTypeTraits.getFixedLength();
} else {
cumul += typeTraits[i].getFixedLength();
- decodedFieldSlots[field++] = cumul;
}
+ decodedFieldSlots[field++] = cumul;
}
dataStartOff = encDec.getPos();
}
@@ -127,4 +130,21 @@
public int getTupleSize() {
return dataStartOff - tupleStartOff + decodedFieldSlots[fieldCount - 1];
}
+
+ protected boolean isNull(byte[] flags, int flagsOffset, int fieldIdx) {
+ int adjustedFieldIdx = getAdjustedFieldIdx(fieldIdx);
+ int flagByteIdx = adjustedFieldIdx / 8;
+ int flagBitIdx = 7 - (adjustedFieldIdx % 8);
+ return BitOperationUtils.getBit(flags, flagsOffset + flagByteIdx, (byte) flagBitIdx);
+ }
+
+ /**
+ * Adjusts the field index in case the null flags section starts with some other special-purpose fields.
+ *
+ * @param fieldIdx logical field index
+ * @return adjusted field index
+ */
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ return fieldIdx;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
index eb6fe2a..882be88 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
@@ -30,10 +31,14 @@
public class TypeAwareTupleWriter implements ITreeIndexTupleWriter {
protected final ITypeTraits[] typeTraits;
- protected VarLenIntEncoderDecoder.VarLenIntDecoder decoder = VarLenIntEncoderDecoder.createDecoder();
+ protected final ITypeTraits nullTypeTraits; // can be null
+ protected final INullIntrospector nullIntrospector; // can be null
- public TypeAwareTupleWriter(ITypeTraits[] typeTraits) {
+ public TypeAwareTupleWriter(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
this.typeTraits = typeTraits;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
@@ -56,7 +61,7 @@
@Override
public TypeAwareTupleReference createTupleReference() {
- return new TypeAwareTupleReference(typeTraits);
+ return new TypeAwareTupleReference(typeTraits, nullTypeTraits);
}
@Override
@@ -68,7 +73,7 @@
public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
int runner = targetOff;
int nullFlagsBytes = getNullFlagsBytes(tuple);
- // write null indicator bits
+ // reset null indicator bits
for (int i = 0; i < nullFlagsBytes; i++) {
targetBuf[runner++] = (byte) 0;
}
@@ -80,10 +85,16 @@
}
}
- // write data fields
+ // write data fields and set null indicator bits
for (int i = 0; i < tuple.getFieldCount(); i++) {
- System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
- runner += tuple.getFieldLength(i);
+ byte[] fieldData = tuple.getFieldData(i);
+ int fieldOffset = tuple.getFieldStart(i);
+ int fieldLength = tuple.getFieldLength(i);
+ if (nullIntrospector != null && nullIntrospector.isNull(fieldData, fieldOffset, fieldLength)) {
+ setNullFlag(targetBuf, targetOff, i);
+ }
+ System.arraycopy(fieldData, fieldOffset, targetBuf, runner, fieldLength);
+ runner += fieldLength;
}
return runner - targetOff;
@@ -93,7 +104,7 @@
public int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf, int targetOff) {
int runner = targetOff;
int nullFlagsBytes = getNullFlagsBytes(numFields);
- // write null indicator bits
+ // reset null indicator bits
for (int i = 0; i < nullFlagsBytes; i++) {
targetBuf[runner++] = (byte) 0;
}
@@ -105,9 +116,15 @@
}
}
- for (int i = startField; i < startField + numFields; i++) {
- System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
- runner += tuple.getFieldLength(i);
+ for (int i = startField, targetField = 0; i < startField + numFields; i++, targetField++) {
+ byte[] fieldData = tuple.getFieldData(i);
+ int fieldOffset = tuple.getFieldStart(i);
+ int fieldLength = tuple.getFieldLength(i);
+ if (nullIntrospector != null && nullIntrospector.isNull(fieldData, fieldOffset, fieldLength)) {
+ setNullFlag(targetBuf, targetOff, targetField);
+ }
+ System.arraycopy(fieldData, fieldOffset, targetBuf, runner, fieldLength);
+ runner += fieldLength;
}
return runner - targetOff;
@@ -145,8 +162,40 @@
return typeTraits;
}
+ public ITypeTraits getNullTypeTraits() {
+ return nullTypeTraits;
+ }
+
+ public INullIntrospector getNullIntrospector() {
+ return nullIntrospector;
+ }
+
@Override
public int getCopySpaceRequired(ITupleReference tuple) {
return bytesRequired(tuple);
}
+
+ /**
+ * Given a field index, this method finds its corresponding bit in the null flags section and sets it.
+ *
+ * @param flags data
+ * @param flagsOffset start of the null flags data
+ * @param fieldIdx logical field index
+ */
+ protected void setNullFlag(byte[] flags, int flagsOffset, int fieldIdx) {
+ int adjustedFieldIdx = getAdjustedFieldIdx(fieldIdx);
+ int flagByteIdx = adjustedFieldIdx / 8;
+ int flagBitIdx = 7 - (adjustedFieldIdx % 8);
+ BitOperationUtils.setBit(flags, flagsOffset + flagByteIdx, (byte) flagBitIdx);
+ }
+
+ /**
+ * Adjusts the field index in case the null flags section starts with some other special-purpose fields.
+ *
+ * @param fieldIdx logical field index
+ * @return adjusted field index
+ */
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ return fieldIdx;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
index a0fa505..586f243 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriterFactory.java
@@ -20,20 +20,26 @@
package org.apache.hyracks.storage.am.common.tuples;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
public class TypeAwareTupleWriterFactory implements ITreeIndexTupleWriterFactory {
- private static final long serialVersionUID = 1L;
- protected ITypeTraits[] typeTraits;
+ private static final long serialVersionUID = 2L;
+ protected final ITypeTraits[] typeTraits;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
- public TypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
+ public TypeAwareTupleWriterFactory(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
this.typeTraits = typeTraits;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
public TypeAwareTupleWriter createTupleWriter() {
- return new TypeAwareTupleWriter(typeTraits);
+ return new TypeAwareTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java
deleted file mode 100644
index 6f2850d..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/IndexFileNameUtil.java
+++ /dev/null
@@ -1,44 +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.
- */
-
-package org.apache.hyracks.storage.am.common.util;
-
-import java.io.File;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.io.FileSplit;
-import org.apache.hyracks.api.io.IIOManager;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-
-public class IndexFileNameUtil {
-
- public static final String IO_DEVICE_NAME_PREFIX = "device_id_";
-
- @Deprecated
- public static String prepareFileName(String path, int ioDeviceId) {
- return path + File.separator + IO_DEVICE_NAME_PREFIX + ioDeviceId;
- }
-
- public static FileReference getIndexAbsoluteFileRef(IFileSplitProvider fileSplitProvider, int partition,
- IIOManager ioManager) throws HyracksDataException {
- FileSplit split = fileSplitProvider.getFileSplits()[partition];
- return split.getFileReference(ioManager);
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
index d94a5e1..e16baf9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResource.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -54,12 +55,13 @@
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter) {
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, path,
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, null, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
private ExternalBTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
@@ -80,7 +82,7 @@
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
ioOpCallbackFactory, pageWriteCallbackFactory, durable, metadataPageManagerFactory,
- serviceCtx.getTracer());
+ serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
index f398370..e4c29ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeLocalResourceFactory.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -45,12 +46,13 @@
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter) {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, bloomFilterKeyFields,
bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
- hasBloomFilter);
+ hasBloomFilter, nullTypeTraits, nullIntrospector);
}
@Override
@@ -59,6 +61,6 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, bloomFilterKeyFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- ioSchedulerProvider, durable, hasBloomFilter);
+ ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
index bf4b06c..863ae56 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResource.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -57,12 +58,13 @@
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory,
- ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter) {
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable, boolean hasBloomFilter,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(typeTraits, cmpFactories, buddyBtreeFields, bloomFilterFalsePositiveRate, isPrimary, path, storageManager,
mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields,
filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, null, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
private ExternalBTreeWithBuddyLocalResource(IPersistedResourceRegistry registry, JsonNode json,
@@ -83,7 +85,7 @@
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
ioOpCallbackFactory, pageWriteCallbackFactory, bloomFilterKeyFields, durable,
- metadataPageManagerFactory, serviceCtx.getTracer());
+ metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
index 91ebee1..39c8aed 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/ExternalBTreeWithBuddyLocalResourceFactory.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -45,12 +46,13 @@
IMetadataPageManagerFactory metadataPageManagerFactory,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] buddyBtreeFields,
- double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter) {
+ double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields, boolean hasBloomFilter,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, buddyBtreeFields,
bloomFilterFalsePositiveRate, isPrimary, btreeFields, NoOpCompressorDecompressorFactory.INSTANCE,
- hasBloomFilter);
+ hasBloomFilter, nullTypeTraits, nullIntrospector);
}
@Override
@@ -59,6 +61,6 @@
bloomFilterFalsePositiveRate, isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory,
mergePolicyProperties, filterTypeTraits, filterCmpFactories, btreeFields, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- ioSchedulerProvider, durable, hasBloomFilter);
+ ioSchedulerProvider, durable, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
index 23be280..3800d17 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
@@ -41,7 +41,7 @@
long outputLimit) {
super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, false, tupleFilterFactory, outputLimit, false, null, null);
+ maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
index dddaab1..4e024c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeBatchPointSearchCursor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
@@ -51,14 +52,15 @@
ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterKeyFields, maxFilterKeyFields, indexHelperFactory, retainInput, retainMissing,
- missingWriterFactory, searchCallbackFactory, false, tupleFilterFactory, outputLimit, false, null, null);
+ missingWriterFactory, searchCallbackFactory, false, null, tupleFilterFactory, outputLimit, false, null,
+ null);
this.keyFields = lowKeyFields;
}
@Override
protected IIndexCursor createCursor() throws HyracksDataException {
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
- return new LSMBTreeBatchPointSearchCursor(lsmAccessor.getOpContext());
+ return ((LSMBTree) index).createBatchPointSearchCursor(lsmAccessor.getOpContext());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
index 860d36e..4aa094b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
@@ -37,7 +37,7 @@
RecordDescriptor inputRecDesc, IIndexDataflowHelperFactory indexHelperFactory,
ISearchOperationCallbackFactory searchCallbackFactory) throws HyracksDataException {
super(ctx, inputRecDesc, partition, null, null, indexHelperFactory, false, false, null, searchCallbackFactory,
- false);
+ false, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
index 6098a1d..656ae02 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResource.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -68,10 +69,12 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, boolean durable,
- ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter) {
+ ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.bloomFilterKeyFields = bloomFilterKeyFields;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
this.isPrimary = isPrimary;
@@ -108,7 +111,7 @@
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
ioOpCallbackFactory, pageWriteCallbackFactory, isPrimary, filterTypeTraits, filterCmpFactories,
btreeFields, filterFields, durable, metadataPageManagerFactory, updateAware, serviceCtx.getTracer(),
- compressorDecompressorFactory, hasBloomFilter);
+ compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
index a2c16b8..e926a48 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeLocalResourceFactory.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -57,10 +58,12 @@
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, boolean durable, int[] bloomFilterKeyFields,
double bloomFilterFalsePositiveRate, boolean isPrimary, int[] btreeFields,
- ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter) {
+ ICompressorDecompressorFactory compressorDecompressorFactory, boolean hasBloomFilter,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.hasBloomFilter = hasBloomFilter;
this.bloomFilterKeyFields = bloomFilterKeyFields;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
@@ -75,7 +78,7 @@
isPrimary, fileRef.getRelativePath(), storageManager, mergePolicyFactory, mergePolicyProperties,
filterTypeTraits, filterCmpFactories, btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
- compressorDecompressorFactory, hasBloomFilter);
+ compressorDecompressorFactory, hasBloomFilter, nullTypeTraits, nullIntrospector);
}
private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index c3d1416..d2fbbef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -341,7 +341,7 @@
try {
List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
long numElements = getNumberOfElements(mergedComponents);
- mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
+ mergedComponent = createDiskComponent(getMergeComponentFactory(), mergeOp.getTarget(), null,
mergeOp.getBloomFilterTarget(), true);
IPageWriteCallback pageWriteCallback = pageWriteCallbackFactory.createPageWriteCallback();
componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false,
@@ -418,7 +418,7 @@
}
public ILSMIndexAccessor createAccessor(AbstractLSMIndexOperationContext opCtx) {
- return new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
+ return new LSMTreeIndexAccessor(getHarness(), opCtx, getCursorFactory());
}
@Override
@@ -483,8 +483,28 @@
returnDeletedTuples = true;
}
IIndexCursorStats stats = new IndexCursorStats();
- LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+ LSMBTreeRangeSearchCursor cursor = createCursor(opCtx, returnDeletedTuples, stats);
return new LSMBTreeMergeOperation(accessor, cursor, stats, mergeFileRefs.getInsertIndexFileReference(),
mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
}
+
+ public LSMBTreeBatchPointSearchCursor createBatchPointSearchCursor(ILSMIndexOperationContext opCtx) {
+ return new LSMBTreeBatchPointSearchCursor(opCtx);
+ }
+
+ protected LSMBTreeRangeSearchCursor createCursor(AbstractLSMIndexOperationContext opCtx,
+ boolean returnDeletedTuples, IIndexCursorStats stats) {
+ return new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples, stats);
+ }
+
+ /**
+ * @return Merge component factory (could be different from {@link #componentFactory}
+ */
+ protected ILSMDiskComponentFactory getMergeComponentFactory() {
+ return componentFactory;
+ }
+
+ protected ICursorFactory getCursorFactory() {
+ return cursorFactory;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1312e30..a00e10e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -43,7 +43,7 @@
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.util.trace.ITracer;
-public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
+public class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
/*
* Finals
@@ -74,9 +74,9 @@
IBinaryComparatorFactory[] filterCmpFactories, ITracer tracer) {
super(index, btreeFields, filterFields, filterCmpFactories, searchCallback, modificationCallback, tracer);
LSMBTreeMemoryComponent c = (LSMBTreeMemoryComponent) mutableComponents.get(0);
- IBinaryComparatorFactory cmpFactories[] = c.getIndex().getComparatorFactories();
+ IBinaryComparatorFactory[] cmpFactories = c.getIndex().getComparatorFactories();
if (cmpFactories[0] != null) {
- this.cmp = MultiComparator.create(c.getIndex().getComparatorFactories());
+ this.cmp = createMultiComparator(c.getIndex().getComparatorFactories());
} else {
this.cmp = null;
}
@@ -112,6 +112,10 @@
insertSearchCursor = new LSMBTreePointSearchCursor(this);
}
+ protected MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+ return MultiComparator.create(cmpFactories);
+ }
+
@Override
public void setOperation(IndexOperation newOp) {
reset();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index d4903d9..7bb96c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -199,7 +199,8 @@
for (int i = 0; i < numBTrees; i++) {
ILSMComponent component = operationalComponents.get(i);
BTree btree = (BTree) component.getIndex();
- if (component.getType() == LSMComponentType.MEMORY) {
+ LSMComponentType type = component.getType();
+ if (type == LSMComponentType.MEMORY) {
includeMutableComponent = true;
if (bloomFilters[i] != null) {
destroyAndNullifyCursorAtIndex(i);
@@ -212,8 +213,8 @@
}
if (btreeAccessors[i] == null) {
- btreeAccessors[i] = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- btreeCursors[i] = btreeAccessors[i].createPointCursor(false, false);
+ btreeAccessors[i] = createAccessor(type, btree, i);
+ btreeCursors[i] = createCursor(type, btreeAccessors[i]);
} else {
// re-use
btreeAccessors[i].reset(btree, NoOpIndexAccessParameters.INSTANCE);
@@ -225,6 +226,14 @@
hashComputed = false;
}
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int i) throws HyracksDataException {
+ return btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ }
+
+ protected ITreeIndexCursor createCursor(LSMComponentType type, BTreeAccessor btreeAccessor) {
+ return btreeAccessor.createPointCursor(false, type == LSMComponentType.DISK);
+ }
+
private void destroyAndNullifyCursorAtIndex(int i) throws HyracksDataException {
// component at location i was a disk component before, and is now a memory component, or vise versa
bloomFilters[i] = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 46d279f..968416c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -49,7 +49,7 @@
private final RangePredicate reusablePred;
private ISearchOperationCallback searchCallback;
private BTreeAccessor[] btreeAccessors;
- private boolean[] isMemoryComponent;
+ protected boolean[] isMemoryComponent;
private ArrayTupleBuilder tupleBuilder;
private boolean canCallProceed = true;
private boolean resultOfSearchCallbackProceed = false;
@@ -149,6 +149,7 @@
// There are no more elements in the memory component.. can safely skip locking for the
// remaining operations
includeMutableComponent = false;
+ excludeMemoryComponent();
}
}
} else {
@@ -180,6 +181,7 @@
// the tree of head tuple
// the head element of PQ is useless now
PriorityQueueElement e = outputPriorityQueue.poll();
+ markAsDeleted(e);
pushIntoQueueFromCursorAndReplaceThisElement(e);
} else {
// If the previous tuple and the head tuple are different
@@ -200,6 +202,14 @@
}
+ protected void excludeMemoryComponent() {
+ //NoOp
+ }
+
+ protected void markAsDeleted(PriorityQueueElement e) throws HyracksDataException {
+ //NoOp
+ }
+
private void pushOutputElementIntoQueueIfNeeded() throws HyracksDataException {
if (needPushElementIntoQueue) {
pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
@@ -236,16 +246,21 @@
rangeCursors[i].close();
btreeAccessors[i].reset(btree, iap);
btreeAccessors[i].search(rangeCursors[i], reusablePred);
- pushIntoQueueFromCursorAndReplaceThisElement(switchedElements[i]);
+ // consume the element that we restarted the search at since before the switch it was consumed
+ if (rangeCursors[i].hasNext()) {
+ rangeCursors[i].next();
+ switchedElements[i].reset(rangeCursors[i].getTuple());
+ }
}
}
switchRequest[i] = false;
+ switchedElements[i] = null;
// any failed switch makes further switches pointless
switchPossible = switchPossible && operationalComponents.get(i).getType() == LSMComponentType.DISK;
}
}
- private int replaceFrom() throws HyracksDataException {
+ protected int replaceFrom() throws HyracksDataException {
int replaceFrom = -1;
if (!switchPossible) {
return replaceFrom;
@@ -264,14 +279,18 @@
if (replaceFrom < 0) {
replaceFrom = i;
}
- // we return the outputElement to the priority queue if it came from this component
+
+ PriorityQueueElement element;
if (outputElement != null && outputElement.getCursorIndex() == i) {
- pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
- needPushElementIntoQueue = false;
- outputElement = null;
- canCallProceed = true;
+ // there should be no element from this cursor in the queue since the element was polled
+ if (findElement(outputPriorityQueue, i) != null) {
+ throw new IllegalStateException("found element in the queue from the cursor of output element");
+ }
+ element = outputElement;
+ } else {
+ element = findElement(outputPriorityQueue, i);
}
- PriorityQueueElement element = remove(outputPriorityQueue, i);
+
// if this cursor is still active (has an element)
// then we copy the search key to restart the operation after
// replacing the component
@@ -331,6 +350,18 @@
return null;
}
+ private PriorityQueueElement findElement(PriorityQueue<PriorityQueueElement> outputPriorityQueue, int cursorIndex) {
+ // Scans the PQ for the component's element
+ Iterator<PriorityQueueElement> it = outputPriorityQueue.iterator();
+ while (it.hasNext()) {
+ PriorityQueueElement e = it.next();
+ if (e.getCursorIndex() == cursorIndex) {
+ return e;
+ }
+ }
+ return null;
+ }
+
@Override
public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
@@ -365,20 +396,21 @@
}
for (int i = 0; i < numBTrees; i++) {
ILSMComponent component = operationalComponents.get(i);
+ LSMComponentType type = component.getType();
BTree btree;
if (component.getType() == LSMComponentType.MEMORY) {
includeMutableComponent = true;
}
btree = (BTree) component.getIndex();
if (btreeAccessors[i] == null || destroyIncompatible(component, i)) {
- btreeAccessors[i] = btree.createAccessor(iap);
- rangeCursors[i] = btreeAccessors[i].createSearchCursor(false);
+ btreeAccessors[i] = createAccessor(type, btree, i);
+ rangeCursors[i] = createCursor(type, btreeAccessors[i]);
} else {
// re-use
btreeAccessors[i].reset(btree, iap);
rangeCursors[i].close();
}
- isMemoryComponent[i] = component.getType() == LSMComponentType.MEMORY;
+ isMemoryComponent[i] = type == LSMComponentType.MEMORY;
}
IndexCursorUtils.open(btreeAccessors, rangeCursors, searchPred);
try {
@@ -412,4 +444,12 @@
return resultOfSearchCallbackProceed;
}
+ protected BTreeAccessor createAccessor(LSMComponentType type, BTree btree, int index) throws HyracksDataException {
+ return btree.createAccessor(iap);
+ }
+
+ protected IIndexCursor createCursor(LSMComponentType type, BTreeAccessor accessor) {
+ return accessor.createSearchCursor(false);
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index efacad1..aa72267 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -46,6 +46,13 @@
scanCursor = new LSMBTreeDiskComponentScanCursor(opCtx);
}
+ protected LSMBTreeSearchCursor(LSMBTreePointSearchCursor pointCursor, LSMBTreeRangeSearchCursor rangeCursor,
+ LSMBTreeDiskComponentScanCursor scanCursor) {
+ this.pointCursor = pointCursor;
+ this.rangeCursor = rangeCursor;
+ this.scanCursor = scanCursor;
+ }
+
@Override
public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriter.java
index 7c924b5..76916ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriter.java
@@ -21,11 +21,14 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
public class LSMBTreeCopyTupleWriter extends LSMBTreeTupleWriter {
- public LSMBTreeCopyTupleWriter(ITypeTraits[] typeTraits, int numKeyFields, boolean updateAware) {
+
+ public LSMBTreeCopyTupleWriter(ITypeTraits[] typeTraits, int numKeyFields, boolean updateAware,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
// Third parameter is never used locally, just give false.
- super(typeTraits, numKeyFields, false, updateAware);
+ super(typeTraits, numKeyFields, false, updateAware, nullTypeTraits, nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriterFactory.java
index 02cd900..dd43ef4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriterFactory.java
@@ -22,18 +22,20 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriter;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
public class LSMBTreeCopyTupleWriterFactory extends BTreeTypeAwareTupleWriterFactory {
private static final long serialVersionUID = 1L;
private final int numKeyFields;
- public LSMBTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits, int numKeyFields, boolean updateAware) {
- super(typeTraits, updateAware);
+ public LSMBTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits, int numKeyFields, boolean updateAware,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ super(typeTraits, updateAware, nullTypeTraits, nullIntrospector);
this.numKeyFields = numKeyFields;
}
@Override
public BTreeTypeAwareTupleWriter createTupleWriter() {
- return new LSMBTreeCopyTupleWriter(typeTraits, numKeyFields, updateAware);
+ return new LSMBTreeCopyTupleWriter(typeTraits, numKeyFields, updateAware, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleReference.java
index 2a5e232..e5c365a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleReference.java
@@ -33,8 +33,9 @@
private boolean resetFieldCount = false;
private final int numKeyFields;
- public LSMBTreeTupleReference(ITypeTraits[] typeTraits, int numKeyFields, boolean updateAware) {
- super(typeTraits, updateAware);
+ public LSMBTreeTupleReference(ITypeTraits[] typeTraits, int numKeyFields, boolean updateAware,
+ ITypeTraits nullTypeTraits) {
+ super(typeTraits, updateAware, nullTypeTraits);
this.numKeyFields = numKeyFields;
}
@@ -94,4 +95,10 @@
public int getTupleStart() {
return tupleStartOff;
}
+
+ @Override
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ // 2 bits when update-aware: 1 for antimatter and 1 for the update, otherwise, only 1 bit for the antimatter
+ return updateAware ? fieldIdx + 2 : fieldIdx + 1;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
index 09ced10..bcbf93f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriter;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
@@ -33,8 +34,9 @@
private boolean isAntimatter;
private final int numKeyFields;
- public LSMBTreeTupleWriter(ITypeTraits[] typeTraits, int numKeyFields, boolean isAntimatter, boolean updateAware) {
- super(typeTraits, updateAware);
+ public LSMBTreeTupleWriter(ITypeTraits[] typeTraits, int numKeyFields, boolean isAntimatter, boolean updateAware,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ super(typeTraits, updateAware, nullTypeTraits, nullIntrospector);
this.numKeyFields = numKeyFields;
this.isAntimatter = isAntimatter;
}
@@ -56,7 +58,7 @@
@Override
public LSMBTreeTupleReference createTupleReference() {
- return new LSMBTreeTupleReference(typeTraits, numKeyFields, updateAware);
+ return new LSMBTreeTupleReference(typeTraits, numKeyFields, updateAware, nullTypeTraits);
}
@Override
@@ -83,6 +85,7 @@
public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
int bytesWritten = -1;
if (isAntimatter) {
+ // TODO(ali): check this with secondary primary-key index
bytesWritten = super.writeTupleFields(tuple, 0, numKeyFields, targetBuf, targetOff);
// Set antimatter bit to 1.
BitOperationUtils.setBit(targetBuf, targetOff, ANTIMATTER_BIT_OFFSET);
@@ -100,4 +103,10 @@
public void setAntimatter(boolean isDelete) {
this.isAntimatter = isDelete;
}
+
+ @Override
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ // 2 bits when update-aware: 1 for antimatter and 1 for the update, otherwise, only 1 bit for the antimatter
+ return updateAware ? fieldIdx + 2 : fieldIdx + 1;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
index ad4f6eb..03bce23 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
@@ -22,6 +22,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriter;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
public class LSMBTreeTupleWriterFactory extends BTreeTypeAwareTupleWriterFactory {
@@ -30,15 +31,16 @@
private final boolean isAntimatter;
public LSMBTreeTupleWriterFactory(ITypeTraits[] typeTraits, int numKeyFields, boolean isAntimatter,
- boolean updateAware) {
- super(typeTraits, updateAware);
+ boolean updateAware, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ super(typeTraits, updateAware, nullTypeTraits, nullIntrospector);
this.numKeyFields = numKeyFields;
this.isAntimatter = isAntimatter;
}
@Override
public BTreeTypeAwareTupleWriter createTupleWriter() {
- return new LSMBTreeTupleWriter(typeTraits, numKeyFields, isAntimatter, updateAware);
+ return new LSMBTreeTupleWriter(typeTraits, numKeyFields, isAntimatter, updateAware, nullTypeTraits,
+ nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index 021f416..018c0d3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
@@ -76,15 +77,16 @@
boolean needKeyDupCheck, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
int[] btreeFields, int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
boolean updateAware, ITracer tracer, ICompressorDecompressorFactory compressorDecompressorFactory,
- boolean hasBloomFilter) throws HyracksDataException {
- LSMBTreeTupleWriterFactory insertTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware);
- LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, true, updateAware);
- LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory =
- new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length, updateAware);
- LSMBTreeTupleWriterFactory bulkLoadTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware);
+ boolean hasBloomFilter, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
+ LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, updateAware, nullTypeTraits, nullIntrospector);
+ LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, true, updateAware, nullTypeTraits, nullIntrospector);
+ LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+ cmpFactories.length, updateAware, nullTypeTraits, nullIntrospector);
+ LSMBTreeTupleWriterFactory bulkLoadTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, updateAware, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
@@ -103,7 +105,8 @@
LSMComponentFilterFrameFactory filterFrameFactory = null;
LSMComponentFilterManager filterManager = null;
if (filterCmpFactories != null) {
- TypeAwareTupleWriterFactory filterTupleWriterFactory = new TypeAwareTupleWriterFactory(filterTypeTraits);
+ TypeAwareTupleWriterFactory filterTupleWriterFactory =
+ new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -136,21 +139,21 @@
int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
- boolean durable, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer)
- throws HyracksDataException {
- LSMBTreeTupleWriterFactory insertTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, false);
- LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, true, false);
- LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory =
- new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length, false);
+ boolean durable, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
+ LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, false, nullTypeTraits, nullIntrospector);
+ LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, true, false, nullTypeTraits, nullIntrospector);
+ LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
// This is the tuple writer that can do both inserts and deletes
- LSMBTreeTupleWriterFactory transactionTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, false);
+ LSMBTreeTupleWriterFactory transactionTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, false, nullTypeTraits, nullIntrospector);
// This is the leaf frame factory for transaction components since it
// can be used for both inserts and deletes
ITreeIndexFrameFactory transactionLeafFrameFactory =
@@ -185,7 +188,8 @@
double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] buddyBTreeFields, boolean durable,
- IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer) throws HyracksDataException {
+ IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
ITypeTraits[] buddyBtreeTypeTraits = new ITypeTraits[buddyBTreeFields.length];
IBinaryComparatorFactory[] buddyBtreeCmpFactories = new IBinaryComparatorFactory[buddyBTreeFields.length];
for (int i = 0; i < buddyBtreeTypeTraits.length; i++) {
@@ -193,15 +197,15 @@
buddyBtreeCmpFactories[i] = cmpFactories[buddyBTreeFields[i]];
}
BTreeTypeAwareTupleWriterFactory buddyBtreeTupleWriterFactory =
- new BTreeTypeAwareTupleWriterFactory(buddyBtreeTypeTraits, false);
+ new BTreeTypeAwareTupleWriterFactory(buddyBtreeTypeTraits, false, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory buddyBtreeInteriorFrameFactory =
new BTreeNSMInteriorFrameFactory(buddyBtreeTupleWriterFactory);
ITreeIndexFrameFactory buddyBtreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(buddyBtreeTupleWriterFactory);
- LSMBTreeTupleWriterFactory insertTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, false);
- LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory =
- new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length, false);
+ LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, false, nullTypeTraits, nullIntrospector);
+ LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+ cmpFactories.length, false, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
index e448ae0..a778a4c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.lsm.common.api;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
public interface ILSMIOOperationCallback {
@@ -80,4 +81,8 @@
* the allocated component
*/
void allocated(ILSMMemoryComponent component) throws HyracksDataException;
+
+ default long getLastValidSequence() throws HyracksDataException {
+ return AbstractLSMIndexFileManager.UNINITIALIZED_COMPONENT_SEQ;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
index 5580387..2d38758 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
@@ -78,4 +78,11 @@
* @throws IOException
*/
LSMComponentFileReferences getNewTransactionFileReference() throws IOException;
+
+ /**
+ * Initializes the last used sequence
+ *
+ * @param lastUsedSeq
+ */
+ void initLastUsedSeq(long lastUsedSeq);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
index fbd74dc..d905a0a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResource.java
@@ -27,6 +27,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -49,7 +50,7 @@
*/
public abstract class LsmResource implements IResource {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
protected String path;
protected final IStorageManager storageManager;
@@ -67,6 +68,8 @@
protected final ILSMMergePolicyFactory mergePolicyFactory;
protected final Map<String, String> mergePolicyProperties;
protected final boolean durable;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
public LsmResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
@@ -75,7 +78,8 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
- Map<String, String> mergePolicyProperties, boolean durable) {
+ Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
this.path = path;
this.storageManager = storageManager;
this.typeTraits = typeTraits;
@@ -92,6 +96,8 @@
this.mergePolicyFactory = mergePolicyFactory;
this.mergePolicyProperties = mergePolicyProperties;
this.durable = durable;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
protected LsmResource(IPersistedResourceRegistry registry, JsonNode json) throws HyracksDataException {
@@ -156,6 +162,16 @@
mergePolicyFactory = (ILSMMergePolicyFactory) registry.deserialize(json.get("mergePolicyFactory"));
mergePolicyProperties = OBJECT_MAPPER.convertValue(json.get("mergePolicyProperties"), Map.class);
durable = json.get("durable").asBoolean();
+ if (json.hasNonNull("nullTypeTraits")) {
+ nullTypeTraits = (ITypeTraits) registry.deserialize(json.get("nullTypeTraits"));
+ } else {
+ nullTypeTraits = null;
+ }
+ if (json.hasNonNull("nullIntrospector")) {
+ nullIntrospector = (INullIntrospector) registry.deserialize(json.get("nullIntrospector"));
+ } else {
+ nullIntrospector = null;
+ }
}
protected void appendToJson(final ObjectNode json, IPersistedResourceRegistry registry)
@@ -206,6 +222,16 @@
json.set("mergePolicyFactory", mergePolicyFactory.toJson(registry));
json.putPOJO("mergePolicyProperties", mergePolicyProperties);
json.put("durable", durable);
+ if (nullTypeTraits != null) {
+ json.set("nullTypeTraits", nullTypeTraits.toJson(registry));
+ } else {
+ json.set("nullTypeTraits", null);
+ }
+ if (nullIntrospector != null) {
+ json.set("nullIntrospector", nullIntrospector.toJson(registry));
+ } else {
+ json.set("nullIntrospector", null);
+ }
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java
index a138f01..e97a779 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LsmResourceFactory.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -34,7 +35,7 @@
public abstract class LsmResourceFactory implements IResourceFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
protected final IStorageManager storageManager;
protected final ITypeTraits[] typeTraits;
protected final IBinaryComparatorFactory[] cmpFactories;
@@ -50,6 +51,8 @@
protected final ILSMMergePolicyFactory mergePolicyFactory;
protected final Map<String, String> mergePolicyProperties;
protected final boolean durable;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
public LsmResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
@@ -58,7 +61,8 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
- Map<String, String> mergePolicyProperties, boolean durable) {
+ Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
this.storageManager = storageManager;
this.typeTraits = typeTraits;
this.cmpFactories = cmpFactories;
@@ -74,5 +78,7 @@
this.mergePolicyFactory = mergePolicyFactory;
this.mergePolicyProperties = mergePolicyProperties;
this.durable = durable;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index a5ff2e6..bf93dc0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -139,6 +139,7 @@
this.inactiveMemoryComponents = new ArrayList<>();
this.durable = durable;
this.tracer = tracer;
+ fileManager.initLastUsedSeq(ioOpCallback.getLastValidSequence());
lsmHarness = new LSMHarness(this, ioScheduler, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(),
tracer);
isActive = false;
@@ -184,6 +185,7 @@
filterManager = null;
treeFields = null;
filterFields = null;
+ fileManager.initLastUsedSeq(ioOpCallback.getLastValidSequence());
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
index 35a11ff..5d70337 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -83,20 +83,19 @@
* Hides transaction components until they are either committed by removing this file or deleted along with the file
*/
public static final String TXN_PREFIX = ".T";
-
+ public static final long UNINITIALIZED_COMPONENT_SEQ = -1;
public static final FilenameFilter COMPONENT_FILES_FILTER = (dir, name) -> !name.startsWith(".");
protected static final FilenameFilter txnFileNameFilter = (dir, name) -> name.startsWith(TXN_PREFIX);
protected static FilenameFilter bloomFilterFilter =
(dir, name) -> !name.startsWith(".") && name.endsWith(BLOOM_FILTER_SUFFIX);
protected static final Comparator<String> cmp = new FileNameComparator();
private static final FilenameFilter dummyFilter = (dir, name) -> true;
- private static final long UNINITALIZED_COMPONENT_SEQ = -1;
protected final IIOManager ioManager;
// baseDir should reflect dataset name and partition name and be absolute
protected final FileReference baseDir;
protected final Comparator<IndexComponentFileReference> recencyCmp = new RecencyComparator();
protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
- private long lastUsedComponentSeq = UNINITALIZED_COMPONENT_SEQ;
+ private long lastUsedComponentSeq = UNINITIALIZED_COMPONENT_SEQ;
private final ICompressorDecompressorFactory compressorDecompressorFactory;
public AbstractLSMIndexFileManager(IIOManager ioManager, FileReference file,
@@ -348,6 +347,11 @@
return null;
}
+ @Override
+ public void initLastUsedSeq(long lastUsedSeq) {
+ lastUsedComponentSeq = lastUsedSeq;
+ }
+
private static FilenameFilter createTransactionFilter(String transactionFileName, final boolean inclusive) {
final String timeStamp =
transactionFileName.substring(transactionFileName.indexOf(TXN_PREFIX) + TXN_PREFIX.length());
@@ -372,7 +376,7 @@
}
protected String getNextComponentSequence(FilenameFilter filenameFilter) throws HyracksDataException {
- if (lastUsedComponentSeq == UNINITALIZED_COMPONENT_SEQ) {
+ if (lastUsedComponentSeq == UNINITIALIZED_COMPONENT_SEQ) {
lastUsedComponentSeq = getOnDiskLastUsedComponentSequence(filenameFilter);
}
return IndexComponentFileReference.getFlushSequence(++lastUsedComponentSeq);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
index dfb1a17..0813a00 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
@@ -171,7 +171,7 @@
public void logPerformanceCounters(int tupleCount) {
if (isTracingEnabled()) {
tracer.instant("store-counters", traceCategory, Scope.t,
- "{\"count\":" + tupleCount + ",\"enter-exit-duration-ns\":" + enterExitTime + "}");
+ () -> "{\"count\":" + tupleCount + ",\"enter-exit-duration-ns\":" + enterExitTime + "}");
resetCounters();
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index 70f6d9e..5b0363a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -193,7 +193,7 @@
for (int i = 0; i < count; i++) {
ILSMComponent removed = ctx.getComponentHolder().remove(swapIndexes[i]);
if (removed.getType() == LSMComponentType.MEMORY) {
- LOGGER.info("Removed a memory component from the search operation");
+ LOGGER.debug("Removed memory component {} from the search operation", removed);
} else {
throw new IllegalStateException("Disk components can't be removed from the search operation");
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 92fa135..950a8e5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -262,7 +262,7 @@
}
if (inactiveMemoryComponentsToBeCleanedUp != null) {
for (ILSMMemoryComponent c : inactiveMemoryComponentsToBeCleanedUp) {
- tracer.instant(c.toString(), traceCategory, Scope.p, lsmIndex.toString());
+ tracer.instant(c.toString(), traceCategory, Scope.p, lsmIndex::toString);
c.cleanup();
synchronized (opTracker) {
c.reset();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
index 13a0e27..acb84e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexBulkLoader.java
@@ -20,7 +20,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex.AbstractTreeIndexBulkLoader;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index b6f6e26..27875c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -40,7 +40,7 @@
import org.apache.hyracks.storage.common.MultiComparator;
public abstract class LSMIndexSearchCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
- protected static final int SWITCH_COMPONENT_CYCLE = 100;
+ public static final int SWITCH_COMPONENT_CYCLE = 100;
protected final ILSMIndexOperationContext opCtx;
protected final boolean returnDeletedTuples;
protected PriorityQueueElement outputElement;
@@ -119,6 +119,7 @@
needPushElementIntoQueue = false;
for (int i = 0; i < switchRequest.length; i++) {
switchRequest[i] = false;
+ switchedElements[i] = null;
}
try {
if (outputPriorityQueue != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
index 4ab57c5..eb48f82 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
@@ -58,7 +58,7 @@
final long traceCategory = tracer.getRegistry().get(TraceUtils.INDEX_IO_OPERATIONS);
if (tracer.isEnabled(traceCategory)) {
tracer.instant("schedule-" + ioOpName, traceCategory, Scope.p,
- "{\"path\": \"" + ioOp.getTarget().getRelativePath() + "\"}");
+ () -> "{\"path\": \"" + ioOp.getTarget().getRelativePath() + "\"}");
return new TracedIOOperation(ioOp, tracer, traceCategory);
}
return ioOp;
@@ -91,11 +91,11 @@
@Override
public LSMIOOperationStatus call() throws HyracksDataException {
final String name = getTarget().getRelativePath();
- final long tid = tracer.durationB(name, traceCategory, null);
+ final long tid = tracer.durationB(name, traceCategory);
try {
return ioOp.call();
} finally {
- tracer.durationE(ioOp.getIOOpertionType().name().toLowerCase(), traceCategory, tid, "{\"size\":"
+ tracer.durationE(ioOp.getIOOpertionType().name().toLowerCase(), traceCategory, tid, () -> "{\"size\":"
+ getTarget().getFile().length() + ", \"path\": \"" + ioOp.getTarget().getRelativePath() + "\"}");
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
index a693a99..ed44dfa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.common.IIndex;
public interface IInvertedIndex extends IIndex {
@@ -32,4 +33,8 @@
ITypeTraits[] getTokenTypeTraits();
IBinaryComparatorFactory[] getTokenCmpFactories();
+
+ ITypeTraits getNullTypeTraits();
+
+ INullIntrospector getNullIntrospector();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 0af1f3f..8a107e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -90,12 +91,13 @@
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+ int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
-
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.tokenizerFactory = tokenizerFactory;
@@ -142,7 +144,7 @@
opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
- metadataPageManagerFactory, serviceCtx.getTracer());
+ metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
} else {
return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, bufferCache,
@@ -150,7 +152,7 @@
opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
- metadataPageManagerFactory, serviceCtx.getTracer());
+ metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
index 09c9699..395331f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -62,10 +63,12 @@
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+ int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
// ToDo: totally replace tokenizerFactory with full-text config
@@ -85,7 +88,8 @@
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, durable, tokenTypeTraits, tokenCmpFactories,
tokenizerFactory, fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate,
+ nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index 2cee54d..b5b951d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -42,6 +42,7 @@
private final int[] minFilterFieldIndexes;
private final int[] maxFilterFieldIndexes;
private final boolean appendIndexFilter;
+ private final IMissingWriterFactory nonFilterWriterFactory;
private final boolean isFullTextSearchQuery;
private final IIndexDataflowHelperFactory indexHelperFactory;
private final IBinaryTokenizerFactory queryTokenizerFactory;
@@ -61,7 +62,7 @@
IInvertedIndexSearchModifierFactory searchModifierFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int numOfFields,
- boolean appendIndexFilter, int frameLimit) {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory, int frameLimit) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.queryTokenizerFactory = queryTokenizerFactory;
@@ -76,6 +77,7 @@
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.isFullTextSearchQuery = isFullTextSearchQuery;
this.appendIndexFilter = appendIndexFilter;
+ this.nonFilterWriterFactory = nonFilterWriterFactory;
this.numOfFields = numOfFields;
this.outRecDescs[0] = outRecDesc;
this.frameLimit = frameLimit;
@@ -89,6 +91,6 @@
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), partition, minFilterFieldIndexes,
maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
searchCallbackFactory, searchModifier, queryTokenizerFactory, fullTextConfigEvaluatorFactory,
- queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, frameLimit);
+ queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, nonFilterWriterFactory, frameLimit);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index e4dbb43..571ae5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -62,10 +62,11 @@
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
IInvertedIndexSearchModifier searchModifier, IBinaryTokenizerFactory binaryTokenizerFactory,
IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, int queryFieldIndex,
- boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter, int frameLimit)
- throws HyracksDataException {
+ boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter,
+ IMissingWriterFactory nonFilterWriterFactory, int frameLimit) throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory);
this.searchModifier = searchModifier;
this.binaryTokenizerFactory = binaryTokenizerFactory;
this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
index 4fa27ca..76a20b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
@@ -22,6 +22,8 @@
import com.fasterxml.jackson.databind.ObjectMapper;
public abstract class AbstractFullTextFilterEvaluatorFactory implements IFullTextFilterEvaluatorFactory {
+ private static final long serialVersionUID = 3535472161323149351L;
+
protected final String name;
protected final FullTextFilterType type;
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
index 7ec6ce2..9886da4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.dataflow.common.utils.TaskUtil;
import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
import org.apache.hyracks.dataflow.std.buffermanager.SingleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -68,6 +69,8 @@
protected int bufferNumLoadedPages;
protected final IInvertedListTupleReference tuple;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
protected final ITypeTraits[] invListFields;
protected ICachedPage page;
// buffer manager to conform to the memory budget
@@ -79,19 +82,22 @@
protected final IIndexCursorStats stats;
protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
- IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
- this(bufferCache, fileId, invListFields, ctx, stats, false);
+ IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
+ this(bufferCache, fileId, invListFields, ctx, stats, false, nullTypeTraits, nullIntrospector);
}
protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
- IIndexCursorStats stats) throws HyracksDataException {
- this(bufferCache, fileId, invListFields, null, stats, true);
+ IIndexCursorStats stats, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
+ this(bufferCache, fileId, invListFields, null, stats, true, nullTypeTraits, nullIntrospector);
}
// If isScan, use the SingleFrameBufferManager to minimize memory cost by allocating only one memory frame,
// elsewhere use a regular buffer manager
protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
- IHyracksTaskContext ctx, IIndexCursorStats stats, boolean isScan) throws HyracksDataException {
+ IHyracksTaskContext ctx, IIndexCursorStats stats, boolean isScan, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
this.bufferCache = bufferCache;
this.fileId = fileId;
@@ -104,7 +110,7 @@
this.lastRandomSearchedElementIx = 0;
this.moreBlocksToRead = true;
this.invListFields = invListFields;
- this.tuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields);
+ this.tuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields, nullTypeTraits);
this.buffers = new ArrayList<ByteBuffer>();
if (ctx == null && !isScan) {
throw HyracksDataException.create(ErrorCode.CANNOT_CONTINUE_TEXT_SEARCH_HYRACKS_TASK_IS_NULL);
@@ -118,6 +124,8 @@
this.bufferManagerForSearch = new SingleFrameBufferManager();
}
this.stats = stats;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index de65e3e..a828d22 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
@@ -92,6 +93,8 @@
protected final IBinaryComparatorFactory[] tokenCmpFactories;
private final int[] filterFieldsForNonBulkLoadOps;
private final int[] invertedIndexFieldsForNonBulkLoadOps;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
public LSMInvertedIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
@@ -103,7 +106,8 @@
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+ int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
componentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
@@ -116,6 +120,8 @@
this.tokenCmpFactories = tokenCmpFactories;
this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
+ this.nullIntrospector = nullIntrospector;
+ this.nullTypeTraits = nullTypeTraits;
int i = 0;
for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
InMemoryInvertedIndex memInvIndex =
@@ -123,7 +129,8 @@
BTree deleteKeysBTree =
BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM,
- ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false);
+ ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false,
+ nullTypeTraits, nullIntrospector);
LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(this, memInvIndex,
deleteKeysBTree, virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
memoryComponents.add(mutableComponent);
@@ -410,7 +417,8 @@
return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
fullTextConfigEvaluatorFactory,
- ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+ ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id), nullTypeTraits,
+ nullIntrospector);
}
@Override
@@ -444,6 +452,16 @@
return tokenCmpFactories;
}
+ @Override
+ public ITypeTraits getNullTypeTraits() {
+ return nullTypeTraits;
+ }
+
+ @Override
+ public INullIntrospector getNullIntrospector() {
+ return nullIntrospector;
+ }
+
public IBinaryTokenizerFactory getTokenizerFactory() {
return tokenizerFactory;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index de1fc49..a801146 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
@@ -56,12 +57,14 @@
ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+ int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, mergePolicy,
opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
- filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer,
+ nullTypeTraits, nullIntrospector);
}
@Override
@@ -70,7 +73,8 @@
return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(virtualBufferCache,
virtualFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
tokenizerFactory, fullTextConfigEvaluatorFactory,
- ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+ ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id), nullTypeTraits,
+ nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index 6998861..7ee6af8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManager;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
@@ -55,12 +56,15 @@
protected final ITypeTraits[] btreeTypeTraits;
protected final IBinaryComparatorFactory[] btreeCmpFactories;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
public InMemoryInvertedIndex(IBufferCache virtualBufferCache, IPageManager virtualFreePageManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
- FileReference btreeFileRef) throws HyracksDataException {
+ FileReference btreeFileRef, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.invListTypeTraits = invListTypeTraits;
@@ -79,8 +83,11 @@
btreeTypeTraits[tokenTypeTraits.length + i] = invListTypeTraits[i];
btreeCmpFactories[tokenTypeTraits.length + i] = invListCmpFactories[i];
}
- this.btree = BTreeUtils.createBTree(virtualBufferCache, virtualFreePageManager, btreeTypeTraits,
- btreeCmpFactories, BTreeLeafFrameType.REGULAR_NSM, btreeFileRef, false);
+ this.btree =
+ BTreeUtils.createBTree(virtualBufferCache, virtualFreePageManager, btreeTypeTraits, btreeCmpFactories,
+ BTreeLeafFrameType.REGULAR_NSM, btreeFileRef, false, nullTypeTraits, nullIntrospector);
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
@@ -219,6 +226,16 @@
}
@Override
+ public ITypeTraits getNullTypeTraits() {
+ return nullTypeTraits;
+ }
+
+ @Override
+ public INullIntrospector getNullIntrospector() {
+ return nullIntrospector;
+ }
+
+ @Override
public int getNumOfFilterFields() {
return 0;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index bd9ce60..e3bce41 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -27,6 +27,7 @@
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManager;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
@@ -49,9 +50,11 @@
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
- FileReference btreeFileRef) throws HyracksDataException {
+ FileReference btreeFileRef, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
super(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
+ tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef, nullTypeTraits,
+ nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
index 0d00ed3..f4086cc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
@@ -20,6 +20,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.fixedsize.FixedSizeElementInvertedListBuilder;
@@ -31,10 +32,15 @@
protected final ITypeTraits[] invListFields;
protected final ITypeTraits[] tokenTypeTraits;
private final boolean isFixedSize;
+ private final ITypeTraits nullTypeTraits;
+ private final INullIntrospector nullIntrospector;
- public InvertedListBuilderFactory(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields) {
+ public InvertedListBuilderFactory(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
this.tokenTypeTraits = tokenTypeTraits;
this.invListFields = invListFields;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
isFixedSize = InvertedIndexUtils.checkTypeTraitsAllFixed(invListFields);
}
@@ -44,7 +50,8 @@
if (isFixedSize) {
return new FixedSizeElementInvertedListBuilder(invListFields);
} else {
- return new VariableSizeElementInvertedListBuilder(tokenTypeTraits, invListFields);
+ return new VariableSizeElementInvertedListBuilder(tokenTypeTraits, invListFields, nullTypeTraits,
+ nullIntrospector);
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 0f26435..48a173e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -41,6 +41,7 @@
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
@@ -103,6 +104,8 @@
protected int rootPageId = 0;
protected IBufferCache bufferCache;
protected int fileId = -1;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
protected final ITypeTraits[] invListTypeTraits;
protected final IBinaryComparatorFactory[] invListCmpFactories;
protected final ITypeTraits[] tokenTypeTraits;
@@ -119,7 +122,8 @@
public OnDiskInvertedIndex(IBufferCache bufferCache, IInvertedListBuilder invListBuilder,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile,
- FileReference invListsFile, IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+ FileReference invListsFile, IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
this.bufferCache = bufferCache;
this.invListBuilder = invListBuilder;
this.invListTypeTraits = invListTypeTraits;
@@ -127,7 +131,8 @@
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.btree = BTreeUtils.createDiskBTree(bufferCache, getBTreeTypeTraits(tokenTypeTraits), tokenCmpFactories,
- BTreeLeafFrameType.REGULAR_NSM, btreeFile, pageManagerFactory.createPageManager(bufferCache), false);
+ BTreeLeafFrameType.REGULAR_NSM, btreeFile, pageManagerFactory.createPageManager(bufferCache), false,
+ nullTypeTraits, nullIntrospector);
this.numTokenFields = btree.getComparatorFactories().length;
this.numInvListKeys = invListCmpFactories.length;
this.invListsFile = invListsFile;
@@ -135,6 +140,8 @@
this.invListEndPageIdField = numTokenFields + 1;
this.invListStartOffField = numTokenFields + 2;
this.invListNumElementsField = numTokenFields + 3;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
@@ -196,10 +203,10 @@
public IInvertedListCursor createInvertedListCursor(IHyracksTaskContext ctx) throws HyracksDataException {
if (InvertedIndexUtils.checkTypeTraitsAllFixed(invListTypeTraits)) {
return new FixedSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, ctx,
- NoOpIndexCursorStats.INSTANCE);
+ NoOpIndexCursorStats.INSTANCE, nullTypeTraits, nullIntrospector);
} else {
return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, ctx,
- NoOpIndexCursorStats.INSTANCE);
+ NoOpIndexCursorStats.INSTANCE, nullTypeTraits, nullIntrospector);
}
}
@@ -209,7 +216,8 @@
if (InvertedIndexUtils.checkTypeTraitsAllFixed(invListTypeTraits)) {
return new FixedSizeElementInvertedListScanCursor(bufferCache, fileId, invListTypeTraits, stats);
} else {
- return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, stats);
+ return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, stats,
+ nullTypeTraits, nullIntrospector);
}
}
@@ -699,6 +707,16 @@
}
@Override
+ public ITypeTraits getNullTypeTraits() {
+ return nullTypeTraits;
+ }
+
+ @Override
+ public INullIntrospector getNullIntrospector() {
+ return nullIntrospector;
+ }
+
+ @Override
public int getNumOfFilterFields() {
return 0;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
index f058e4a..780f36d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.IndexFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
@@ -39,12 +40,14 @@
protected final ITypeTraits[] tokenTypeTraits;
protected final IBinaryComparatorFactory[] tokenCmpFactories;
protected final IInvertedIndexFileNameMapper fileNameMapper;
+ protected final ITypeTraits nullTypeTraits;
+ protected final INullIntrospector nullIntrospector;
public OnDiskInvertedIndexFactory(IIOManager ioManager, IBufferCache bufferCache,
IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper,
- IPageManagerFactory pageManagerFactory) {
+ IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(ioManager, bufferCache, pageManagerFactory);
this.invListBuilderFactory = invListBuilderFactory;
this.invListTypeTraits = invListTypeTraits;
@@ -52,6 +55,8 @@
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.fileNameMapper = fileNameMapper;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
@@ -60,6 +65,7 @@
FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
return new OnDiskInvertedIndex(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory);
+ tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory, nullTypeTraits,
+ nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index fa5c365..ca4d94c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.data.std.primitive.ShortPointable;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
@@ -46,9 +47,10 @@
public PartitionedOnDiskInvertedIndex(IBufferCache bufferCache, IInvertedListBuilder invListBuilder,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile,
- FileReference invListsFile, IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+ FileReference invListsFile, IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
super(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
- btreeFile, invListsFile, pageManagerFactory);
+ btreeFile, invListsFile, pageManagerFactory, nullTypeTraits, nullIntrospector);
}
public class PartitionedOnDiskInvertedIndexAccessor extends OnDiskInvertedIndexAccessor {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
index 7f10658..f9f4012 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
@@ -35,9 +36,9 @@
IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper,
- IPageManagerFactory pageManagerFactory) {
+ IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(ioManager, bufferCache, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, fileNameMapper, pageManagerFactory);
+ tokenCmpFactories, fileNameMapper, pageManagerFactory, nullTypeTraits, nullIntrospector);
}
@Override
@@ -46,6 +47,7 @@
FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
return new PartitionedOnDiskInvertedIndex(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory);
+ tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory, nullTypeTraits,
+ nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
index 11c0901..ce6daa8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.AbstractOnDiskInvertedListCursor;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -49,11 +50,12 @@
private int[] elementIndexes = new int[10];
public FixedSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
- IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
- super(bufferCache, fileId, invListFields, ctx, stats);
+ IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
+ super(bufferCache, fileId, invListFields, ctx, stats, nullTypeTraits, nullIntrospector);
this.bufferEndElementIx = 0;
- this.bufferEndElementTuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields);
+ this.bufferEndElementTuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields, nullTypeTraits);
int tmpSize = 0;
for (int i = 0; i < invListFields.length; i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
index cfbb779..c667487 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
@@ -22,6 +22,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.AbstractInvertedListBuilder;
@@ -35,8 +36,8 @@
// The tokenTypeTraits is necessary because the underlying TypeAwareTupleWriter requires all the type traits of the tuple
// even if the first a few fields in the tuple are never accessed by the writer
- public VariableSizeElementInvertedListBuilder(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields)
- throws HyracksDataException {
+ public VariableSizeElementInvertedListBuilder(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
super(invListFields);
this.allFields = new ITypeTraits[invListFields.length + tokenTypeTraits.length];
@@ -46,7 +47,7 @@
for (int i = 0; i < invListFields.length; i++) {
allFields[i + tokenTypeTraits.length] = invListFields[i];
}
- this.writer = new TypeAwareTupleWriter(allFields);
+ this.writer = new TypeAwareTupleWriter(allFields, nullTypeTraits, nullIntrospector);
InvertedIndexUtils.verifyHasVarSizeTypeTrait(invListFields);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
index ee2bc58..8d863b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
@@ -49,16 +50,18 @@
private ITreeIndexTupleWriter tupleWriter;
public VariableSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId,
- ITypeTraits[] invListFields, IIndexCursorStats stats) throws HyracksDataException {
- super(bufferCache, fileId, invListFields, stats);
+ ITypeTraits[] invListFields, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
+ super(bufferCache, fileId, invListFields, stats, nullTypeTraits, nullIntrospector);
this.isInit = true;
- this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields);
- this.tupleWriter = new TypeAwareTupleWriter(invListFields);
+ this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields, nullTypeTraits);
+ this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
}
public VariableSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId,
- ITypeTraits[] invListFields, IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
- super(bufferCache, fileId, invListFields, ctx, stats);
+ ITypeTraits[] invListFields, IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
+ super(bufferCache, fileId, invListFields, ctx, stats, nullTypeTraits, nullIntrospector);
isInit = true;
}
@@ -69,8 +72,8 @@
// Note that the cursors can be re-used in the upper-layer callers so we need to reset the state variables when open()
currentElementIxForScan = 0;
isInit = true;
- this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields);
- this.tupleWriter = new TypeAwareTupleWriter(invListFields);
+ this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields, nullTypeTraits);
+ this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
}
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
index ab71c0a..f71f347 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
@@ -54,12 +55,12 @@
private IInvertedListTupleReference tupleReference;
private ITreeIndexTupleWriter tupleWriter;
- public VariableSizeInvertedListSearchResultFrameTupleAccessor(int frameSize, ITypeTraits[] fields)
- throws HyracksDataException {
+ public VariableSizeInvertedListSearchResultFrameTupleAccessor(int frameSize, ITypeTraits[] fields,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
super(frameSize, fields);
- this.tupleWriter = new TypeAwareTupleWriter(fields);
- this.tupleReference = new VariableSizeInvertedListTupleReference(fields);
+ this.tupleWriter = new TypeAwareTupleWriter(fields, nullTypeTraits, nullIntrospector);
+ this.tupleReference = new VariableSizeInvertedListTupleReference(fields, nullTypeTraits);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
index 8e1db66..997c93a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
@@ -28,17 +28,18 @@
public class VariableSizeInvertedListTupleReference extends AbstractInvertedListTupleReference {
- private ITreeIndexTupleReference tupleReference;
+ private final ITreeIndexTupleReference tupleReference;
@Override
protected void verifyTypeTrait() throws HyracksDataException {
InvertedIndexUtils.verifyHasVarSizeTypeTrait(typeTraits);
}
- public VariableSizeInvertedListTupleReference(ITypeTraits[] typeTraits) throws HyracksDataException {
+ public VariableSizeInvertedListTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits)
+ throws HyracksDataException {
super(typeTraits);
- this.tupleReference = new TypeAwareTupleReference(typeTraits);
+ this.tupleReference = new TypeAwareTupleReference(typeTraits, nullTypeTraits);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 2fb620c..aa6f7c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -103,8 +103,8 @@
if (bufferManager == null) {
throw HyracksDataException.create(ErrorCode.CANNOT_CONTINUE_TEXT_SEARCH_BUFFER_MANAGER_IS_NULL);
}
- this.finalSearchResult =
- new InvertedIndexFinalSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
+ this.finalSearchResult = new InvertedIndexFinalSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+ invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
this.invListMerger = new InvertedListMerger(ctx, invIndex, bufferManager);
this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
this.invListCursorFactory = new InvertedListCursorFactory(invIndex, ctx);
@@ -118,9 +118,10 @@
this.queryTokenAppender = new FrameTupleAppenderAccessor(QUERY_TOKEN_REC_DESC);
this.queryTokenAppender.reset(queryTokenFrame, true);
this.isSingleInvertedList = false;
- this.searchResultTuple = InvertedIndexUtils.createInvertedListTupleReference(invIndex.getInvListTypeTraits());
+ this.searchResultTuple = InvertedIndexUtils.createInvertedListTupleReference(invIndex.getInvListTypeTraits(),
+ invIndex.getNullTypeTraits());
this.searchResultFta = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(),
- invIndex.getInvListTypeTraits());
+ invIndex.getInvListTypeTraits(), invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
}
protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
index e5f4622..f07983f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
/**
* This is an in-memory based storage for final results of inverted-index searches.
@@ -34,8 +35,9 @@
public class InvertedIndexFinalSearchResult extends InvertedIndexSearchResult {
public InvertedIndexFinalSearchResult(ITypeTraits[] invListFields, IHyracksTaskContext ctx,
- ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
- super(invListFields, ctx, bufferManager);
+ ISimpleFrameBufferManager bufferManager, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
+ super(invListFields, ctx, bufferManager, nullTypeTraits, nullIntrospector);
}
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
index 707d85d..35d0dd8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
@@ -37,6 +37,7 @@
import org.apache.hyracks.dataflow.common.io.RunFileWriter;
import org.apache.hyracks.dataflow.std.buffermanager.BufferManagerBackedVSizeFrame;
import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListSearchResultFrameTupleAppender;
@@ -84,14 +85,16 @@
protected byte[] tempBytes;
public InvertedIndexSearchResult(ITypeTraits[] invListFields, IHyracksTaskContext ctx,
- ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
+ ISimpleFrameBufferManager bufferManager, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
this.invListFields = invListFields;
- this.tupleWriter = new TypeAwareTupleWriter(invListFields);
+ this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
initTypeTraits(invListFields);
this.ctx = ctx;
appender = new InvertedListSearchResultFrameTupleAppender(ctx.getInitialFrameSize());
- accessor = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits);
- tuple = InvertedIndexUtils.createInvertedListTupleReference(typeTraits);
+ accessor = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits,
+ nullTypeTraits, nullIntrospector);
+ tuple = InvertedIndexUtils.createInvertedListTupleReference(typeTraits, nullTypeTraits);
this.bufferManager = bufferManager;
this.isInReadMode = false;
this.isWriteFinished = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
index deea12a..8bf16fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
@@ -92,8 +92,10 @@
public InvertedListMerger(IHyracksTaskContext ctx, IInvertedIndex invIndex, ISimpleFrameBufferManager bufferManager)
throws HyracksDataException {
this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
- this.prevSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
- this.newSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
+ this.prevSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+ invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
+ this.newSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+ invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
}
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index dcde832..aefb461 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManager;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -84,41 +85,45 @@
IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
- throws HyracksDataException {
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef,
+ nullTypeTraits, nullIntrospector);
}
public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
- throws HyracksDataException {
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
return new PartitionedInMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- fullTextConfigEvaluatorFactory, btreeFileRef);
+ fullTextConfigEvaluatorFactory, btreeFileRef, nullTypeTraits, nullIntrospector);
}
public static OnDiskInvertedIndex createOnDiskInvertedIndex(IIOManager ioManager, IBufferCache bufferCache,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile,
- IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+ IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
FileReference btreeFile = getBTreeFile(ioManager, invListsFile);
return new OnDiskInvertedIndex(bufferCache, builder, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory);
+ tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory, nullTypeTraits, nullIntrospector);
}
public static PartitionedOnDiskInvertedIndex createPartitionedOnDiskInvertedIndex(IIOManager ioManager,
IBufferCache bufferCache, ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile,
- IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+ IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
FileReference btreeFile = getBTreeFile(ioManager, invListsFile);
return new PartitionedOnDiskInvertedIndex(bufferCache, builder, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory);
+ tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory, nullTypeTraits,
+ nullIntrospector);
}
public static FileReference getBTreeFile(IIOManager ioManager, FileReference invListsFile)
@@ -128,9 +133,10 @@
public static BTreeFactory createDeletedKeysBTreeFactory(IIOManager ioManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, IBufferCache diskBufferCache,
- IPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+ IPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+ throws HyracksDataException {
BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
- new BTreeTypeAwareTupleWriterFactory(invListTypeTraits, false);
+ new BTreeTypeAwareTupleWriterFactory(invListTypeTraits, false, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory leafFrameFactory =
BTreeUtils.getLeafFrameFactory(tupleWriterFactory, BTreeLeafFrameType.REGULAR_NSM);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
@@ -148,11 +154,11 @@
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
- boolean durable, IMetadataPageManagerFactory pageManagerFactory, ITracer tracer)
- throws HyracksDataException {
+ boolean durable, IMetadataPageManagerFactory pageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
- invListCmpFactories, diskBufferCache, pageManagerFactory);
+ invListCmpFactories, diskBufferCache, pageManagerFactory, nullTypeTraits, nullIntrospector);
int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
for (int i = 0; i < invListCmpFactories.length; i++) {
@@ -165,16 +171,17 @@
new LSMInvertedIndexFileManager(ioManager, onDiskDirFileRef, deletedKeysBTreeFactory);
IInvertedListBuilderFactory invListBuilderFactory =
- new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits);
- OnDiskInvertedIndexFactory invIndexFactory =
- new OnDiskInvertedIndexFactory(ioManager, diskBufferCache, invListBuilderFactory, invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, fileManager, pageManagerFactory);
+ new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits, nullTypeTraits, nullIntrospector);
+ OnDiskInvertedIndexFactory invIndexFactory = new OnDiskInvertedIndexFactory(ioManager, diskBufferCache,
+ invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+ fileManager, pageManagerFactory, nullTypeTraits, nullIntrospector);
ComponentFilterHelper filterHelper = null;
LSMComponentFilterFrameFactory filterFrameFactory = null;
LSMComponentFilterManager filterManager = null;
if (filterCmpFactories != null) {
- TypeAwareTupleWriterFactory filterTupleWriterFactory = new TypeAwareTupleWriterFactory(filterTypeTraits);
+ TypeAwareTupleWriterFactory filterTupleWriterFactory =
+ new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -187,7 +194,7 @@
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
}
public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -200,10 +207,11 @@
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
- boolean durable, IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
+ boolean durable, IPageManagerFactory pageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
- invListCmpFactories, diskBufferCache, pageManagerFactory);
+ invListCmpFactories, diskBufferCache, pageManagerFactory, nullTypeTraits, nullIntrospector);
int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
for (int i = 0; i < invListCmpFactories.length; i++) {
@@ -216,16 +224,17 @@
new LSMInvertedIndexFileManager(ioManager, onDiskDirFileRef, deletedKeysBTreeFactory);
IInvertedListBuilderFactory invListBuilderFactory =
- new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits);
+ new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits, nullTypeTraits, nullIntrospector);
PartitionedOnDiskInvertedIndexFactory invIndexFactory = new PartitionedOnDiskInvertedIndexFactory(ioManager,
diskBufferCache, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, fileManager, pageManagerFactory);
+ tokenCmpFactories, fileManager, pageManagerFactory, nullTypeTraits, nullIntrospector);
ComponentFilterHelper filterHelper = null;
LSMComponentFilterFrameFactory filterFrameFactory = null;
LSMComponentFilterManager filterManager = null;
if (filterCmpFactories != null) {
- TypeAwareTupleWriterFactory filterTupleWriterFactory = new TypeAwareTupleWriterFactory(filterTypeTraits);
+ TypeAwareTupleWriterFactory filterTupleWriterFactory =
+ new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -238,7 +247,7 @@
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
}
public static boolean checkTypeTraitsAllFixed(ITypeTraits[] typeTraits) {
@@ -264,21 +273,22 @@
}
}
- public static IInvertedListTupleReference createInvertedListTupleReference(ITypeTraits[] typeTraits)
- throws HyracksDataException {
+ public static IInvertedListTupleReference createInvertedListTupleReference(ITypeTraits[] typeTraits,
+ ITypeTraits nullTypeTraits) throws HyracksDataException {
if (checkTypeTraitsAllFixed(typeTraits)) {
return new FixedSizeInvertedListTupleReference(typeTraits);
} else {
- return new VariableSizeInvertedListTupleReference(typeTraits);
+ return new VariableSizeInvertedListTupleReference(typeTraits, nullTypeTraits);
}
}
- public static IFrameTupleAccessor createInvertedListFrameTupleAccessor(int frameSize, ITypeTraits[] typeTraits)
- throws HyracksDataException {
+ public static IFrameTupleAccessor createInvertedListFrameTupleAccessor(int frameSize, ITypeTraits[] typeTraits,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
if (checkTypeTraitsAllFixed(typeTraits)) {
return new FixedSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits);
} else {
- return new VariableSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits);
+ return new VariableSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits, nullTypeTraits,
+ nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
index 9404d91..e6d50d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -61,12 +62,13 @@
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
- boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+ boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
- bloomFilterFalsePositiveRate);
+ bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
}
private ExternalRTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json,
@@ -90,7 +92,7 @@
opTrackerProvider.getOperationTracker(ncServiceCtx, this),
ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
linearizeCmpFactory, buddyBTreeFields, durable, isPointMBR, metadataPageManagerFactory,
- ncServiceCtx.getTracer());
+ ncServiceCtx.getTracer(), nullTypeTraits, nullIntrospector);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
index 1f4971d..ca8caec 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -49,12 +50,13 @@
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
- boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+ boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
- bloomFilterFalsePositiveRate);
+ bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
}
@Override
@@ -63,7 +65,8 @@
filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
- linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
+ linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate,
+ nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
index bfae1cd..dd21f7c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -73,10 +74,12 @@
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
- boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+ boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.btreeCmpFactories = btreeCmpFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -117,7 +120,7 @@
opTrackerProvider.getOperationTracker(ncServiceCtx, this),
ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
linearizeCmpFactory, rtreeFields, buddyBTreeFields, filterTypeTraits, filterCmpFactories, filterFields,
- durable, isPointMBR, metadataPageManagerFactory);
+ durable, isPointMBR, metadataPageManagerFactory, nullTypeTraits, nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
index 635f1ac..873a023 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -60,10 +61,12 @@
Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
- boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+ boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.btreeCmpFactories = btreeCmpFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -81,6 +84,6 @@
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories,
rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
- bloomFilterFalsePositiveRate);
+ bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
index e4a8287..33b9da6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.api.io.IJsonSerializable;
import org.apache.hyracks.api.io.IPersistedResourceRegistry;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -70,10 +71,12 @@
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, IBinaryComparatorFactory[] btreeCmpFactories,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable) {
+ ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.btreeCmpFactories = btreeCmpFactories;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
@@ -108,7 +111,8 @@
mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
ioOpCallbackFactory, pageWriteCallbackFactory, linearizeCmpFactory, rtreeFields, filterTypeTraits,
- filterCmpFactories, filterFields, durable, isPointMBR, metadataPageManagerFactory);
+ filterCmpFactories, filterFields, durable, isPointMBR, metadataPageManagerFactory, nullTypeTraits,
+ nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
index 6edf877..e9b4c90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -57,10 +58,12 @@
Map<String, String> mergePolicyProperties, boolean durable,
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR,
- IBinaryComparatorFactory[] btreeComparatorFactories) {
+ IBinaryComparatorFactory[] btreeComparatorFactories, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
super(storageManager, rteeTypeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
- vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+ vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+ nullIntrospector);
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
this.linearizeCmpFactory = linearizeCmpFactory;
@@ -75,6 +78,7 @@
cmpFactories, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider,
ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider,
ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, btreeComparatorFactories,
- valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR, durable);
+ valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR, durable,
+ nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index b7eb115..11385de 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -28,6 +28,7 @@
import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
@@ -50,7 +51,7 @@
protected boolean open;
protected RTreeSearchCursor[] rtreeCursors;
- protected BTreeRangeSearchCursor[] btreeCursors;
+ protected ITreeIndexCursor[] btreeCursors;
protected RTreeAccessor[] rtreeAccessors;
protected BTreeAccessor[] btreeAccessors;
protected BloomFilter[] bloomFilters;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index 8e5cb35..729ca74 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -142,7 +142,8 @@
bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBuddyIndex().getFileId(), linearizerArray,
btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(),
flushingComponent.getBuddyIndex().getBufferCache(), comparatorFields);
- BTreeRangeSearchCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
+ BTreeRangeSearchCursor btreeScanCursor =
+ (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(false);
try {
isEmpty = true;
memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
index 3b7c150..4f3b49b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
@@ -21,11 +21,13 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
public class LSMRTreeCopyTupleWriter extends LSMRTreeTupleWriter {
- public LSMRTreeCopyTupleWriter(ITypeTraits[] typeTraits) {
+ public LSMRTreeCopyTupleWriter(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
// Third parameter is never used locally, just give false.
- super(typeTraits, false);
+ super(typeTraits, false, nullTypeTraits, nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
index 6585ffe..41fa508 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
@@ -20,18 +20,20 @@
package org.apache.hyracks.storage.am.lsm.rtree.tuples;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
public class LSMRTreeCopyTupleWriterFactory extends RTreeTypeAwareTupleWriterFactory {
private static final long serialVersionUID = 1L;
- public LSMRTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits) {
- super(typeTraits);
+ public LSMRTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
}
@Override
public RTreeTypeAwareTupleWriter createTupleWriter() {
- return new LSMRTreeCopyTupleWriter(typeTraits);
+ return new LSMRTreeCopyTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
index 86a6a39..3e25e8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
@@ -26,8 +26,8 @@
public class LSMRTreeTupleReference extends RTreeTypeAwareTupleReference implements ILSMTreeTupleReference {
- public LSMRTreeTupleReference(ITypeTraits[] typeTraits) {
- super(typeTraits);
+ public LSMRTreeTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
+ super(typeTraits, nullTypeTraits);
}
@Override
@@ -45,4 +45,10 @@
public int getTupleStart() {
return tupleStartOff;
}
+
+ @Override
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ // 1 for antimatter
+ return fieldIdx + 1;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
index e28d5a2..878606f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleReference;
public class LSMRTreeTupleReferenceForPointMBR extends RTreeTypeAwareTupleReference implements ILSMTreeTupleReference {
+
private final int inputKeyFieldCount; //double field count for mbr secondary key of an input tuple
private final int inputTotalFieldCount; //total field count (key + value fields) of an input tuple.
private final int storedKeyFieldCount; //double field count to be stored for the mbr secondary key
@@ -33,8 +34,8 @@
private final boolean antimatterAware;
public LSMRTreeTupleReferenceForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
- boolean antimatterAware) {
- super(typeTraits);
+ boolean antimatterAware, ITypeTraits nullTypeTraits) {
+ super(typeTraits, nullTypeTraits);
this.inputKeyFieldCount = keyFieldCount;
this.inputTotalFieldCount = keyFieldCount + valueFieldCount;
this.storedKeyFieldCount = keyFieldCount / 2;
@@ -52,10 +53,21 @@
// decode field slots in three steps
int field = 0;
int cumul = 0;
+ int nullFlagsRunner = 0;
//step1. decode field slots for stored key
- for (int i = 0; i < storedKeyFieldCount; i++) {
- //key or value fields
- cumul += typeTraits[i].getFixedLength();
+ encDec.reset(buf, tupleStartOff + nullFlagsBytes);
+ for (int i = 0; i < storedKeyFieldCount; i++, nullFlagsRunner++) {
+ if (!typeTraits[i].isFixedLength()) {
+ // for variable length fields, read the encoded length whether it's a value or null
+ cumul += encDec.decode();
+ } else {
+ // for fixed length fields, if the field is null, get the null length
+ if (nullTypeTraits != null && isNull(buf, tupleStartOff, nullFlagsRunner)) {
+ cumul += nullTypeTraits.getFixedLength();
+ } else {
+ cumul += typeTraits[i].getFixedLength();
+ }
+ }
decodedFieldSlots[field++] = cumul;
}
//step2. decode field slots for non-stored (duplicated point) key
@@ -64,17 +76,19 @@
decodedFieldSlots[field++] = decodedFieldSlots[i];
}
//step3. decode field slots for value field
- encDec.reset(buf, tupleStartOff + nullFlagsBytes);
- for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++) {
+ for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++, nullFlagsRunner++) {
if (!typeTraits[i].isFixedLength()) {
- //value fields
+ // for variable length fields, read the encoded length whether it's a value or null
cumul += encDec.decode();
- decodedFieldSlots[field++] = cumul;
} else {
- //key or value fields
- cumul += typeTraits[i].getFixedLength();
- decodedFieldSlots[field++] = cumul;
+ // for fixed length fields, if the field is null, get the null length
+ if (nullTypeTraits != null && isNull(buf, tupleStartOff, nullFlagsRunner)) {
+ cumul += nullTypeTraits.getFixedLength();
+ } else {
+ cumul += typeTraits[i].getFixedLength();
+ }
}
+ decodedFieldSlots[field++] = cumul;
}
dataStartOff = encDec.getPos();
@@ -102,19 +116,21 @@
@Override
public int getFieldLength(int fIdx) {
- if (getInternalFieldIdx(fIdx) == 0) {
+ int internalFieldIdx = getInternalFieldIdx(fIdx);
+ if (internalFieldIdx == 0) {
return decodedFieldSlots[0];
} else {
- return decodedFieldSlots[getInternalFieldIdx(fIdx)] - decodedFieldSlots[getInternalFieldIdx(fIdx) - 1];
+ return decodedFieldSlots[internalFieldIdx] - decodedFieldSlots[internalFieldIdx - 1];
}
}
@Override
public int getFieldStart(int fIdx) {
- if (getInternalFieldIdx(fIdx) == 0) {
+ int internalFieldIdx = getInternalFieldIdx(fIdx);
+ if (internalFieldIdx == 0) {
return dataStartOff;
} else {
- return dataStartOff + decodedFieldSlots[getInternalFieldIdx(fIdx) - 1];
+ return dataStartOff + decodedFieldSlots[internalFieldIdx - 1];
}
}
@@ -140,7 +156,14 @@
@Override
public boolean isAntimatter() {
+ // TODO(ali): antimatterAware should be checked?
// Check antimatter bit.
return BitOperationUtils.getBit(buf, tupleStartOff, ANTIMATTER_BIT_OFFSET);
}
+
+ @Override
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ // 1 for antimatter
+ return antimatterAware ? fieldIdx + 1 : fieldIdx;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
index 5650cbb..6417839 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
@@ -30,14 +31,15 @@
public class LSMRTreeTupleWriter extends RTreeTypeAwareTupleWriter implements ILSMTreeTupleWriter {
private boolean isAntimatter;
- public LSMRTreeTupleWriter(ITypeTraits[] typeTraits, boolean isAntimatter) {
- super(typeTraits);
+ public LSMRTreeTupleWriter(ITypeTraits[] typeTraits, boolean isAntimatter, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
this.isAntimatter = isAntimatter;
}
@Override
public LSMRTreeTupleReference createTupleReference() {
- return new LSMRTreeTupleReference(typeTraits);
+ return new LSMRTreeTupleReference(typeTraits, nullTypeTraits);
}
@Override
@@ -72,4 +74,9 @@
this.isAntimatter = isAntimatter;
}
+ @Override
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ // 1 for antimatter
+ return fieldIdx + 1;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
index 4ad5e3a..040c805 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
@@ -20,6 +20,7 @@
package org.apache.hyracks.storage.am.lsm.rtree.tuples;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
public class LSMRTreeTupleWriterFactory extends RTreeTypeAwareTupleWriterFactory {
@@ -27,14 +28,15 @@
private static final long serialVersionUID = 1L;
private final boolean isAntimatter;
- public LSMRTreeTupleWriterFactory(ITypeTraits[] typeTraits, boolean isAntimatter) {
- super(typeTraits);
+ public LSMRTreeTupleWriterFactory(ITypeTraits[] typeTraits, boolean isAntimatter, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
this.isAntimatter = isAntimatter;
}
@Override
public LSMRTreeTupleWriter createTupleWriter() {
- return new LSMRTreeTupleWriter(typeTraits, isAntimatter);
+ return new LSMRTreeTupleWriter(typeTraits, isAntimatter, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
index 6a87af2..625883f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
@@ -20,6 +20,7 @@
package org.apache.hyracks.storage.am.lsm.rtree.tuples;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
@@ -32,8 +33,8 @@
private final boolean isAntimatter;
public LSMRTreeTupleWriterFactoryForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
- boolean antimatterAware, boolean isDelete) {
- super(typeTraits);
+ boolean antimatterAware, boolean isDelete, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
this.keyFieldCount = keyFieldCount;
this.valueFieldCount = valueFieldCount;
this.antimatterAware = antimatterAware;
@@ -43,7 +44,7 @@
@Override
public RTreeTypeAwareTupleWriter createTupleWriter() {
return new LSMRTreeTupleWriterForPointMBR(typeTraits, keyFieldCount, valueFieldCount, antimatterAware,
- isAntimatter);
+ isAntimatter, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
index ce6bdeb..ff66a85 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
@@ -44,6 +45,7 @@
*/
public class LSMRTreeTupleWriterForPointMBR extends RTreeTypeAwareTupleWriter implements ILSMTreeTupleWriter {
+
private final int inputKeyFieldCount; //double field count for mbr secondary key of an input tuple
private final int valueFieldCount; //value(or payload or primary key) field count (same for an input tuple and a stored tuple)
private final int inputTotalFieldCount; //total field count (key + value fields) of an input tuple.
@@ -53,8 +55,9 @@
private boolean isAntimatter;
public LSMRTreeTupleWriterForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
- boolean antimatterAware, boolean isAntimatter) {
- super(typeTraits);
+ boolean antimatterAware, boolean isAntimatter, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
this.inputKeyFieldCount = keyFieldCount;
this.valueFieldCount = valueFieldCount;
this.inputTotalFieldCount = keyFieldCount + valueFieldCount;
@@ -80,14 +83,15 @@
@Override
public LSMRTreeTupleReferenceForPointMBR createTupleReference() {
- return new LSMRTreeTupleReferenceForPointMBR(typeTraits, inputKeyFieldCount, valueFieldCount, antimatterAware);
+ return new LSMRTreeTupleReferenceForPointMBR(typeTraits, inputKeyFieldCount, valueFieldCount, antimatterAware,
+ nullTypeTraits);
}
@Override
public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
int runner = targetOff;
int nullFlagsBytes = getNullFlagsBytes(tuple);
- // write null indicator bits
+ // reset null indicator bits
for (int i = 0; i < nullFlagsBytes; i++) {
targetBuf[runner++] = (byte) 0;
}
@@ -99,15 +103,28 @@
}
}
+ int nullRunner = 0;
// write key fields
- for (int i = 0; i < storedKeyFieldCount; i++) {
- System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
- runner += tuple.getFieldLength(i);
+ for (int i = 0; i < storedKeyFieldCount; i++, nullRunner++) {
+ byte[] fieldData = tuple.getFieldData(i);
+ int fieldOffset = tuple.getFieldStart(i);
+ int fieldLength = tuple.getFieldLength(i);
+ if (nullIntrospector != null && nullIntrospector.isNull(fieldData, fieldOffset, fieldLength)) {
+ setNullFlag(targetBuf, targetOff, nullRunner);
+ }
+ System.arraycopy(fieldData, fieldOffset, targetBuf, runner, fieldLength);
+ runner += fieldLength;
}
// write value fields
- for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++) {
- System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
- runner += tuple.getFieldLength(i);
+ for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++, nullRunner++) {
+ byte[] fieldData = tuple.getFieldData(i);
+ int fieldOffset = tuple.getFieldStart(i);
+ int fieldLength = tuple.getFieldLength(i);
+ if (nullIntrospector != null && nullIntrospector.isNull(fieldData, fieldOffset, fieldLength)) {
+ setNullFlag(targetBuf, targetOff, nullRunner);
+ }
+ System.arraycopy(fieldData, fieldOffset, targetBuf, runner, fieldLength);
+ runner += fieldLength;
}
//set antimatter bit if necessary
@@ -137,7 +154,7 @@
@Override
protected int getNullFlagsBytes(ITupleReference tuple) {
- return BitOperationUtils.getFlagBytes(storedTotalFieldCount + (antimatterAware ? 1 : 0));
+ return BitOperationUtils.getFlagBytes(storedTotalFieldCount + valueFieldCount + (antimatterAware ? 1 : 0));
}
@Override
@@ -157,6 +174,7 @@
}
protected void setAntimatterBit(byte[] targetBuf, int targetOff) {
+ // TODO(ali): antimatterAware should be checked?
// Set antimatter bit to 1.
BitOperationUtils.setBit(targetBuf, targetOff, ANTIMATTER_BIT_OFFSET);
}
@@ -165,4 +183,9 @@
public void setAntimatter(boolean isAntimatter) {
this.isAntimatter = isAntimatter;
}
+
+ @Override
+ protected int getAdjustedFieldIdx(int fieldIdx) {
+ return antimatterAware ? fieldIdx + 1 : fieldIdx;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index ba6015d..a436c43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.storage.am.btree.impls.BTree;
import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
@@ -85,7 +86,8 @@
ILSMPageWriteCallbackFactory pageWriteCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
int[] rtreeFields, int[] buddyBTreeFields, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
- IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+ IMetadataPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
int valueFieldCount = buddyBTreeFields.length;
int keyFieldCount = typeTraits.length - valueFieldCount;
ITypeTraits[] btreeTypeTraits = new ITypeTraits[valueFieldCount];
@@ -93,16 +95,16 @@
btreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
}
RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
- new RTreeTypeAwareTupleWriterFactory(typeTraits);
+ new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory = null;
if (isPointMBR) {
- rtreeLeafFrameTupleWriterFactory =
- new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, false, false);
+ rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+ valueFieldCount, false, false, nullTypeTraits, nullIntrospector);
} else {
rtreeLeafFrameTupleWriterFactory = rtreeInteriorFrameTupleWriterFactory;
}
BTreeTypeAwareTupleWriterFactory btreeTupleWriterFactory =
- new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false);
+ new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, nullTypeTraits, nullIntrospector);
RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
RTreeFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeLeafFrameTupleWriterFactory,
@@ -127,7 +129,8 @@
LSMComponentFilterFrameFactory filterFrameFactory = null;
LSMComponentFilterManager filterManager = null;
if (filterCmpFactories != null) {
- TypeAwareTupleWriterFactory filterTupleWriterFactory = new TypeAwareTupleWriterFactory(filterTypeTraits);
+ TypeAwareTupleWriterFactory filterTupleWriterFactory =
+ new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -154,28 +157,33 @@
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
ILinearizeComparatorFactory linearizerCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
- IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+ IMetadataPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) throws HyracksDataException {
RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
- new LSMRTreeTupleWriterFactory(typeTraits, false);
+ new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory;
RTreeTypeAwareTupleWriterFactory rtreeLeafFrameCopyTupleWriterFactory;
RTreeTypeAwareTupleWriterFactory rtreeLeafFrameBulkLoadWriterFactory;
if (isPointMBR) {
int keyFieldCount = rtreeCmpFactories.length;
int valueFieldCount = btreeComparatorFactories.length - keyFieldCount;
- rtreeLeafFrameTupleWriterFactory =
- new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
- rtreeLeafFrameCopyTupleWriterFactory =
- new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
- rtreeLeafFrameBulkLoadWriterFactory =
- new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
+ rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+ valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
+ rtreeLeafFrameCopyTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+ valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
+ rtreeLeafFrameBulkLoadWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+ valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
} else {
- rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
- rtreeLeafFrameCopyTupleWriterFactory = new LSMRTreeCopyTupleWriterFactory(typeTraits);
- rtreeLeafFrameBulkLoadWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
+ rtreeLeafFrameTupleWriterFactory =
+ new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
+ rtreeLeafFrameCopyTupleWriterFactory =
+ new LSMRTreeCopyTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
+ rtreeLeafFrameBulkLoadWriterFactory =
+ new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
}
- LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
+ LSMRTreeTupleWriterFactory btreeTupleWriterFactory =
+ new LSMRTreeTupleWriterFactory(typeTraits, true, nullTypeTraits, nullIntrospector);
RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
@@ -217,7 +225,8 @@
LSMComponentFilterFrameFactory filterFrameFactory = null;
LSMComponentFilterManager filterManager = null;
if (filterCmpFactories != null) {
- TypeAwareTupleWriterFactory filterTupleWriterFactory = new TypeAwareTupleWriterFactory(filterTypeTraits);
+ TypeAwareTupleWriterFactory filterTupleWriterFactory =
+ new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -245,8 +254,8 @@
ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
ILinearizeComparatorFactory linearizeCmpFactory, int[] buddyBTreeFields, boolean durable,
- boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer)
- throws HyracksDataException {
+ boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
int keyFieldCount = rtreeCmpFactories.length;
int valueFieldCount = typeTraits.length - keyFieldCount;
@@ -255,16 +264,16 @@
btreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
}
RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
- new RTreeTypeAwareTupleWriterFactory(typeTraits);
+ new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory = null;
if (isPointMBR) {
- rtreeLeafFrameTupleWriterFactory =
- new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, false, false);
+ rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+ valueFieldCount, false, false, nullTypeTraits, nullIntrospector);
} else {
rtreeLeafFrameTupleWriterFactory = rtreeInteriorFrameTupleWriterFactory;
}
BTreeTypeAwareTupleWriterFactory btreeTupleWriterFactory =
- new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false);
+ new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, nullTypeTraits, nullIntrospector);
RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
RTreeFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeLeafFrameTupleWriterFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
index f9eb844..9195982 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -42,10 +43,13 @@
private final IPageManagerFactory pageManagerFactory;
private final IPrimitiveValueProviderFactory[] valueProviderFactories;
private final RTreePolicyType rtreePolicyType;
+ private final ITypeTraits nullTypeTraits;
+ private final INullIntrospector nullIntrospector;
public RTreeResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType) {
+ IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
this.path = path;
this.storageManager = storageManager;
this.typeTraits = typeTraits;
@@ -53,6 +57,8 @@
this.pageManagerFactory = pageManagerFactory;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
@@ -60,7 +66,8 @@
IIOManager ioManager = ctx.getIoManager();
FileReference resourceRef = ioManager.resolve(path);
return RTreeUtils.createRTree(storageManager.getBufferCache(ctx), typeTraits, valueProviderFactories,
- comparatorFactories, rtreePolicyType, resourceRef, false, pageManagerFactory);
+ comparatorFactories, rtreePolicyType, resourceRef, false, pageManagerFactory, nullTypeTraits,
+ nullIntrospector);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
index 1a3bb0b..43ccc0d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -30,29 +31,34 @@
public class RTreeResourceFactory implements IResourceFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 2L;
private final IStorageManager storageManager;
private final ITypeTraits[] typeTraits;
private final IBinaryComparatorFactory[] comparatorFactories;
private final IPageManagerFactory pageManagerFactory;
private final IPrimitiveValueProviderFactory[] valueProviderFactories;
private final RTreePolicyType rtreePolicyType;
+ private final ITypeTraits nullTypeTraits;
+ private final INullIntrospector nullIntrospector;
public RTreeResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
- IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType) {
+ IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+ ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
this.storageManager = storageManager;
this.typeTraits = typeTraits;
this.comparatorFactories = comparatorFactories;
this.pageManagerFactory = pageManagerFactory;
this.valueProviderFactories = valueProviderFactories;
this.rtreePolicyType = rtreePolicyType;
+ this.nullTypeTraits = nullTypeTraits;
+ this.nullIntrospector = nullIntrospector;
}
@Override
public IResource createResource(FileReference fileRef) {
return new RTreeResource(fileRef.getRelativePath(), storageManager, typeTraits, comparatorFactories,
- pageManagerFactory, valueProviderFactories, rtreePolicyType);
+ pageManagerFactory, valueProviderFactories, rtreePolicyType, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 46727cd..cc6d76d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -39,6 +39,7 @@
protected final int[] minFilterFieldIndexes;
protected final int[] maxFilterFieldIndexes;
protected final boolean appendIndexFilter;
+ protected final IMissingWriterFactory nonFilterWriterFactory;
protected final IIndexDataflowHelperFactory indexHelperFactory;
protected final boolean retainInput;
protected final boolean retainMissing;
@@ -52,18 +53,19 @@
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
- int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+ int[] maxFilterFieldIndexes, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) {
this(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, false, null, null);
}
public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc, int[] keyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
- int[] maxFilterFieldIndexes, boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
+ int[] maxFilterFieldIndexes, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -76,6 +78,7 @@
this.minFilterFieldIndexes = minFilterFieldIndexes;
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.appendIndexFilter = appendIndexFilter;
+ this.nonFilterWriterFactory = nonFilterWriterFactory;
this.outRecDescs[0] = outRecDesc;
this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
@@ -88,7 +91,7 @@
return new RTreeSearchOperatorNodePushable(ctx, partition,
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), keyFields, minFilterFieldIndexes,
maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
- searchCallbackFactory, appendIndexFilter, appendOpCallbackProceedResult,
+ searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, appendOpCallbackProceedResult,
searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 4d460b1..a8a4252 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -42,23 +42,23 @@
int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- boolean appendIndexFilter) throws HyracksDataException {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
this(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
- null);
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, false, null, null);
}
public RTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
- boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
- byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
- throws HyracksDataException {
+ boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory,
+ boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+ byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
// TODO: predicate & limit pushdown not enabled for RTree yet
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
- appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ nonFilterWriterFactory, null, -1, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
searchCallbackProceedResultTrueValue);
if (keyFields != null && keyFields.length > 0) {
searchKey = new PermutingFrameTupleReference();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java
index 14c5eac..4398297 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/DoublePrimitiveValueProviderFactory.java
@@ -37,12 +37,7 @@
@Override
public IPrimitiveValueProvider createPrimitiveValueProvider() {
- return new IPrimitiveValueProvider() {
- @Override
- public double getValue(byte[] bytes, int offset) {
- return DoublePointable.getDouble(bytes, offset);
- }
- };
+ return DoublePointable::getDouble;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java
index f3c4b62..1793c76 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/FloatPrimitiveValueProviderFactory.java
@@ -37,12 +37,7 @@
@Override
public IPrimitiveValueProvider createPrimitiveValueProvider() {
- return new IPrimitiveValueProvider() {
- @Override
- public double getValue(byte[] bytes, int offset) {
- return FloatPointable.getFloat(bytes, offset);
- }
- };
+ return FloatPointable::getFloat;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java
index 3e26d82..cdec59f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/IntegerPrimitiveValueProviderFactory.java
@@ -37,12 +37,7 @@
@Override
public IPrimitiveValueProvider createPrimitiveValueProvider() {
- return new IPrimitiveValueProvider() {
- @Override
- public double getValue(byte[] bytes, int offset) {
- return IntegerPointable.getInteger(bytes, offset);
- }
- };
+ return IntegerPointable::getInteger;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 7e8f249..d85200f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -41,6 +41,7 @@
import org.apache.hyracks.storage.am.common.frames.AbstractSlotManager;
import org.apache.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndexBulkLoader;
import org.apache.hyracks.storage.am.common.impls.NodeFrontier;
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -902,7 +903,7 @@
return new RTreeBulkLoader(fillFactor, callback);
}
- public class RTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
+ public class RTreeBulkLoader extends AbstractTreeIndexBulkLoader {
ITreeIndexFrame lowerFrame, prevInteriorFrame;
RTreeTypeAwareTupleWriter interiorFrameTupleWriter =
((RTreeTypeAwareTupleWriter) interiorFrame.getTupleWriter());
@@ -911,7 +912,7 @@
List<Integer> prevNodeFrontierPages = new ArrayList<>();
public RTreeBulkLoader(float fillFactor, IPageWriteCallback callback) throws HyracksDataException {
- super(fillFactor, callback);
+ super(fillFactor, callback, RTree.this);
prevInteriorFrame = interiorFrameFactory.createFrame();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
index 61e2326..8f1b660 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
@@ -26,8 +26,8 @@
public class RTreeTypeAwareTupleReference extends TypeAwareTupleReference implements ITreeIndexTupleReference {
- public RTreeTypeAwareTupleReference(ITypeTraits[] typeTraits) {
- super(typeTraits);
+ public RTreeTypeAwareTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
+ super(typeTraits, nullTypeTraits);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
index 40b0481..02bf4a2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
@@ -22,14 +22,16 @@
import java.nio.ByteBuffer;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
public class RTreeTypeAwareTupleWriter extends TypeAwareTupleWriter {
- public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits) {
- super(typeTraits);
+ public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
}
public int writeTupleFields(ITreeIndexTupleReference[] refs, int startField, ByteBuffer targetBuf, int targetOff) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
index 4536a0c..56241e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
@@ -20,18 +20,20 @@
package org.apache.hyracks.storage.am.rtree.tuples;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
public class RTreeTypeAwareTupleWriterFactory extends TypeAwareTupleWriterFactory {
private static final long serialVersionUID = 1L;
- public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
- super(typeTraits);
+ public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+ INullIntrospector nullIntrospector) {
+ super(typeTraits, nullTypeTraits, nullIntrospector);
}
@Override
public RTreeTypeAwareTupleWriter createTupleWriter() {
- return new RTreeTypeAwareTupleWriter(typeTraits);
+ return new RTreeTypeAwareTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
index 204248f..7df5802 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.data.std.api.IPointableFactory;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -45,9 +46,10 @@
public static RTree createRTree(IBufferCache bufferCache, ITypeTraits[] typeTraits,
IPrimitiveValueProviderFactory[] valueProviderFactories, IBinaryComparatorFactory[] cmpFactories,
RTreePolicyType rtreePolicyType, FileReference file, boolean isPointMBR,
- IPageManagerFactory pageManagerFactory) {
+ IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
- RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
+ RTreeTypeAwareTupleWriterFactory tupleWriterFactory =
+ new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
valueProviderFactories, rtreePolicyType, isPointMBR);
ITreeIndexFrameFactory leafFrameFactory =
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
index f5ae30d..a1c4b02 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -69,7 +69,6 @@
<dependency>
<groupId>org.xerial.snappy</groupId>
<artifactId>snappy-java</artifactId>
- <version>1.1.7.1</version>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
index c485b32..46e3eec 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestJobletContext.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.test.support;
import java.nio.ByteBuffer;
+import java.time.ZoneId;
import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.context.IHyracksJobletContext;
@@ -40,6 +41,7 @@
private final JobId jobId;
private final WorkspaceFileFactory fileFactory;
private final long jobStartTime;
+ private final String jobStartTimeZoneId;
TestJobletContext(int frameSize, INCServiceContext serviceContext, JobId jobId) throws HyracksException {
this.serviceContext = serviceContext;
@@ -47,6 +49,7 @@
fileFactory = new WorkspaceFileFactory(this, getIoManager());
this.frameManger = new FrameManager(frameSize);
this.jobStartTime = System.currentTimeMillis();
+ this.jobStartTimeZoneId = ZoneId.systemDefault().getId();
}
@Override
@@ -125,6 +128,11 @@
}
@Override
+ public String getJobStartTimeZoneId() {
+ return jobStartTimeZoneId;
+ }
+
+ @Override
public Object getGlobalJobData() {
return null;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
index a2da285..ee10176 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -63,12 +63,7 @@
private static int pageSize;
private static int numPages;
private static int maxOpenFiles;
- private final static ThreadFactory threadFactory = new ThreadFactory() {
- @Override
- public Thread newThread(Runnable r) {
- return new Thread(r);
- }
- };
+ private final static ThreadFactory threadFactory = Thread::new;
public static void init(int pageSize, int numPages, int maxOpenFiles) {
TestStorageManagerComponentHolder.pageSize = pageSize;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeExamplesTest.java
index 8c0f5a5..9d91e14 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeExamplesTest.java
@@ -53,7 +53,7 @@
LinkedMetaDataPageManager freePageManager =
new LinkedMetaDataPageManager(harness.getBufferCache(), metaFrameFactory);
return BTreeUtils.createBTree(harness.getBufferCache(), typeTraits, cmpFactories,
- BTreeLeafFrameType.REGULAR_NSM, harness.getFileReference(), freePageManager, false);
+ BTreeLeafFrameType.REGULAR_NSM, harness.getFileReference(), freePageManager, false, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
index ad9198c..786a6af 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
@@ -39,7 +39,7 @@
IMetadataPageManager freePageManager = freePageManagerFactory.createPageManager(harness.getBufferCache());
index = BTreeUtils.createBTree(harness.getBufferCache(), SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), BTreeLeafFrameType.REGULAR_NSM,
- harness.getFileReference(), freePageManager, false);
+ harness.getFileReference(), freePageManager, false, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
index 2e2f306..82cd909 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
@@ -66,7 +66,7 @@
public static final int FIELD_COUNT = 2;
public static final ITypeTraits[] TYPE_TRAITS = { IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS };
public static final BTreeTypeAwareTupleWriterFactory TUPLE_WRITER_FACTORY =
- new BTreeTypeAwareTupleWriterFactory(TYPE_TRAITS, false);
+ new BTreeTypeAwareTupleWriterFactory(TYPE_TRAITS, false, null, null);
public static final ITreeIndexMetadataFrameFactory META_FRAME_FACTORY = new LIFOMetaDataFrameFactory();
public static final int KEY_FIELDS_COUNT = 1;
public static final IBinaryComparatorFactory[] CMP_FACTORIES = { IntegerBinaryComparatorFactory.INSTANCE };
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
index 89399bd..a7b4c4d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
@@ -40,7 +40,7 @@
new LinkedMetaDataPageManager(harness.getBufferCache(), metaFrameFactory);
index = BTreeUtils.createBTree(harness.getBufferCache(), SerdeUtils.serdesToTypeTraits(keySerdes),
SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), BTreeLeafFrameType.REGULAR_NSM,
- harness.getFileReference(), freePageManager, false);
+ harness.getFileReference(), freePageManager, false, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java
index 7a24727..97e5e46 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -84,7 +84,8 @@
int keyFieldCount = 1;
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- BTreeTypeAwareTupleWriterFactory tupleWriterFactory = new BTreeTypeAwareTupleWriterFactory(typeTraits, false);
+ BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
+ new BTreeTypeAwareTupleWriterFactory(typeTraits, false, null, null);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
ITreeIndexMetadataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
index 5b86b4f..720dd59 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
@@ -74,7 +74,8 @@
ISerializerDeserializer[] recDescSers =
{ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
- BTreeTypeAwareTupleWriterFactory tupleWriterFactory = new BTreeTypeAwareTupleWriterFactory(typeTraits, false);
+ BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
+ new BTreeTypeAwareTupleWriterFactory(typeTraits, false, null, null);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
ITreeIndexMetadataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/FieldPrefixNSMTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/FieldPrefixNSMTest.java
index fe05828..e9b9e7f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/FieldPrefixNSMTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/FieldPrefixNSMTest.java
@@ -131,7 +131,7 @@
ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(btreeFileId, 0), true);
try {
- BTreeTypeAwareTupleWriter tupleWriter = new BTreeTypeAwareTupleWriter(typeTraits, false);
+ BTreeTypeAwareTupleWriter tupleWriter = new BTreeTypeAwareTupleWriter(typeTraits, false, null, null);
BTreeFieldPrefixNSMLeafFrame frame = new BTreeFieldPrefixNSMLeafFrame(tupleWriter);
frame.setPage(page);
frame.initBuffer((byte) 0);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
index dcdbdc6..6b94b03 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
@@ -54,7 +54,7 @@
int[] bloomFilterKeyFields) throws HyracksDataException {
return BTreeUtils.createBTree(harness.getBufferCache(), typeTraits, cmpFactories,
BTreeLeafFrameType.REGULAR_NSM, harness.getFileReference(),
- harness.getPageManagerFactory().createPageManager(harness.getBufferCache()), false);
+ harness.getPageManagerFactory().createPageManager(harness.getBufferCache()), false, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java
index 9d66b64..eaa1c28 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/util/BTreeTestContext.java
@@ -56,7 +56,8 @@
IPageManager pageManager) throws Exception {
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
- BTree btree = BTreeUtils.createBTree(bufferCache, typeTraits, cmpFactories, leafType, file, pageManager, false);
+ BTree btree = BTreeUtils.createBTree(bufferCache, typeTraits, cmpFactories, leafType, file, pageManager, false,
+ null, null);
BTreeTestContext testCtx = new BTreeTestContext(fieldSerdes, btree);
return testCtx;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
index cb791c5..7e0ee35 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
@@ -100,6 +100,11 @@
encapsulated.allocated(component);
}
+ @Override
+ public long getLastValidSequence() throws HyracksDataException {
+ return encapsulated.getLastValidSequence();
+ }
+
public ILSMIOOperationCallback getEncapsulated() {
return encapsulated;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index 1d41280..3313443 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -64,7 +64,7 @@
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
harness.getPageWriteCallbackFactory(), true, filterTypeTraits, filterCmpFactories, btreeFields,
filterFields, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
- NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null);
+ NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null);
}
@Before
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 8d8ec23..2476761 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -54,7 +54,7 @@
NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(), true, null, null, null,
null, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
- NoOpCompressorDecompressorFactory.INSTANCE, HAS_BLOOM_FILTER);
+ NoOpCompressorDecompressorFactory.INSTANCE, HAS_BLOOM_FILTER, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index ab068ae..6dec9f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -63,7 +63,7 @@
NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(), true, null, null, null,
null, true, harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
- NoOpCompressorDecompressorFactory.INSTANCE, HAS_BLOOM_FILTER);
+ NoOpCompressorDecompressorFactory.INSTANCE, HAS_BLOOM_FILTER, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
index 2954854..0a4b37b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
@@ -76,7 +76,7 @@
NoOpOperationTrackerFactory.INSTANCE.getOperationTracker(null, null), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(), true, null, null, null,
null, true, harness.getMetadataPageManagerFactory(), true, ITracer.NONE,
- NoOpCompressorDecompressorFactory.INSTANCE, HAS_BLOOM_FILTER);
+ NoOpCompressorDecompressorFactory.INSTANCE, HAS_BLOOM_FILTER, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
index 3450531..8e2d6dd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResource.java
@@ -61,7 +61,7 @@
storageManager, mergePolicyFactory, mergePolicyProperties, filterTypeTraits, filterCmpFactories,
btreeFields, filterFields, opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory,
metadataPageManagerFactory, vbcProvider, ioSchedulerProvider, durable,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
}
protected TestLsmBtreeLocalResource(IPersistedResourceRegistry registry, JsonNode json, int[] bloomFilterKeyFields,
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
index 229b027..7d9b24a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeLocalResourceFactory.java
@@ -51,7 +51,7 @@
opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable,
bloomFilterKeyFields, bloomFilterFalsePositiveRate, isPrimary, btreeFields,
- NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter);
+ NoOpCompressorDecompressorFactory.INSTANCE, hasBloomFilter, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
index eebfb2b..dd42838 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeUtil.java
@@ -68,11 +68,11 @@
int[] btreeFields, int[] filterFields, boolean durable, IMetadataPageManagerFactory freePageManagerFactory,
boolean updateAware, ITracer tracer) throws HyracksDataException {
LSMBTreeTupleWriterFactory insertTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware);
+ new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, false, updateAware, null, null);
LSMBTreeTupleWriterFactory deleteTupleWriterFactory =
- new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, true, updateAware);
+ new LSMBTreeTupleWriterFactory(typeTraits, cmpFactories.length, true, updateAware, null, null);
LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory =
- new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length, updateAware);
+ new LSMBTreeCopyTupleWriterFactory(typeTraits, cmpFactories.length, updateAware, null, null);
ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
@@ -88,7 +88,8 @@
LSMComponentFilterFrameFactory filterFrameFactory = null;
LSMComponentFilterManager filterManager = null;
if (filterCmpFactories != null) {
- TypeAwareTupleWriterFactory filterTupleWriterFactory = new TypeAwareTupleWriterFactory(filterTypeTraits);
+ TypeAwareTupleWriterFactory filterTupleWriterFactory =
+ new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
filterManager = new LSMComponentFilterManager(filterFrameFactory);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index fb1244f..66af4b2 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -60,7 +60,7 @@
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
harness.getPageWriteCallbackFactory(), true, null, null, null, null, true,
harness.getMetadataPageManagerFactory(), false, ITracer.NONE,
- NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null);
+ NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeRunner.java
index 60a054e..baa5508 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/BTreeRunner.java
@@ -49,6 +49,6 @@
ITreeIndexMetadataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
LinkedMetaDataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, metaFrameFactory);
btree = BTreeUtils.createBTree(bufferCache, typeTraits, cmpFactories, BTreeLeafFrameType.REGULAR_NSM, file,
- freePageManager, false);
+ freePageManager, false, null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/ConcurrentSkipListRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/ConcurrentSkipListRunner.java
index c476a8c..c57badc 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/ConcurrentSkipListRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/ConcurrentSkipListRunner.java
@@ -116,7 +116,7 @@
this.dataGen = dataGen;
this.numBatches = numBatches;
this.skipList = skipList;
- tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits, null, null);
tupleWriter = tupleWriterFactory.createTupleWriter();
int numTuples = numBatches * batchSize;
tuples = new TypeAwareTupleReference[numTuples];
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
index 3307093..1eb4646 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
@@ -68,7 +68,8 @@
protected void init(int pageSize, int numPages, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories)
throws HyracksDataException {
bufferCache = new VirtualBufferCache(new HeapBufferAllocator(), pageSize, numPages);
- BTreeTypeAwareTupleWriterFactory tupleWriterFactory = new BTreeTypeAwareTupleWriterFactory(typeTraits, false);
+ BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
+ new BTreeTypeAwareTupleWriterFactory(typeTraits, false, null, null);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
IPageManager freePageManager = new VirtualFreePageManager(bufferCache);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
index 4501485..1455fb3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
@@ -71,7 +71,7 @@
this.batchSize = batchSize;
this.typeTraits = typeTraits;
tupleCmp = new TupleComparator(cmp);
- tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits, null, null);
tupleWriter = tupleWriterFactory.createTupleWriter();
int numTuples = numBatches * batchSize;
tuples = new ArrayList<TypeAwareTupleReference>();
@@ -131,7 +131,7 @@
this.dataGen = dataGen;
this.numBatches = numBatches;
this.skipList = skipList;
- tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits, null, null);
tupleWriter = tupleWriterFactory.createTupleWriter();
int numTuples = numBatches * batchSize;
tuples = new TypeAwareTupleReference[numTuples];
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index f487bf1..6126040 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -81,12 +81,7 @@
protected IBufferCache memBufferCache;
private final int onDiskPageSize;
private final int onDiskNumPages;
- private final static ThreadFactory threadFactory = new ThreadFactory() {
- @Override
- public Thread newThread(Runnable r) {
- return new Thread(r);
- }
- };
+ private final static ThreadFactory threadFactory = Thread::new;
public LSMTreeRunner(int numBatches, int inMemPageSize, int inMemNumPages, int onDiskPageSize, int onDiskNumPages,
ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
@@ -128,7 +123,7 @@
new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallbackFactory.INSTANCE,
NoOpPageWriteCallbackFactory.INSTANCE, true, null, null, null, null, true,
TestStorageManagerComponentHolder.getMetadataPageManagerFactory(), false, ITracer.NONE,
- NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null);
+ NoOpCompressorDecompressorFactory.INSTANCE, bloomFilterKeyFields != null, null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
index b81167b..0a34c45 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
@@ -68,7 +68,7 @@
for (int numFields = numKeyFields; numFields <= maxFieldSerdes.length; numFields++) {
// Create and write tuple to bytes using an LSMBTreeTupleWriter.
LSMBTreeTupleWriter maxMatterTupleWriter =
- new LSMBTreeTupleWriter(maxTypeTraits, numKeyFields, false, false);
+ new LSMBTreeTupleWriter(maxTypeTraits, numKeyFields, false, false, null, null);
ITupleReference maxTuple = TupleUtils.createTuple(maxFieldSerdes, (Object[]) maxFields);
ByteBuffer maxMatterBuf = writeTuple(maxTuple, maxMatterTupleWriter);
// Tuple reference should work for both matter and antimatter tuples (doesn't matter which factory creates it).
@@ -84,10 +84,12 @@
}
// Create and write tuple to bytes using an LSMBTreeTupleWriter.
ITupleReference tuple = TupleUtils.createTuple(fieldSerdes, (Object[]) fields);
- LSMBTreeTupleWriter matterTupleWriter = new LSMBTreeTupleWriter(typeTraits, numKeyFields, false, false);
+ LSMBTreeTupleWriter matterTupleWriter =
+ new LSMBTreeTupleWriter(typeTraits, numKeyFields, false, false, null, null);
LSMBTreeTupleWriter antimatterTupleWriter =
- new LSMBTreeTupleWriter(typeTraits, numKeyFields, true, false);
- LSMBTreeCopyTupleWriter copyTupleWriter = new LSMBTreeCopyTupleWriter(typeTraits, numKeyFields, false);
+ new LSMBTreeTupleWriter(typeTraits, numKeyFields, true, false, null, null);
+ LSMBTreeCopyTupleWriter copyTupleWriter =
+ new LSMBTreeCopyTupleWriter(typeTraits, numKeyFields, false, null, null);
ByteBuffer matterBuf = writeTuple(tuple, matterTupleWriter);
ByteBuffer antimatterBuf = writeTuple(tuple, antimatterTupleWriter);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index 6da04e7..928f60c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -103,7 +103,7 @@
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, needKeyDupCheck, filterTypeTraits,
filterCmp, btreefields, filterfields, true, metadataPageManagerFactory, updateAware, ITracer.NONE,
- NoOpCompressorDecompressorFactory.INSTANCE, true);
+ NoOpCompressorDecompressorFactory.INSTANCE, true, null, null);
} else {
lsmTree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, diskBufferCache, typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
@@ -111,7 +111,7 @@
true, metadataPageManagerFactory,
updateAware, new Tracer(LSMBTreeTestContext.class.getSimpleName(),
ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
- NoOpCompressorDecompressorFactory.INSTANCE, true);
+ NoOpCompressorDecompressorFactory.INSTANCE, true, null, null);
}
LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree, filtered);
return testCtx;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMComponentFilterReferenceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMComponentFilterReferenceTest.java
index 143204f..612ca4b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMComponentFilterReferenceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/LSMComponentFilterReferenceTest.java
@@ -34,7 +34,7 @@
@Test
public void test() throws HyracksDataException {
LSMComponentFilterReference filter = new LSMComponentFilterReference(
- new TypeAwareTupleWriter(new ITypeTraits[] { IntegerPointable.TYPE_TRAITS }));
+ new TypeAwareTupleWriter(new ITypeTraits[] { IntegerPointable.TYPE_TRAITS }, null, null));
Assert.assertEquals(filter.getLength(), 0);
Assert.assertFalse(filter.isMaxTupleSet() || filter.isMinTupleSet());
filter.writeMaxTuple(TupleUtils.createIntegerTuple(false, Integer.MAX_VALUE));
@@ -46,7 +46,7 @@
Assert.assertTrue(filter.getLength() == 20);
byte[] serFilter = filter.getByteArray();
LSMComponentFilterReference deserFilter = new LSMComponentFilterReference(
- new TypeAwareTupleWriter((new ITypeTraits[] { IntegerPointable.TYPE_TRAITS })));
+ new TypeAwareTupleWriter(new ITypeTraits[] { IntegerPointable.TYPE_TRAITS }, null, null));
deserFilter.set(serFilter, 0, 20);
Assert.assertTrue(deserFilter.isMaxTupleSet() && deserFilter.isMinTupleSet());
Assert.assertEquals(
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
index 0c626df..1b17ef9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
@@ -65,7 +65,7 @@
.resolveAbsolutePath(harness.getInvListsFileRef().getFile().getAbsolutePath() + "_btree");
index = new OnDiskInvertedIndex(harness.getDiskBufferCache(), invListBuilder, invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, harness.getInvListsFileRef(), btreeFile,
- harness.getMetadataPageManagerFactory());
+ harness.getMetadataPageManagerFactory(), null, null);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 3e8db34..e745f5d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -159,7 +159,8 @@
invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCaches().get(0),
new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+ fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()), null,
+ null);
break;
}
case PARTITIONED_INMEMORY: {
@@ -167,47 +168,53 @@
harness.getVirtualBufferCaches().get(0),
new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+ fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()), null,
+ null);
break;
}
case ONDISK: {
invIndex = InvertedIndexUtils.createOnDiskInvertedIndex(ioManager, harness.getDiskBufferCache(),
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
- harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory());
+ harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory(), null, null);
break;
}
case PARTITIONED_ONDISK: {
- invIndex = InvertedIndexUtils.createPartitionedOnDiskInvertedIndex(ioManager,
- harness.getDiskBufferCache(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory());
+ invIndex =
+ InvertedIndexUtils.createPartitionedOnDiskInvertedIndex(ioManager, harness.getDiskBufferCache(),
+ invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+ harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory(), null, null);
break;
}
case LSM: {
- invIndex = InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
- invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
- invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
- harness.getMetadataPageManagerFactory(),
- new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
- ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
+ invIndex =
+ InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+ invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+ tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
+ harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+ harness.getMetadataPageManagerFactory(),
+ new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+ ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
+ null, null);
break;
}
case PARTITIONED_LSM: {
- invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager,
- harness.getVirtualBufferCaches(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
- harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
- harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
- invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
- harness.getMetadataPageManagerFactory(),
- new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
- ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
+ invIndex = InvertedIndexUtils
+ .createPartitionedLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+ invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+ tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
+ harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+ harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+ invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+ harness.getMetadataPageManagerFactory(),
+ new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+ ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
+ null, null);
break;
}
default: {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index 3ca25ab..9e6fb83 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -50,7 +50,7 @@
harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields, btreeFields,
filterTypeTraits, filterCmpFactories, filterFields, true, false,
- harness.getMetadataPageManagerFactory());
+ harness.getMetadataPageManagerFactory(), null, null);
}
@Before
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 75f656d..baf54f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -50,7 +50,7 @@
harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields,
filterTypeTraits, filterCmpFactories, filterFields, true, false,
- harness.getMetadataPageManagerFactory());
+ harness.getMetadataPageManagerFactory(), null, null);
}
@Before
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index bbf6263..980f21f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -66,7 +66,7 @@
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, btreeFields, null,
- null, null, true, false, harness.getMetadataPageManagerFactory());
+ null, null, true, false, harness.getMetadataPageManagerFactory(), null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index 7d6e2e6..f40add9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -67,7 +67,7 @@
harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
harness.getPageWriteCallbackFactory(),
LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, null, null, null, true,
- false, harness.getMetadataPageManagerFactory());
+ false, harness.getMetadataPageManagerFactory(), null, null);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index 2140558..02742a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -109,7 +109,7 @@
typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
- null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory);
+ null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory, null, null);
LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
return testCtx;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index f9d9046..ff953f1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -90,7 +90,7 @@
virtualBufferCaches, file, diskBufferCache, typeTraits, rtreeCmpFactories, btreeCmpFactories,
valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
- null, null, null, null, true, false, metadataPageManagerFactory);
+ null, null, null, null, true, false, metadataPageManagerFactory, null, null);
LSMRTreeWithAntiMatterTuplesTestContext testCtx =
new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes, lsmTree);
return testCtx;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
index 999fd65..cb68d77 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
@@ -54,7 +54,7 @@
RTreePolicyType rtreePolicyType, int[] rtreeFields, int[] btreeFields, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException {
return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits, valueProviderFactories, rtreeCmpFactories,
- rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory());
+ rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory(), null, null);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index bef7ac7..19672dd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -76,7 +76,7 @@
public static final IPrimitiveValueProviderFactory[] VALUE_PROVIDER_FACTORY =
RTreeUtils.createPrimitiveValueProviderFactories(CMP_FACTORIES.length, IntegerPointable.FACTORY);
public static final RTreeTypeAwareTupleWriterFactory TUPLE_WRITER_FACTORY =
- new RTreeTypeAwareTupleWriterFactory(TYPE_TRAITS);
+ new RTreeTypeAwareTupleWriterFactory(TYPE_TRAITS, null, null);
public static final ITreeIndexMetadataFrameFactory META_FRAME_FACTORY = new LIFOMetaDataFrameFactory();
public static final ITreeIndexFrameFactory INTERIOR_FRAME_FACTORY = new RTreeNSMInteriorFrameFactory(
TUPLE_WRITER_FACTORY, VALUE_PROVIDER_FACTORY, RTreePolicyType.RTREE, false);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
index faa63c5..02c267d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
@@ -61,7 +61,7 @@
IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
RTreePolicyType rtreePolicyType, int[] btreeFields) throws HyracksDataException {
return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits, valueProviderFactories, rtreeCmpFactories,
- rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory());
+ rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory(), null, null);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
index bf70dc1..06ee100 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
@@ -60,7 +60,7 @@
ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
RTree rtree = RTreeUtils.createRTree(bufferCache, typeTraits, valueProviderFactories, cmpFactories,
- rtreePolicyType, file, false, pageManagerFactory);
+ rtreePolicyType, file, false, pageManagerFactory, null, null);
RTreeTestContext testCtx = new RTreeTestContext(fieldSerdes, rtree);
return testCtx;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MaintainedThreadNameExecutorService.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MaintainedThreadNameExecutorService.java
index 9adac04..9b316ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MaintainedThreadNameExecutorService.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MaintainedThreadNameExecutorService.java
@@ -45,6 +45,10 @@
@Override
protected void afterExecute(Runnable r, Throwable t) {
super.afterExecute(r, t);
- Thread.currentThread().setName(savedName.get());
+ try {
+ Thread.currentThread().setName(savedName.get());
+ } finally {
+ savedName.remove();
+ }
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/OptionalBoolean.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/OptionalBoolean.java
new file mode 100644
index 0000000..b64a315
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/OptionalBoolean.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+import java.util.NoSuchElementException;
+
+/**
+ * Provides functionality similar to {@link java.util.Optional} for the primitive boolean type
+ */
+public class OptionalBoolean {
+ private static final OptionalBoolean EMPTY = new OptionalBoolean(false) {
+ @Override
+ public boolean isPresent() {
+ return false;
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return true;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ return o == this;
+ }
+
+ @Override
+ public int hashCode() {
+ return -1;
+ }
+
+ @Override
+ public boolean get() {
+ throw new NoSuchElementException();
+ }
+ };
+
+ private static final OptionalBoolean TRUE = new OptionalBoolean(true);
+ private static final OptionalBoolean FALSE = new OptionalBoolean(false);
+
+ private final boolean value;
+
+ public static OptionalBoolean of(boolean value) {
+ return value ? TRUE : FALSE;
+ }
+
+ public static OptionalBoolean ofNullable(Boolean value) {
+ return value == null ? EMPTY : value ? TRUE : FALSE;
+ }
+
+ public static OptionalBoolean empty() {
+ return EMPTY;
+ }
+
+ public static OptionalBoolean TRUE() {
+ return TRUE;
+ }
+
+ public static OptionalBoolean FALSE() {
+ return FALSE;
+ }
+
+ private OptionalBoolean(boolean value) {
+ this.value = value;
+ }
+
+ public boolean isPresent() {
+ return true;
+ }
+
+ public boolean isEmpty() {
+ return false;
+ }
+
+ public boolean get() {
+ return value;
+ }
+
+ public boolean getOrElse(boolean alternate) {
+ return isPresent() ? get() : alternate;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (this == EMPTY || o == EMPTY)
+ return false;
+ if (o == null || getClass() != o.getClass())
+ return false;
+ OptionalBoolean that = (OptionalBoolean) o;
+ return value == that.value;
+ }
+
+ @Override
+ public int hashCode() {
+ return value ? 1 : 0;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/PidHelper.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/PidHelper.java
index 46e77e3..3cba2d3 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/PidHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/PidHelper.java
@@ -20,9 +20,6 @@
import java.lang.management.ManagementFactory;
import java.lang.management.RuntimeMXBean;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -40,22 +37,8 @@
}
public static long getPid(RuntimeMXBean runtimeMXBean) {
- // TODO: replace with direct invoke of getPid() once compatibility is at JDK 10 or higher
try {
- Method getPidMethod = runtimeMXBean.getClass().getMethod("getPid");
- return (Long) getPidMethod.invoke(runtimeMXBean);
- } catch (NoSuchMethodException e) {
- LOGGER.debug("ignoring exception trying to find getPid() (expected pre-JDK 10)", e);
- } catch (IllegalAccessException | InvocationTargetException e) {
- LOGGER.debug("ignoring exception trying to execute getPid()", e);
- }
- try {
- Field jvmField = runtimeMXBean.getClass().getDeclaredField("jvm");
- jvmField.setAccessible(true);
- Object vmManagement = jvmField.get(runtimeMXBean);
- Method getProcessIdMethod = vmManagement.getClass().getDeclaredMethod("getProcessId");
- getProcessIdMethod.setAccessible(true);
- return (Integer) getProcessIdMethod.invoke(vmManagement);
+ return runtimeMXBean.getPid();
} catch (Exception e) {
LOGGER.log(Level.INFO, "Unable to determine PID due to exception", e);
return -1;
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
index dc8bc11..1a40360 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.util;
+import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
@@ -70,6 +71,12 @@
}
@Override
+ public boolean await(CountDownLatch latch) throws InterruptedException {
+ latch.await();
+ return true;
+ }
+
+ @Override
public String toString() {
return "<INFINITE>";
}
@@ -143,6 +150,10 @@
TimeUnit.NANOSECONDS.timedWait(monitor, remaining(TimeUnit.NANOSECONDS));
}
+ public boolean await(CountDownLatch latch) throws InterruptedException {
+ return latch.await(remaining(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+ }
+
public void loopUntilExhausted(ThrowingAction action) throws Exception {
loopUntilExhausted(action, 0, TimeUnit.NANOSECONDS);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
index 78155c9..df33f6b 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
@@ -50,4 +50,21 @@
}
return sb.toString();
}
+
+ @FunctionalInterface
+ public interface ICharAccessor<T> {
+ char charAt(T input, int index);
+ }
+
+ public static ICharAccessor<CharSequence> getCharSequenceAccessor() {
+ return CharSequence::charAt;
+ }
+
+ public static ICharAccessor<char[]> getCharArrayAccessor() {
+ return (input, index) -> input[index];
+ }
+
+ public static ICharAccessor<byte[]> getByteArrayAsCharAccessor() {
+ return (input, index) -> (char) input[index];
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingBiConsumer.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingBiConsumer.java
new file mode 100644
index 0000000..3d7541b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingBiConsumer.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+import java.util.function.BiConsumer;
+
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
+@FunctionalInterface
+public interface ThrowingBiConsumer<L, R> {
+ void process(L left, R right) throws Exception;
+
+ @SuppressWarnings("Duplicates")
+ static <L, R> BiConsumer<L, R> asUnchecked(ThrowingBiConsumer<L, R> consumer) {
+ return (left, right) -> {
+ try {
+ consumer.process(left, right);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new UncheckedExecutionException(e);
+ } catch (Exception e) {
+ throw new UncheckedExecutionException(e);
+ }
+ };
+ }
+
+ static <I, J> ThrowingBiFunction<I, J, Void> asFunction(ThrowingBiConsumer<I, J> consumer) {
+ return (p1, p2) -> {
+ consumer.process(p1, p2);
+ return null;
+ };
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingBiFunction.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingBiFunction.java
new file mode 100644
index 0000000..b8fdc8b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingBiFunction.java
@@ -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.
+ */
+package org.apache.hyracks.util;
+
+import java.util.function.BiFunction;
+
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
+@FunctionalInterface
+public interface ThrowingBiFunction<I, J, R> {
+ R process(I p1, J p2) throws Exception;
+
+ @SuppressWarnings("Duplicates")
+ static <I, J, R> BiFunction<I, J, R> asUnchecked(ThrowingBiFunction<I, J, R> function) {
+ return (p1, p2) -> {
+ try {
+ return function.process(p1, p2);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new UncheckedExecutionException(e);
+ } catch (Exception e) {
+ throw new UncheckedExecutionException(e);
+ }
+ };
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/Base64Parser.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/Base64Parser.java
index 9f527e97..c47d47a 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/Base64Parser.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/Base64Parser.java
@@ -21,6 +21,8 @@
import java.util.Arrays;
+import org.apache.hyracks.util.StringUtil;
+
public class Base64Parser {
private static final byte[] DECODE_MAP = initDecodeMap();
private static final byte PADDING = 127;
@@ -52,42 +54,6 @@
/**
* Parse the Base64 sequence from {@code input} into {@code out}
- * Note, the out should have enough space by checking the {@link #guessLength(char[], int, int)} first
- *
- * @param input
- * @param start
- * @param length
- * @param out
- * @param offset
- * @return
- */
- public int parseBase64String(char[] input, int start, int length, byte[] out, int offset) {
- int outLength = 0;
-
- int i;
- int q = 0;
-
- // convert each quadruplet to three bytes.
- for (i = 0; i < length; i++) {
- char ch = input[start + i];
- byte v = DECODE_MAP[ch];
-
- if (v == -1) {
- throw new IllegalArgumentException("Invalid Base64 character");
- }
- quadruplet[q++] = v;
-
- if (q == 4) {
- outLength += dumpQuadruplet(out, offset + outLength);
- q = 0;
- }
- }
-
- return outLength;
- }
-
- /**
- * Parse the Base64 sequence from {@code input} into {@code out}
* Note, the out should have enough space by checking the {@link #guessLength(byte[], int, int)} first
*
* @param input
@@ -98,6 +64,52 @@
* @return the number of written bytes
*/
public int parseBase64String(byte[] input, int start, int length, byte[] out, int offset) {
+ return parseBase64String(input, start, length, StringUtil.getByteArrayAsCharAccessor(), out, offset);
+ }
+
+ /**
+ * Parse the Base64 sequence from {@code input} into {@code out}
+ * Note, the out should have enough space by checking the {@link #guessLength(char[], int, int)} first
+ *
+ * @param input
+ * @param start
+ * @param length
+ * @param out
+ * @param offset
+ * @return
+ */
+ public int parseBase64String(char[] input, int start, int length, byte[] out, int offset) {
+ return parseBase64String(input, start, length, StringUtil.getCharArrayAccessor(), out, offset);
+ }
+
+ /**
+ * Parse the Base64 sequence from {@code input} into {@code out}
+ * Note, the out should have enough space by checking the {@link #guessLength(CharSequence, int, int)} first
+ *
+ * @param input
+ * @param start
+ * @param length
+ * @param out
+ * @param offset
+ * @return
+ */
+ public int parseBase64String(CharSequence input, int start, int length, byte[] out, int offset) {
+ return parseBase64String(input, start, length, StringUtil.getCharSequenceAccessor(), out, offset);
+ }
+
+ /**
+ * Parse the Base64 sequence from {@code input} into {@code out}
+ * Note, the out should have enough space by checking the {@link #guessLength(char[], int, int)} first
+ *
+ * @param input
+ * @param start
+ * @param length
+ * @param out
+ * @param offset
+ * @return
+ */
+ private <T> int parseBase64String(T input, int start, int length, StringUtil.ICharAccessor<T> stringAccessor,
+ byte[] out, int offset) {
int outLength = 0;
int i;
@@ -105,7 +117,7 @@
// convert each quadruplet to three bytes.
for (i = 0; i < length; i++) {
- char ch = (char) input[start + i];
+ char ch = stringAccessor.charAt(input, start + i);
byte v = DECODE_MAP[ch];
if (v == -1) {
@@ -136,40 +148,66 @@
* (like what most web services produce), then the speculation of this method
* will be correct, so we get the performance benefit.
*/
- public static int guessLength(char[] chars, int start, int length) {
-
- // compute the tail '=' chars
- int j = length - 1;
- for (; j >= 0; j--) {
- byte code = DECODE_MAP[chars[start + j]];
- if (code == PADDING) {
- continue;
- }
- if (code == -1) // most likely this base64 text is indented. go with the upper bound
- {
- return length / 4 * 3;
- }
- break;
- }
-
- j++; // text.charAt(j) is now at some base64 char, so +1 to make it the size
- int padSize = length - j;
- if (padSize > 2) // something is wrong with base64. be safe and go with the upper bound
- {
- return length / 4 * 3;
- }
-
- // so far this base64 looks like it's unindented tightly packed base64.
- // take a chance and create an array with the expected size
- return length / 4 * 3 - padSize;
+ public static int guessLength(byte[] input, int start, int length) {
+ return guessLength(input, start, length, StringUtil.getByteArrayAsCharAccessor());
}
- public static int guessLength(byte[] chars, int start, int length) {
+ /**
+ * computes the length of binary data speculatively.
+ * Our requirement is to create byte[] of the exact length to store the binary data.
+ * If we do this in a straight-forward way, it takes two passes over the data.
+ * Experiments show that this is a non-trivial overhead (35% or so is spent on
+ * the first pass in calculating the length.)
+ * So the approach here is that we compute the length speculatively, without looking
+ * at the whole contents. The obtained speculative value is never less than the
+ * actual length of the binary data, but it may be bigger. So if the speculation
+ * goes wrong, we'll pay the cost of reallocation and buffer copying.
+ * If the base64 text is tightly packed with no indentation nor illegal char
+ * (like what most web services produce), then the speculation of this method
+ * will be correct, so we get the performance benefit.
+ */
+ public static int guessLength(char[] input, int start, int length) {
+ return guessLength(input, start, length, StringUtil.getCharArrayAccessor());
+ }
+ /**
+ * computes the length of binary data speculatively.
+ * Our requirement is to create byte[] of the exact length to store the binary data.
+ * If we do this in a straight-forward way, it takes two passes over the data.
+ * Experiments show that this is a non-trivial overhead (35% or so is spent on
+ * the first pass in calculating the length.)
+ * So the approach here is that we compute the length speculatively, without looking
+ * at the whole contents. The obtained speculative value is never less than the
+ * actual length of the binary data, but it may be bigger. So if the speculation
+ * goes wrong, we'll pay the cost of reallocation and buffer copying.
+ * If the base64 text is tightly packed with no indentation nor illegal char
+ * (like what most web services produce), then the speculation of this method
+ * will be correct, so we get the performance benefit.
+ */
+ public static int guessLength(CharSequence input, int start, int length) {
+ return guessLength(input, start, length, StringUtil.getCharSequenceAccessor());
+ }
+
+ /**
+ * computes the length of binary data speculatively.
+ * Our requirement is to create byte[] of the exact length to store the binary data.
+ * If we do this in a straight-forward way, it takes two passes over the data.
+ * Experiments show that this is a non-trivial overhead (35% or so is spent on
+ * the first pass in calculating the length.)
+ * So the approach here is that we compute the length speculatively, without looking
+ * at the whole contents. The obtained speculative value is never less than the
+ * actual length of the binary data, but it may be bigger. So if the speculation
+ * goes wrong, we'll pay the cost of reallocation and buffer copying.
+ * If the base64 text is tightly packed with no indentation nor illegal char
+ * (like what most web services produce), then the speculation of this method
+ * will be correct, so we get the performance benefit.
+ */
+ private static <T> int guessLength(T input, int start, int length, StringUtil.ICharAccessor<T> charAtFunction) {
// compute the tail '=' chars
int j = length - 1;
for (; j >= 0; j--) {
- byte code = DECODE_MAP[chars[start + j]];
+ char ch = charAtFunction.charAt(input, start + j);
+ byte code = DECODE_MAP[ch];
if (code == PADDING) {
continue;
}
@@ -208,24 +246,47 @@
* @param length
*/
public void generatePureByteArrayFromBase64String(byte[] input, int start, int length) {
- // The base64 character length equals to utf8length
- if (length % 4 != 0) {
- throw new IllegalArgumentException(
- "Invalid Base64 string, the length of the string should be a multiple of 4");
- }
- final int buflen = guessLength(input, start, length);
- ensureCapacity(buflen);
- this.length = parseBase64String(input, start, length, storage, 0);
+ generatePureByteArrayFromBase64String(input, start, length, StringUtil.getByteArrayAsCharAccessor());
}
+ /**
+ * Same as {@link #parseBase64String(char[], int, int, byte[], int)}, but we will provide the storage for caller
+ *
+ * @param input
+ * @param start
+ * @param length
+ */
public void generatePureByteArrayFromBase64String(char[] input, int start, int length) {
+ generatePureByteArrayFromBase64String(input, start, length, StringUtil.getCharArrayAccessor());
+ }
+
+ /**
+ * Same as {@link #parseBase64String(CharSequence, int, int, byte[], int)}, but we will provide the storage for caller
+ *
+ * @param input
+ * @param start
+ * @param length
+ */
+ public void generatePureByteArrayFromBase64String(CharSequence input, int start, int length) {
+ generatePureByteArrayFromBase64String(input, start, length, StringUtil.getCharSequenceAccessor());
+ }
+
+ /**
+ * Same as {@link #parseBase64String(Object, int, int, StringUtil.ICharAccessor, byte[], int)}, but we will provide the storage for caller
+ *
+ * @param input
+ * @param start
+ * @param length
+ */
+ private <T> void generatePureByteArrayFromBase64String(T input, int start, int length,
+ StringUtil.ICharAccessor<T> charAtFunction) {
if (length % 4 != 0) {
throw new IllegalArgumentException(
"Invalid Base64 string, the length of the string should be a multiple of 4");
}
- final int buflen = guessLength(input, start, length);
+ final int buflen = guessLength(input, start, length, charAtFunction);
ensureCapacity(buflen);
- this.length = parseBase64String(input, start, length, storage, 0);
+ this.length = parseBase64String(input, start, length, charAtFunction, storage, 0);
}
private void ensureCapacity(int length) {
@@ -246,5 +307,4 @@
}
return outLength;
}
-
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexParser.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexParser.java
index 46bc0a4..260e99f 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexParser.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexParser.java
@@ -79,15 +79,17 @@
public static void generateByteArrayFromHexString(char[] input, int start, int length, byte[] output, int offset) {
for (int i = 0; i < length; i += 2) {
- output[offset + i / 2] = (byte) ((getValueFromValidHexChar(input[start + i]) << 4)
- + getValueFromValidHexChar(input[start + i + 1]));
+ output[offset + i / 2] = getByteFromValidHexChars(input[start + i], input[start + i + 1]);
}
}
public static void generateByteArrayFromHexString(byte[] input, int start, int length, byte[] output, int offset) {
for (int i = 0; i < length; i += 2) {
- output[offset + i / 2] = (byte) ((getValueFromValidHexChar((char) input[start + i]) << 4)
- + getValueFromValidHexChar((char) input[start + i + 1]));
+ output[offset + i / 2] = getByteFromValidHexChars((char) input[start + i], (char) input[start + i + 1]);
}
}
+
+ public static byte getByteFromValidHexChars(char c0, char c1) {
+ return (byte) ((getValueFromValidHexChar(c0) << 4) + getValueFromValidHexChar(c1));
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexPrinter.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexPrinter.java
index d340526..f143e9f 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexPrinter.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/bytes/HexPrinter.java
@@ -37,12 +37,14 @@
return (byte) (i < 10 ? i + '0' : i + (c.a - 10));
}
- public static Appendable printHexString(byte[] bytes, int start, int length, Appendable appendable)
- throws IOException {
+ public static void printHexString(byte[] bytes, int start, int length, Appendable appendable) throws IOException {
for (int i = 0; i < length; ++i) {
- appendable.append((char) hex((bytes[start + i] >>> 4) & 0x0f, Case.UPPER_CASE));
- appendable.append((char) hex((bytes[start + i] & 0x0f), Case.UPPER_CASE));
+ printByte(bytes[start + i], appendable);
}
- return appendable;
+ }
+
+ public static void printByte(byte b, Appendable appendable) throws IOException {
+ appendable.append((char) hex((b >>> 4) & 0x0f, Case.UPPER_CASE));
+ appendable.append((char) hex((b & 0x0f), Case.UPPER_CASE));
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/ITracer.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/ITracer.java
index 908dc1c..3748cd8 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/ITracer.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/ITracer.java
@@ -19,6 +19,8 @@
package org.apache.hyracks.util.trace;
+import java.util.function.Supplier;
+
public interface ITracer {
enum Phase {
@@ -83,7 +85,12 @@
}
@Override
- public void durationE(long tid, long cat, String args) {
+ public void durationE(long cat, long tid, String args) {
+ // nothing to do here
+ }
+
+ @Override
+ public void instant(String name, long cat, Scope scope) {
// nothing to do here
}
@@ -91,6 +98,36 @@
public void instant(String name, long cat, Scope scope, String args) {
// nothing to do here
}
+
+ @Override
+ public long durationB(String name, long cat) {
+ return -1;
+ }
+
+ @Override
+ public long durationB(String name, long cat, Supplier<String> args) {
+ return -1;
+ }
+
+ @Override
+ public void durationE(long cat, long tid) {
+ // nothing to do here
+ }
+
+ @Override
+ public void durationE(long cat, long tid, Supplier<String> args) {
+ // nothing to do here
+ }
+
+ @Override
+ public void durationE(String name, long cat, long tid, Supplier<String> args) {
+ // nothing to do here
+ }
+
+ @Override
+ public void instant(String name, long cat, Scope scope, Supplier<String> args) {
+ // nothing to do here
+ }
};
String getName();
@@ -101,14 +138,28 @@
boolean isEnabled(long cat);
+ long durationB(String name, long cat);
+
long durationB(String name, long cat, String args);
- void durationE(long tid, long cat, String args);
+ long durationB(String name, long cat, Supplier<String> args);
+
+ void durationE(long cat, long tid);
+
+ void durationE(long cat, long tid, String args);
+
+ void durationE(long cat, long tid, Supplier<String> args);
void durationE(String name, long cat, long tid, String args);
+ void durationE(String name, long cat, long tid, Supplier<String> args);
+
+ void instant(String name, long cat, Scope scope);
+
void instant(String name, long cat, Scope scope, String args);
+ void instant(String name, long cat, Scope scope, Supplier<String> args);
+
@Override
String toString();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/Tracer.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/Tracer.java
index c1c38cf..bc24949 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/Tracer.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/trace/Tracer.java
@@ -23,6 +23,7 @@
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Date;
+import java.util.function.Supplier;
import org.apache.hyracks.util.PidHelper;
import org.apache.logging.log4j.Level;
@@ -44,7 +45,7 @@
private long categories;
private final TraceCategoryRegistry registry;
- private static final long pid = PidHelper.getPid();
+ private static final long PID = PidHelper.getPid();
public Tracer(String name, long categories, TraceCategoryRegistry registry) {
final String traceLoggerName = Tracer.class.getName() + ".Traces." + name;
@@ -53,7 +54,7 @@
this.categories = categories;
this.registry = registry;
final long traceCategory = getRegistry().get(TraceUtils.TRACER);
- instant("Trace-Start", traceCategory, Scope.p, dateTimeStamp());
+ instant("Trace-Start", traceCategory, Scope.p, Tracer::dateTimeStamp);
}
public Tracer(String name, String[] categories, TraceCategoryRegistry registry) {
@@ -100,11 +101,25 @@
}
@Override
+ public long durationB(String name, long cat) {
+ if (isEnabled(cat)) {
+ return emitDurationB(name, cat, null);
+ }
+ return -1;
+ }
+
+ @Override
public long durationB(String name, long cat, String args) {
if (isEnabled(cat)) {
- Event e = Event.create(name, cat, Phase.B, pid, Thread.currentThread().getId(), null, args, getRegistry());
- traceLog.log(TRACE_LOG_LEVEL, e.toJson());
- return e.tid;
+ return emitDurationB(name, cat, args);
+ }
+ return -1;
+ }
+
+ @Override
+ public long durationB(String name, long cat, Supplier<String> args) {
+ if (isEnabled(cat)) {
+ return emitDurationB(name, cat, args.get());
}
return -1;
}
@@ -112,24 +127,67 @@
@Override
public void durationE(String name, long cat, long tid, String args) {
if (isEnabled(cat)) {
- Event e = Event.create(name, cat, Phase.E, pid, tid, null, args, getRegistry());
- traceLog.log(TRACE_LOG_LEVEL, e.toJson());
+ emit(name, cat, null, Phase.E, tid, args);
}
}
@Override
- public void durationE(long tid, long cat, String args) {
+ public void durationE(String name, long cat, long tid, Supplier<String> args) {
if (isEnabled(cat)) {
- Event e = Event.create(null, 0L, Phase.E, pid, tid, null, args, getRegistry());
- traceLog.log(TRACE_LOG_LEVEL, e.toJson());
+ emit(name, cat, null, Phase.E, tid, args.get());
+ }
+ }
+
+ @Override
+ public void durationE(long cat, long tid) {
+ if (isEnabled(cat)) {
+ emit(null, 0L, null, Phase.E, tid, null);
+ }
+ }
+
+ @Override
+ public void durationE(long cat, long tid, String args) {
+ if (isEnabled(cat)) {
+ emit(null, 0L, null, Phase.E, tid, args);
+ }
+ }
+
+ @Override
+ public void durationE(long cat, long tid, Supplier<String> args) {
+ if (isEnabled(cat)) {
+ emit(null, 0L, null, Phase.E, tid, args.get());
+ }
+ }
+
+ @Override
+ public void instant(String name, long cat, Scope scope) {
+ if (isEnabled(cat)) {
+ emit(name, cat, scope, Phase.i, Thread.currentThread().getId(), null);
}
}
@Override
public void instant(String name, long cat, Scope scope, String args) {
if (isEnabled(cat)) {
- Event e = Event.create(name, cat, Phase.i, pid, Thread.currentThread().getId(), scope, args, getRegistry());
- traceLog.log(TRACE_LOG_LEVEL, e.toJson());
+ emit(name, cat, scope, Phase.i, Thread.currentThread().getId(), args);
}
}
+
+ @Override
+ public void instant(String name, long cat, Scope scope, Supplier<String> args) {
+ if (isEnabled(cat)) {
+ emit(name, cat, scope, Phase.i, Thread.currentThread().getId(), args.get());
+ }
+ }
+
+ private long emitDurationB(String name, long cat, String args) {
+ Event e = Event.create(name, cat, Phase.B, PID, Thread.currentThread().getId(), null, args, getRegistry());
+ traceLog.log(TRACE_LOG_LEVEL, e.toJson());
+ return e.tid;
+ }
+
+ private void emit(String name, long cat, Scope scope, Phase i, long tid, String args) {
+ Event e = Event.create(name, cat, i, PID, tid, scope, args, getRegistry());
+ traceLog.log(TRACE_LOG_LEVEL, e.toJson());
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/trace/TraceTest.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/trace/TraceTest.java
index f842f3e..9df05b5 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/trace/TraceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/trace/TraceTest.java
@@ -61,9 +61,9 @@
ITracer tracer = new Tracer(name, new String[] { "CAT1", "CAT2" }, registry);
Log4j2Monitor.start();
- tracer.instant("test1", cat1, ITracer.Scope.p, null);
- tracer.instant("test2", cat2, ITracer.Scope.p, null);
- tracer.instant("test3", cat3, ITracer.Scope.p, null);
+ tracer.instant("test1", cat1, ITracer.Scope.p);
+ tracer.instant("test2", cat2, ITracer.Scope.p);
+ tracer.instant("test3", cat3, ITracer.Scope.p);
List<String> lines = Log4j2Monitor.getLogs();
for (String line : lines) {
@@ -76,9 +76,9 @@
tracer.setCategories("CAT1", "CAT3");
Log4j2Monitor.reset();
- tracer.instant("test1", cat1, ITracer.Scope.p, null);
- tracer.instant("test2", cat2, ITracer.Scope.p, null);
- tracer.instant("test3", cat3, ITracer.Scope.p, null);
+ tracer.instant("test1", cat1, ITracer.Scope.p);
+ tracer.instant("test2", cat2, ITracer.Scope.p);
+ tracer.instant("test3", cat3, ITracer.Scope.p);
lines = Log4j2Monitor.getLogs();
for (String line : lines) {
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index d327f60..bb17d0f 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -70,9 +70,11 @@
<test.includes>${global.test.includes}</test.includes>
<test.excludes>${global.test.excludes}</test.excludes>
<!-- Versions under dependencymanagement or used in many projects via properties -->
- <hadoop.version>2.8.5</hadoop.version>
+ <hadoop.version>3.3.1</hadoop.version>
<jacoco.version>0.7.6.201602180812</jacoco.version>
- <log4j.version>2.14.1</log4j.version>
+ <log4j.version>2.17.1</log4j.version>
+ <snappy.version>1.1.8.4</snappy.version>
+ <jackson.version>2.13.1</jackson.version>
<implementation.title>Apache Hyracks and Algebricks - ${project.name}</implementation.title>
<implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -83,13 +85,53 @@
<dependencies>
<dependency>
<groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
- <version>4.1.63.Final</version>
+ <artifactId>netty-buffer</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-common</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-handler</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-resolver-dns</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http2</artifactId>
+ <version>4.1.73.Final</version>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-unix-common</artifactId>
+ <version>4.1.73.Final</version>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
- <version>4.13</version>
+ <version>4.13.2</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
@@ -107,6 +149,10 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
+ <groupId>com.nimbusds</groupId>
+ <artifactId>nimbus-jose-jwt</artifactId>
+ </exclusion>
+ <exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>jsp-api</artifactId>
</exclusion>
@@ -122,6 +168,14 @@
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
</exclusions>
</dependency>
<dependency>
@@ -130,6 +184,10 @@
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-all</artifactId>
+ </exclusion>
+ <exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
@@ -167,7 +225,7 @@
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
- <version>2.8.0</version>
+ <version>2.11.0</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
@@ -177,22 +235,22 @@
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
- <version>2.12.3</version>
+ <version>${jackson.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
- <version>2.12.3</version>
+ <version>${jackson.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
- <version>2.12.3</version>
+ <version>${jackson.version}</version>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
- <version>30.1-jre</version>
+ <version>31.0.1-jre</version>
<exclusions>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
@@ -229,7 +287,7 @@
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId>
- <version>4.4.14</version>
+ <version>4.4.15</version>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
@@ -279,37 +337,52 @@
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-model</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-artifact</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-compat</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-core</artifactId>
- <version>3.6.3</version>
- </dependency>
+ <version>3.8.4</version>
+ </dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-plugin-api</artifactId>
- <version>3.6.3</version>
+ <version>3.8.4</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.plugin-tools</groupId>
+ <artifactId>maven-plugin-annotations</artifactId>
+ <version>3.6.4</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-remote-resources-plugin</artifactId>
+ <version>1.7.0</version>
</dependency>
<dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil-core</artifactId>
- <version>8.5.4</version>
+ <version>8.5.6</version>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-simple</artifactId>
+ <version>1.7.33</version>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-1.2-api</artifactId>
- <version>2.13.1</version>
+ <version>${log4j.version}</version>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
@@ -331,6 +404,11 @@
<artifactId>powermock-core</artifactId>
<version>2.0.9</version>
</dependency>
+ <dependency>
+ <groupId>org.xerial.snappy</groupId>
+ <artifactId>snappy-java</artifactId>
+ <version>${snappy.version}</version>
+ </dependency>
</dependencies>
</dependencyManagement>
<build>