Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 4CF70200C0F for ; Thu, 2 Feb 2017 19:24:14 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 4BBCB160B65; Thu, 2 Feb 2017 18:24:14 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 17893160B44 for ; Thu, 2 Feb 2017 19:24:11 +0100 (CET) Received: (qmail 7785 invoked by uid 500); 2 Feb 2017 18:24:11 -0000 Mailing-List: contact commits-help@asterixdb.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@asterixdb.apache.org Delivered-To: mailing list commits@asterixdb.apache.org Received: (qmail 7524 invoked by uid 99); 2 Feb 2017 18:24:10 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 02 Feb 2017 18:24:10 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 548B1F17AE; Thu, 2 Feb 2017 18:24:10 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: buyingyi@apache.org To: commits@asterixdb.apache.org Date: Thu, 02 Feb 2017 18:24:19 -0000 Message-Id: <47a4c56d94ce46a79bbc51910ff25bd8@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [10/28] asterixdb git commit: Introduce IStorageComponentProvider archived-at: Thu, 02 Feb 2017 18:24:14 -0000 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 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 - splitProviderAndPartitionConstraintsForDataverse(String dataverse) { - FileSplit[] splits = splitsForDataverse(dataverse); + public static Pair getDataverseSplitProviderAndConstraints( + String dataverse) { + FileSplit[] splits = getDataverseSplits(dataverse); return StoragePathUtil.splitProviderAndPartitionConstraints(splits); } - public static Pair - splitProviderAndPartitionConstraintsForFilesIndex(MetadataTransactionContext mdTxnCtx, String dataverseName, - String datasetName, String targetIdxName, boolean create) throws AlgebricksException { - FileSplit[] splits = splitsForFilesIndex(mdTxnCtx, dataverseName, datasetName, targetIdxName, create); + public static Pair 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 createEnforcedType(ARecordType recordType, ARecordType metaType, + List 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> nestedTypeStack = new ArrayDeque<>(); + List 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 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 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 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 createRecordNameTypeMap(ARecordType recordType) { + LinkedHashMap 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 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 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 subFieldName) throws AsterixException { + public IAType getSubFieldType(List 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 getFieldTypes(List> fields) throws AlgebricksException { List typeList = new ArrayList<>(); for (List 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 unionTypes = new ArrayList(); - 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 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 unionTypes = new ArrayList(); + 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 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); + } +}