http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
index 24d185f..35e7acb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SplitsAndConstraintsUtil.java
@@ -29,7 +29,7 @@ import org.apache.asterix.metadata.MetadataException;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.runtime.util.ClusterStateManager;
+import org.apache.asterix.runtime.utils.ClusterStateManager;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -41,7 +41,7 @@ public class SplitsAndConstraintsUtil {
private SplitsAndConstraintsUtil() {
}
- private static FileSplit[] splitsForDataverse(String dataverseName) {
+ private static FileSplit[] getDataverseSplits(String dataverseName) {
File relPathFile = new File(dataverseName);
List<FileSplit> splits = new ArrayList<>();
// get all partitions
@@ -55,7 +55,7 @@ public class SplitsAndConstraintsUtil {
return splits.toArray(new FileSplit[] {});
}
- public static FileSplit[] splitsForDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
+ public static FileSplit[] getDatasetSplits(MetadataTransactionContext mdTxnCtx, String dataverseName,
String datasetName, String targetIdxName, boolean temp) throws AlgebricksException {
try {
File relPathFile =
@@ -92,7 +92,7 @@ public class SplitsAndConstraintsUtil {
}
}
- private static FileSplit[] splitsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName,
+ private static FileSplit[] getFilesIndexSplits(MetadataTransactionContext mdTxnCtx, String dataverseName,
String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
try {
File relPathFile =
@@ -131,16 +131,16 @@ public class SplitsAndConstraintsUtil {
}
}
- public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint>
- splitProviderAndPartitionConstraintsForDataverse(String dataverse) {
- FileSplit[] splits = splitsForDataverse(dataverse);
+ public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> getDataverseSplitProviderAndConstraints(
+ String dataverse) {
+ FileSplit[] splits = getDataverseSplits(dataverse);
return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
}
- public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint>
- splitProviderAndPartitionConstraintsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName,
- String datasetName, String targetIdxName, boolean create) throws AlgebricksException {
- FileSplit[] splits = splitsForFilesIndex(mdTxnCtx, dataverseName, datasetName, targetIdxName, create);
+ public static Pair<IFileSplitProvider, AlgebricksPartitionConstraint> getFilesIndexSplitProviderAndConstraints(
+ MetadataTransactionContext mdTxnCtx, String dataverseName, String datasetName, String targetIdxName,
+ boolean create) throws AlgebricksException {
+ FileSplit[] splits = getFilesIndexSplits(mdTxnCtx, dataverseName, datasetName, targetIdxName, create);
return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..d629940
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.utils;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.metadata.entities.Index;
+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.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * Provider utility methods for data types
+ */
+public class TypeUtil {
+
+ private TypeUtil() {
+ }
+
+ /**
+ * Merges typed index fields with specified recordType, allowing indexed fields to be optional.
+ * I.e. the type { "personId":int32, "name": string, "address" : { "street": string } } with typed indexes
+ * on age:int32, address.state:string will be merged into type { "personId":int32, "name": string,
+ * "age": int32? "address" : { "street": string, "state": string? } } Used by open indexes to enforce
+ * the type of an indexed record
+ */
+ public static Pair<ARecordType, ARecordType> createEnforcedType(ARecordType recordType, ARecordType metaType,
+ List<Index> indexes) throws AlgebricksException {
+ ARecordType enforcedRecordType = recordType;
+ ARecordType enforcedMetaType = metaType;
+ for (Index index : indexes) {
+ if (!index.isSecondaryIndex() || !index.isEnforcingKeyFileds()) {
+ continue;
+ }
+ if (index.hasMetaFields()) {
+ throw new AlgebricksException("Indexing an open field is only supported on the record part");
+ }
+ for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
+ Deque<Pair<ARecordType, String>> nestedTypeStack = new ArrayDeque<>();
+ List<String> splits = index.getKeyFieldNames().get(i);
+ ARecordType nestedFieldType = enforcedRecordType;
+ boolean openRecords = false;
+ String bridgeName = nestedFieldType.getTypeName();
+ int j;
+ // Build the stack for the enforced type
+ for (j = 1; j < splits.size(); j++) {
+ nestedTypeStack.push(new Pair<>(nestedFieldType, splits.get(j - 1)));
+ bridgeName = nestedFieldType.getTypeName();
+ nestedFieldType = (ARecordType) enforcedRecordType.getSubFieldType(splits.subList(0, j));
+ if (nestedFieldType == null) {
+ openRecords = true;
+ break;
+ }
+ }
+ if (openRecords) {
+ // create the smallest record
+ enforcedRecordType = new ARecordType(splits.get(splits.size() - 2),
+ new String[] { splits.get(splits.size() - 1) },
+ new IAType[] { AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)) }, true);
+ // create the open part of the nested field
+ for (int k = splits.size() - 3; k > (j - 2); k--) {
+ enforcedRecordType = new ARecordType(splits.get(k), new String[] { splits.get(k + 1) },
+ new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) }, true);
+ }
+ // Bridge the gap
+ Pair<ARecordType, String> gapPair = nestedTypeStack.pop();
+ ARecordType parent = gapPair.first;
+
+ IAType[] parentFieldTypes = ArrayUtils.addAll(parent.getFieldTypes().clone(),
+ new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) });
+ enforcedRecordType = new ARecordType(bridgeName,
+ ArrayUtils.addAll(parent.getFieldNames(), enforcedRecordType.getTypeName()),
+ parentFieldTypes, true);
+ } else {
+ //Schema is closed all the way to the field
+ //enforced fields are either null or strongly typed
+ Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(nestedFieldType);
+ // if a an enforced field already exists and the type is correct
+ IAType enforcedFieldType = recordNameTypesMap.get(splits.get(splits.size() - 1));
+ if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
+ && ((AUnionType) enforcedFieldType).isUnknownableType()) {
+ enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
+ }
+ if (enforcedFieldType != null && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(),
+ index.getKeyFieldTypes().get(i).getTypeTag())) {
+ throw new AlgebricksException("Cannot enforce field " + index.getKeyFieldNames().get(i)
+ + " to have type " + index.getKeyFieldTypes().get(i));
+ }
+ if (enforcedFieldType == null) {
+ recordNameTypesMap.put(splits.get(splits.size() - 1),
+ AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)));
+ }
+ enforcedRecordType = new ARecordType(nestedFieldType.getTypeName(),
+ recordNameTypesMap.keySet().toArray(new String[recordNameTypesMap.size()]),
+ recordNameTypesMap.values().toArray(new IAType[recordNameTypesMap.size()]),
+ nestedFieldType.isOpen());
+ }
+
+ // Create the enforced type for the nested fields in the schema, from the ground up
+ if (!nestedTypeStack.isEmpty()) {
+ while (!nestedTypeStack.isEmpty()) {
+ Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
+ ARecordType nestedRecType = nestedTypePair.first;
+ IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
+ nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedTypePair.second)] =
+ enforcedRecordType;
+ enforcedRecordType = new ARecordType(nestedRecType.getTypeName() + "_enforced",
+ nestedRecType.getFieldNames(), nestedRecTypeFieldTypes, nestedRecType.isOpen());
+ }
+ }
+ }
+ }
+ return new Pair<>(enforcedRecordType, enforcedMetaType);
+ }
+
+ /**
+ * Creates a map from name to type for fields in the passed type
+ *
+ * @param recordType
+ * the type to be mapped
+ * @return a map mapping all fields to their types
+ */
+ public static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
+ LinkedHashMap<String, IAType> recordNameTypesMap = new LinkedHashMap<>();
+ for (int j = 0; j < recordType.getFieldNames().length; j++) {
+ recordNameTypesMap.put(recordType.getFieldNames()[j], recordType.getFieldTypes()[j]);
+ }
+ return recordNameTypesMap;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java
index b3cb416..f64206e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/AbstractListBuilder.java
@@ -25,7 +25,7 @@ import java.io.IOException;
import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AbstractCollectionType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IValueReference;
import org.apache.hyracks.data.std.util.GrowableArray;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 44181d5..797901e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -29,7 +29,7 @@ import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUt
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
import org.apache.hyracks.api.exceptions.HyracksDataException;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
index ace692f..90af41b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
@@ -22,7 +22,7 @@ import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
index 6be61d6..437958c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
@@ -34,7 +34,7 @@ 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.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 1d3a420..75aa61d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -39,7 +39,8 @@ 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.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
@@ -196,7 +197,7 @@ public class ARecordSerializerDeserializer implements ISerializerDeserializer<AR
ISerializerDeserializer<AString> stringSerde = SerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ASTRING);
RecordBuilder confRecordBuilder = new RecordBuilder();
- confRecordBuilder.reset(ARecordType.FULLY_OPEN_RECORD_TYPE);
+ confRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
ArrayBackedValueStorage fieldNameBytes = new ArrayBackedValueStorage();
ArrayBackedValueStorage fieldValueBytes = new ArrayBackedValueStorage();
for (int i = 0; i < record.getType().getFieldNames().length; i++) {
@@ -218,7 +219,7 @@ public class ARecordSerializerDeserializer implements ISerializerDeserializer<AR
ISerializerDeserializer<AString> stringSerde = SerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ASTRING);
RecordBuilder confRecordBuilder = new RecordBuilder();
- confRecordBuilder.reset(ARecordType.FULLY_OPEN_RECORD_TYPE);
+ confRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
ArrayBackedValueStorage fieldNameBytes = new ArrayBackedValueStorage();
ArrayBackedValueStorage fieldValueBytes = new ArrayBackedValueStorage();
for (int i = 0; i < record.size(); i++) {
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
index fad023f..02b662e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AUnorderedListSerializerDeserializer.java
@@ -34,7 +34,7 @@ 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.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
index cda4d2b..147ef6d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/ARecord.java
@@ -21,13 +21,14 @@ package org.apache.asterix.om.base;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.utils.RecordUtil;
import org.apache.asterix.om.visitors.IOMVisitor;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
public class ARecord implements IAObject {
- public static final ARecord EMPTY_OPEN_RECORD = new ARecord(ARecordType.FULLY_OPEN_RECORD_TYPE, new IAObject[] {});
+ public static final ARecord EMPTY_OPEN_RECORD = new ARecord(RecordUtil.FULLY_OPEN_RECORD_TYPE, new IAObject[] {});
protected ARecordType type;
protected IAObject[] fields;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
index 2f7d9af..ec9bf48 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/AListVisitablePointable.java
@@ -32,9 +32,9 @@ import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
/**
* This class interprets the binary data representation of a list, one can
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
index d3c18a9..c5a0ee5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/ARecordVisitablePointable.java
@@ -33,9 +33,9 @@ import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
import org.apache.hyracks.util.string.UTF8StringWriter;
/**
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
index 7115827..94cfe75 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/AListCaster.java
@@ -35,7 +35,7 @@ import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
import org.apache.hyracks.algebricks.common.utils.Triple;
/**
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
index e0deb5a..c2aa3d8 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ARecordCaster.java
@@ -42,8 +42,8 @@ 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.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
-import org.apache.asterix.om.util.ResettableByteArrayOutputStream;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.ResettableByteArrayOutputStream;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
index 336aaac..f2e7299 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AListPointable.java
@@ -26,8 +26,8 @@ import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AbstractCollectionType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.data.std.api.AbstractPointable;
import org.apache.hyracks.data.std.api.IPointable;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
index 08cde54..2fae304 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
@@ -28,8 +28,9 @@ import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.data.std.api.AbstractPointable;
import org.apache.hyracks.data.std.api.IPointable;
@@ -190,7 +191,7 @@ public class ARecordPointable extends AbstractPointable {
}
public int getNullBitmapSize(ARecordType recordType) {
- return ARecordType.computeNullBitmapSize(recordType);
+ return RecordUtil.computeNullBitmapSize(recordType);
}
public boolean isClosedFieldNull(ARecordType recordType, int fieldId) {
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
index 0e351e2..b328cf4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ClosedRecordConstructorResultType.java
@@ -28,7 +28,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.IAType;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+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;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
index 0c9d538..0e04480 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByIndexResultType.java
@@ -24,7 +24,7 @@ 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.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
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.AbstractFunctionCallExpression;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
index deb667c..ffa3029 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/FieldAccessByNameResultType.java
@@ -24,7 +24,7 @@ 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.util.ConstantExpressionUtil;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
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.AbstractFunctionCallExpression;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/OpenRecordConstructorResultType.java
----------------------------------------------------------------------
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 baee36a..1442dfe 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
@@ -32,7 +32,7 @@ import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeHelper;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+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;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
index 8c643dc..cb6d90b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddFieldsTypeComputer.java
@@ -36,7 +36,7 @@ import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeHelper;
-import org.apache.asterix.om.util.ConstantExpressionUtil;
+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;
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
index 823f51e..9110c12 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TypeComputeUtils.java
@@ -25,6 +25,7 @@ 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.utils.RecordUtil;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -197,7 +198,7 @@ public class TypeComputeUtils {
return null;
}
case ANY:
- return ARecordType.FULLY_OPEN_RECORD_TYPE;
+ return RecordUtil.FULLY_OPEN_RECORD_TYPE;
default:
return null;
}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
----------------------------------------------------------------------
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 4ce1a6d..0196e94 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
@@ -26,13 +26,13 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
-import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.util.NonTaggedFormatUtil;
import org.apache.asterix.om.visitors.IOMVisitor;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
@@ -44,9 +44,6 @@ import com.fasterxml.jackson.databind.node.ObjectNode;
*/
public class ARecordType extends AbstractComplexType {
- public static final ARecordType FULLY_OPEN_RECORD_TYPE = new ARecordType("OpenRecord", new String[0], new IAType[0],
- true);
-
private static final long serialVersionUID = 1L;
private final String[] fieldNames;
private final IAType[] fieldTypes;
@@ -209,7 +206,7 @@ public class ARecordType extends AbstractComplexType {
* @return the type of the child
* @throws AsterixException
*/
- public IAType getSubFieldType(List<String> subFieldName) throws AsterixException {
+ public IAType getSubFieldType(List<String> subFieldName) throws AlgebricksException {
IAType subRecordType = getFieldType(subFieldName.get(0));
for (int i = 1; i < subFieldName.size(); i++) {
if (subRecordType == null) {
@@ -327,7 +324,7 @@ public class ARecordType extends AbstractComplexType {
}
@Override
- public ObjectNode toJSON() {
+ public ObjectNode toJSON() {
ObjectMapper om = new ObjectMapper();
ObjectNode type = om.createObjectNode();
type.put("type", ARecordType.class.getName());
@@ -349,10 +346,6 @@ public class ARecordType extends AbstractComplexType {
return type;
}
- public static int computeNullBitmapSize(ARecordType rt) {
- return NonTaggedFormatUtil.hasOptionalField(rt) ? (int) Math.ceil(rt.getFieldNames().length / 4.0) : 0;
- }
-
public List<IAType> getFieldTypes(List<List<String>> fields) throws AlgebricksException {
List<IAType> typeList = new ArrayList<>();
for (List<String> field : fields) {
@@ -370,14 +363,4 @@ public class ARecordType extends AbstractComplexType {
}
return false;
}
-
- /**
- * Create a fully open record type with the passed name
- *
- * @param name
- * @return
- */
- public static ARecordType createOpenRecordType(String name) {
- return new ARecordType(name, new String[0], new IAType[0], true);
- }
}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
index b6c5712..1b0beab 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
@@ -19,6 +19,7 @@
package org.apache.asterix.om.types;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.om.utils.RecordUtil;
public class TypeTagUtil {
@@ -81,7 +82,7 @@ public class TypeTagUtil {
case UUID:
return BuiltinType.AUUID;
case RECORD:
- return ARecordType.FULLY_OPEN_RECORD_TYPE;
+ return RecordUtil.FULLY_OPEN_RECORD_TYPE;
case UNORDEREDLIST:
return AUnorderedListType.FULLY_OPEN_UNORDEREDLIST_TYPE;
case ORDEREDLIST:
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ConstantExpressionUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ConstantExpressionUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ConstantExpressionUtil.java
deleted file mode 100644
index 406f356..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ConstantExpressionUtil.java
+++ /dev/null
@@ -1,114 +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.om.util;
-
-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.AOrderedList;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.base.AUnorderedList;
-import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.constants.AsterixConstantValue;
-import org.apache.asterix.om.types.ATypeTag;
-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.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
-
-public class ConstantExpressionUtil {
-
- private ConstantExpressionUtil() {
- }
-
- public static IAObject getConstantIaObject(ILogicalExpression expr, ATypeTag typeTag) {
- if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- return null;
- }
- final IAlgebricksConstantValue acv = ((ConstantExpression) expr).getValue();
- if (!(acv instanceof AsterixConstantValue)) {
- return null;
- }
- final IAObject iaObject = ((AsterixConstantValue) acv).getObject();
- if (typeTag != null) {
- return iaObject.getType().getTypeTag() == typeTag ? iaObject : null;
- } else {
- return iaObject;
- }
- }
-
- public static ATypeTag getConstantIaObjectType(ILogicalExpression expr) {
- IAObject iaObject = getConstantIaObject(expr, null);
- return iaObject.getType().getTypeTag();
- }
-
- public static Long getLongConstant(ILogicalExpression expr) {
- final IAObject iaObject = getConstantIaObject(expr, ATypeTag.INT64);
- return iaObject != null ? ((AInt64) iaObject).getLongValue() : null;
- }
-
- public static Integer getIntConstant(ILogicalExpression expr) {
- final IAObject iaObject = getConstantIaObject(expr, ATypeTag.INT32);
- return iaObject != null ? ((AInt32) iaObject).getIntegerValue() : null;
- }
-
- public static String getStringConstant(ILogicalExpression expr) {
- final IAObject iaObject = getConstantIaObject(expr, ATypeTag.STRING);
- return iaObject != null ? ((AString) iaObject).getStringValue() : null;
- }
-
- public static String getStringConstant(IAObject iaObject) {
- // Make sure to call this method after checking the type of the given object.
- return iaObject != null ? ((AString) iaObject).getStringValue() : null;
- }
-
- public static AOrderedList getOrderedListConstant(IAObject iaObject) {
- // Make sure to call this method after checking the type of the given object.
- return iaObject != null ? (AOrderedList) iaObject : null;
- }
-
- public static AUnorderedList getUnorderedListConstant(IAObject iaObject) {
- // Make sure to call this method after checking the type of the given object.
- return iaObject != null ? (AUnorderedList) iaObject : null;
- }
-
- public static Boolean getBooleanConstant(ILogicalExpression expr) {
- final IAObject iaObject = getConstantIaObject(expr, ATypeTag.BOOLEAN);
- return iaObject != null ? ((ABoolean) iaObject).getBoolean() : null;
- }
-
- public static Integer getIntArgument(AbstractFunctionCallExpression f, int index) {
- return getIntConstant(f.getArguments().get(index).getValue());
- }
-
- public static String getStringArgument(AbstractFunctionCallExpression f, int index) {
- return getStringConstant(f.getArguments().get(index).getValue());
- }
-
- public static Integer getIntArgument(ILogicalExpression expr, int index) {
- return expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
- ? getIntArgument((AbstractFunctionCallExpression) expr, index) : null;
- }
-
- public static String getStringArgument(ILogicalExpression expr, int index) {
- return expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
- ? getStringArgument((AbstractFunctionCallExpression) expr, index) : null;
- }
-}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/JSONDeserializerForTypes.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/JSONDeserializerForTypes.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/JSONDeserializerForTypes.java
deleted file mode 100644
index 8e8993a..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/JSONDeserializerForTypes.java
+++ /dev/null
@@ -1,95 +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.om.util;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
-import com.google.common.collect.Lists;
-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;
-
-public class JSONDeserializerForTypes {
-
- /**
- * Deserialize an arbitrary JSON representation of a type.
- *
- * @param typeInJSON
- * the JSON representation of the type.
- * @return an valid AsterixDB type.
- * @throws Exception
- */
- public static IAType convertFromJSON(JsonNode typeInJSON) throws Exception {
- String typeName = typeInJSON.get("type").asText();
- // Deals with ordered list.
- if (typeName.equals(AOrderedListType.class.getName())) {
- IAType itemType = convertFromJSON(typeInJSON.get("item-type"));
- return new AOrderedListType(itemType, "ordered-list");
- }
-
- // Deals with unordered list.
- if (typeName.equals(AUnorderedListType.class.getName())) {
- IAType itemType = convertFromJSON(typeInJSON.get("item-type"));
- return new AUnorderedListType(itemType, "unordered-list");
- }
-
- // Deals with Union Type.
- if (typeName.equals(AUnionType.class.getName())) {
- List<IAType> unionTypes = new ArrayList<IAType>();
- JsonNode fields = typeInJSON.get("fields");
- for (int i = 0; i < fields.size(); i++) {
- JsonNode fieldType = fields.get(i);
- unionTypes.add(convertFromJSON(fieldType));
- }
- return new AUnionType(unionTypes, "union");
- }
-
- // Deals with record types.
- if (typeName.equals(ARecordType.class.getName())) {
- String name = typeInJSON.get("name").asText();
- boolean openType = typeInJSON.get("open").asBoolean();
- JsonNode fields = typeInJSON.get("fields");
- String[] fieldNames = new String[fields.size()];
- IAType[] fieldTypes = new IAType[fields.size()];
- for (int i = 0; i < fields.size(); ++i) {
- JsonNode field = fields.get(i);
- List<String> names = Lists.newArrayList(field.fieldNames());
- String fieldName = names.get(0);
- fieldNames[i] = fieldName;
- fieldTypes[i] = convertFromJSON(field.get(fieldName));
- }
- return new ARecordType(name, fieldNames, fieldTypes, openType);
- }
-
- // Deals with primitive types.
- Class<?> cl = BuiltinType.class;
- Field typeField = cl.getDeclaredField(typeName.toUpperCase());
- return (IAType) typeField.get(null);
- }
-}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/NonTaggedFormatUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/NonTaggedFormatUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/NonTaggedFormatUtil.java
deleted file mode 100644
index 0608b79..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/NonTaggedFormatUtil.java
+++ /dev/null
@@ -1,272 +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.om.util;
-
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
-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.formats.nontagged.BinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.BinaryTokenizerFactoryProvider;
-import org.apache.asterix.formats.nontagged.TypeTraitProvider;
-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.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import org.apache.hyracks.util.string.UTF8StringUtil;
-
-public final class NonTaggedFormatUtil {
-
- public static final boolean isFixedSizedCollection(IAType type) {
- switch (type.getTypeTag()) {
- case UNION:
- if (!((AUnionType) type).isUnknownableType()) {
- return false;
- } else {
- return isFixedSizedCollection(((AUnionType) type).getActualType());
- }
- default:
- return isFixedSizedCollection(type.getTypeTag());
- }
- }
-
- public static final boolean isFixedSizedCollection(ATypeTag type) {
- switch (type) {
- case STRING:
- case BINARY:
- case RECORD:
- case INTERVAL:
- case ORDEREDLIST:
- case UNORDEREDLIST:
- case POLYGON:
- case ANY:
- return false;
- default:
- return true;
- }
- }
-
- public static final boolean hasOptionalField(ARecordType recType) {
- for (int i = 0; i < recType.getFieldTypes().length; i++) {
- IAType type = recType.getFieldTypes()[i];
- if (type != null) {
- ATypeTag tag = type.getTypeTag();
- if (tag == ATypeTag.NULL || tag == ATypeTag.MISSING) {
- return true;
- }
- if (tag != ATypeTag.UNION) {
- continue;
- }
- // union
- AUnionType unionType = (AUnionType) type;
- if (unionType.isUnknownableType()) {
- return true;
- }
- }
- }
- return false;
- }
-
- /**
- * Decide whether a type is an optional type
- *
- * @param type
- * @return true if it is optional; false otherwise
- */
- public static boolean isOptional(IAType type) {
- return type.getTypeTag() == ATypeTag.UNION && ((AUnionType) type).isUnknownableType();
- }
-
- public static int getFieldValueLength(byte[] serNonTaggedAObject, int offset, ATypeTag typeTag, boolean tagged)
- throws AsterixException {
- switch (typeTag) {
- case ANY:
- ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serNonTaggedAObject[offset]);
- if (tag == ATypeTag.ANY) {
- throw new AsterixException("Field value has type tag ANY, but it should have a concrete type.");
- }
- return getFieldValueLength(serNonTaggedAObject, offset, tag, true) + 1;
- case MISSING:
- case NULL:
- return 0;
- case BOOLEAN:
- case INT8:
- return 1;
- case INT16:
- return 2;
- case INT32:
- case FLOAT:
- case DATE:
- case YEARMONTHDURATION:
- return 4;
- case TIME:
- return 4;
- case INT64:
- case DOUBLE:
- case DATETIME:
- case DAYTIMEDURATION:
- return 8;
- case DURATION:
- return 12;
- case POINT:
- case UUID:
- return 16;
- case INTERVAL:
- if (tagged) {
- return AIntervalSerializerDeserializer.getIntervalLength(serNonTaggedAObject, offset + 1);
- } else {
- return AIntervalSerializerDeserializer.getIntervalLength(serNonTaggedAObject, offset);
- }
- case POINT3D:
- case CIRCLE:
- return 24;
- case LINE:
- case RECTANGLE:
- return 32;
- case POLYGON:
- if (tagged) {
- return AInt16SerializerDeserializer.getShort(serNonTaggedAObject, offset + 1) * 16 + 2;
- } else {
- return AInt16SerializerDeserializer.getShort(serNonTaggedAObject, offset) * 16 + 2;
- }
- case STRING:
- if (tagged) {
- int len = UTF8StringUtil.getUTFLength(serNonTaggedAObject, offset + 1);
- return len + UTF8StringUtil.getNumBytesToStoreLength(len);
- } else {
- int len = UTF8StringUtil.getUTFLength(serNonTaggedAObject, offset);
- return len + UTF8StringUtil.getNumBytesToStoreLength(len);
- }
- case BINARY:
- if (tagged) {
- int len = ByteArrayPointable.getContentLength(serNonTaggedAObject, offset + 1);
- return len + ByteArrayPointable.getNumberBytesToStoreMeta(len);
- } else {
- int len = ByteArrayPointable.getContentLength(serNonTaggedAObject, offset);
- return len + ByteArrayPointable.getNumberBytesToStoreMeta(len);
- }
- case RECORD:
- if (tagged) {
- return ARecordSerializerDeserializer.getRecordLength(serNonTaggedAObject, offset + 1) - 1;
- } else {
- return ARecordSerializerDeserializer.getRecordLength(serNonTaggedAObject, offset) - 1;
- }
- case ORDEREDLIST:
- if (tagged) {
- return AOrderedListSerializerDeserializer.getOrderedListLength(serNonTaggedAObject, offset + 1) - 1;
- } else {
- return AOrderedListSerializerDeserializer.getOrderedListLength(serNonTaggedAObject, offset) - 1;
- }
- case UNORDEREDLIST:
- if (tagged) {
- return AUnorderedListSerializerDeserializer.getUnorderedListLength(serNonTaggedAObject, offset + 1)
- - 1;
- } else {
- return AUnorderedListSerializerDeserializer.getUnorderedListLength(serNonTaggedAObject, offset) - 1;
- }
- default:
- throw new NotImplementedException(
- "No getLength implemented for a value of this type " + typeTag + " .");
- }
- }
-
- public static int getNumDimensions(ATypeTag typeTag) {
- switch (typeTag) {
- case POINT:
- case LINE:
- case POLYGON:
- case CIRCLE:
- case RECTANGLE:
- return 2;
- case POINT3D:
- return 3;
- default:
- throw new NotImplementedException(
- "getNumDimensions is not implemented for this type " + typeTag + " .");
- }
- }
-
- public static IAType getNestedSpatialType(ATypeTag typeTag) {
- switch (typeTag) {
- case POINT:
- case LINE:
- case POLYGON:
- case CIRCLE:
- case RECTANGLE:
- return BuiltinType.ADOUBLE;
- default:
- throw new NotImplementedException(typeTag + " is not a supported spatial data type.");
- }
- }
-
- public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
- int gramLength) throws AlgebricksException {
- switch (indexType) {
- case SINGLE_PARTITION_WORD_INVIX:
- case LENGTH_PARTITIONED_WORD_INVIX: {
- return BinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(keyType, false, false);
- }
- case SINGLE_PARTITION_NGRAM_INVIX:
- case LENGTH_PARTITIONED_NGRAM_INVIX: {
- return BinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType, gramLength, true,
- false);
- }
- default: {
- throw new AlgebricksException("Tokenizer not applicable to index type '" + indexType + "'.");
- }
- }
- }
-
- public static IAType getTokenType(IAType keyType) throws AlgebricksException {
- IAType type = keyType;
- ATypeTag typeTag = keyType.getTypeTag();
- // Extract item type from list.
- if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
- AbstractCollectionType listType = (AbstractCollectionType) keyType;
- if (!listType.isTyped()) {
- throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
- }
- type = listType.getItemType();
- }
- return type;
- }
-
- public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
- IAType type = getTokenType(keyType);
- // Ignore case for string types.
- return BinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
- }
-
- public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
- IAType type = getTokenType(keyType);
- return TypeTraitProvider.INSTANCE.getTypeTrait(type);
- }
-}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ResettableByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ResettableByteArrayOutputStream.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ResettableByteArrayOutputStream.java
deleted file mode 100644
index a40005f..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/util/ResettableByteArrayOutputStream.java
+++ /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.
- */
-package org.apache.asterix.om.util;
-
-import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-
-/**
- * This class extends ByteArrayAccessibleOutputStream to allow reset to a given
- * size.
- */
-public class ResettableByteArrayOutputStream extends ByteArrayAccessibleOutputStream {
-
- public void reset(int size) {
- count = size;
- }
-}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ConstantExpressionUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ConstantExpressionUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ConstantExpressionUtil.java
new file mode 100644
index 0000000..28b3da2
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/ConstantExpressionUtil.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.om.utils;
+
+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.AOrderedList;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.AUnorderedList;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.types.ATypeTag;
+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.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+
+public class ConstantExpressionUtil {
+
+ private ConstantExpressionUtil() {
+ }
+
+ public static IAObject getConstantIaObject(ILogicalExpression expr, ATypeTag typeTag) {
+ if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return null;
+ }
+ final IAlgebricksConstantValue acv = ((ConstantExpression) expr).getValue();
+ if (!(acv instanceof AsterixConstantValue)) {
+ return null;
+ }
+ final IAObject iaObject = ((AsterixConstantValue) acv).getObject();
+ if (typeTag != null) {
+ return iaObject.getType().getTypeTag() == typeTag ? iaObject : null;
+ } else {
+ return iaObject;
+ }
+ }
+
+ public static ATypeTag getConstantIaObjectType(ILogicalExpression expr) {
+ IAObject iaObject = getConstantIaObject(expr, null);
+ return iaObject.getType().getTypeTag();
+ }
+
+ public static Long getLongConstant(ILogicalExpression expr) {
+ final IAObject iaObject = getConstantIaObject(expr, ATypeTag.INT64);
+ return iaObject != null ? ((AInt64) iaObject).getLongValue() : null;
+ }
+
+ public static Integer getIntConstant(ILogicalExpression expr) {
+ final IAObject iaObject = getConstantIaObject(expr, ATypeTag.INT32);
+ return iaObject != null ? ((AInt32) iaObject).getIntegerValue() : null;
+ }
+
+ public static String getStringConstant(ILogicalExpression expr) {
+ final IAObject iaObject = getConstantIaObject(expr, ATypeTag.STRING);
+ return iaObject != null ? ((AString) iaObject).getStringValue() : null;
+ }
+
+ public static String getStringConstant(IAObject iaObject) {
+ // Make sure to call this method after checking the type of the given object.
+ return iaObject != null ? ((AString) iaObject).getStringValue() : null;
+ }
+
+ public static AOrderedList getOrderedListConstant(IAObject iaObject) {
+ // Make sure to call this method after checking the type of the given object.
+ return iaObject != null ? (AOrderedList) iaObject : null;
+ }
+
+ public static AUnorderedList getUnorderedListConstant(IAObject iaObject) {
+ // Make sure to call this method after checking the type of the given object.
+ return iaObject != null ? (AUnorderedList) iaObject : null;
+ }
+
+ public static Boolean getBooleanConstant(ILogicalExpression expr) {
+ final IAObject iaObject = getConstantIaObject(expr, ATypeTag.BOOLEAN);
+ return iaObject != null ? ((ABoolean) iaObject).getBoolean() : null;
+ }
+
+ public static Integer getIntArgument(AbstractFunctionCallExpression f, int index) {
+ return getIntConstant(f.getArguments().get(index).getValue());
+ }
+
+ public static String getStringArgument(AbstractFunctionCallExpression f, int index) {
+ return getStringConstant(f.getArguments().get(index).getValue());
+ }
+
+ public static Integer getIntArgument(ILogicalExpression expr, int index) {
+ return expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+ ? getIntArgument((AbstractFunctionCallExpression) expr, index) : null;
+ }
+
+ public static String getStringArgument(ILogicalExpression expr, int index) {
+ return expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+ ? getStringArgument((AbstractFunctionCallExpression) expr, index) : null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/ef9be0f9/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.java
new file mode 100644
index 0000000..069e47b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/JSONDeserializerForTypes.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.om.utils;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+import com.google.common.collect.Lists;
+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;
+
+public class JSONDeserializerForTypes {
+
+ /**
+ * Deserialize an arbitrary JSON representation of a type.
+ *
+ * @param typeInJSON
+ * the JSON representation of the type.
+ * @return an valid AsterixDB type.
+ * @throws Exception
+ */
+ public static IAType convertFromJSON(JsonNode typeInJSON) throws Exception {
+ String typeName = typeInJSON.get("type").asText();
+ // Deals with ordered list.
+ if (typeName.equals(AOrderedListType.class.getName())) {
+ IAType itemType = convertFromJSON(typeInJSON.get("item-type"));
+ return new AOrderedListType(itemType, "ordered-list");
+ }
+
+ // Deals with unordered list.
+ if (typeName.equals(AUnorderedListType.class.getName())) {
+ IAType itemType = convertFromJSON(typeInJSON.get("item-type"));
+ return new AUnorderedListType(itemType, "unordered-list");
+ }
+
+ // Deals with Union Type.
+ if (typeName.equals(AUnionType.class.getName())) {
+ List<IAType> unionTypes = new ArrayList<IAType>();
+ JsonNode fields = typeInJSON.get("fields");
+ for (int i = 0; i < fields.size(); i++) {
+ JsonNode fieldType = fields.get(i);
+ unionTypes.add(convertFromJSON(fieldType));
+ }
+ return new AUnionType(unionTypes, "union");
+ }
+
+ // Deals with record types.
+ if (typeName.equals(ARecordType.class.getName())) {
+ String name = typeInJSON.get("name").asText();
+ boolean openType = typeInJSON.get("open").asBoolean();
+ JsonNode fields = typeInJSON.get("fields");
+ String[] fieldNames = new String[fields.size()];
+ IAType[] fieldTypes = new IAType[fields.size()];
+ for (int i = 0; i < fields.size(); ++i) {
+ JsonNode field = fields.get(i);
+ List<String> names = Lists.newArrayList(field.fieldNames());
+ String fieldName = names.get(0);
+ fieldNames[i] = fieldName;
+ fieldTypes[i] = convertFromJSON(field.get(fieldName));
+ }
+ return new ARecordType(name, fieldNames, fieldTypes, openType);
+ }
+
+ // Deals with primitive types.
+ Class<?> cl = BuiltinType.class;
+ Field typeField = cl.getDeclaredField(typeName.toUpperCase());
+ return (IAType) typeField.get(null);
+ }
+}
|