You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org> on 2015/11/18 07:33:23 UTC

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Yingyi Bu has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/501

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................

ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.

Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
---
M asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
M asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
A asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
19 files changed, 311 insertions(+), 267 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/01/501/1

diff --git a/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index c90a52a..0d7955d 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -20,18 +20,16 @@
 
 import java.util.List;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
 import org.apache.asterix.formats.base.IDataFormat;
-import org.apache.asterix.metadata.declared.AqlMetadataProvider;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.ComparisonEvalFactory;
 import org.apache.asterix.runtime.formats.FormatUtils;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
@@ -65,7 +63,7 @@
     @Override
     public ICopyAggregateFunctionFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
         switch (fd.getFunctionDescriptorTag()) {
@@ -74,9 +72,9 @@
             case AGGREGATE:
                 return fd.createAggregateFunctionFactory(args);
             default:
-                throw new IllegalStateException("Invalid function descriptor " + fd.getFunctionDescriptorTag()
-                        + " expected " + FunctionDescriptorTag.SERIALAGGREGATE + " or "
-                        + FunctionDescriptorTag.AGGREGATE);
+                throw new IllegalStateException(
+                        "Invalid function descriptor " + fd.getFunctionDescriptorTag() + " expected "
+                                + FunctionDescriptorTag.SERIALAGGREGATE + " or " + FunctionDescriptorTag.AGGREGATE);
         }
     }
 
@@ -91,7 +89,7 @@
     @Override
     public ICopyUnnestingFunctionFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         return getFunctionDescriptor(expr, env, context).createUnnestingFunctionFactory(args);
     }
@@ -136,7 +134,7 @@
 
     private ICopyEvaluatorFactory createScalarFunctionEvaluatorFactory(AbstractFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         FunctionIdentifier fi = expr.getFunctionIdentifier();
         ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(fi);
@@ -146,13 +144,12 @@
 
         IFunctionDescriptor fd = null;
         if (!(expr.getFunctionInfo() instanceof IExternalFunctionInfo)) {
-            AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
             IDataFormat format = FormatUtils.getDefaultFormat();
             fd = format.resolveFunction(expr, env);
         } else {
             try {
-                fd = ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor((IExternalFunctionInfo) expr
-                        .getFunctionInfo());
+                fd = ExternalFunctionDescriptorProvider
+                        .getExternalFunctionDescriptor((IExternalFunctionInfo) expr.getFunctionInfo());
             } catch (AsterixException ae) {
                 throw new AlgebricksException(ae);
             }
@@ -162,7 +159,6 @@
 
     private ICopyEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr,
             IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         IDataFormat format = FormatUtils.getDefaultFormat();
         return format.getConstantEvalFactory(expr.getValue());
     }
@@ -204,17 +200,15 @@
             }
 
             default:
-                throw new IllegalStateException("Invalid function descriptor " + fd.getFunctionDescriptorTag()
-                        + " expected " + FunctionDescriptorTag.SERIALAGGREGATE + " or "
-                        + FunctionDescriptorTag.AGGREGATE);
+                throw new IllegalStateException(
+                        "Invalid function descriptor " + fd.getFunctionDescriptorTag() + " expected "
+                                + FunctionDescriptorTag.SERIALAGGREGATE + " or " + FunctionDescriptorTag.AGGREGATE);
         }
     }
 
-    private IFunctionDescriptor getFunctionDescriptor(AbstractFunctionCallExpression expr,
-            IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
-        IFunctionDescriptor fd;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        fd = FormatUtils.getDefaultFormat().resolveFunction(expr, env);
+    private IFunctionDescriptor getFunctionDescriptor(AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,
+            JobGenContext context) throws AlgebricksException {
+        IFunctionDescriptor fd = FormatUtils.getDefaultFormat().resolveFunction(expr, env);
         return fd;
     }
 
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
index 8cdd78b..20bfea5 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
@@ -151,7 +151,7 @@
         if (s == null) {
             return null;
         }
-        int k = recType.findFieldPosition(s);
+        int k = recType.getFieldIndex(s);
         if (k < 0) {
             return null;
         }
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 5def2f8..f51d454 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -196,7 +196,7 @@
                         .getValue()).getValue()).getObject()).getStringValue();
                 int k;
                 try {
-                    k = rt.findFieldPosition(str);
+                    k = rt.getFieldIndex(str);
                 } catch (IOException e) {
                     throw new AlgebricksException(e);
                 }
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index ee30063..f5ef424 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -499,7 +499,7 @@
                         Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
                         ARecordType nestedRecType = nestedTypePair.first;
                         IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
-                        nestedRecTypeFieldTypes[nestedRecType.findFieldPosition(nestedTypePair.second)] = enforcedType;
+                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedTypePair.second)] = enforcedType;
                         enforcedType = new ARecordType(nestedRecType.getTypeName(), nestedRecType.getFieldNames(),
                                 nestedRecTypeFieldTypes, nestedRecType.isOpen());
                     }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 095adbf..442cfdc 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -2912,12 +2912,11 @@
                     <output-dir compare="Text">btree-secondary-equi-join</output-dir>
                 </compilation-unit>
             </test-case>
-            <!-- ASTERIXDB-1159 <test-case FilePath="nested-open-index/index-join">
+            <test-case FilePath="nested-open-index/index-join">
                 <compilation-unit name="ngram-edit-distance">
                     <output-dir compare="Text">ngram-edit-distance</output-dir>
                 </compilation-unit>
             </test-case>
-            -->
             <!--  <test-case FilePath="nested-open-index/index-join">
                 <compilation-unit name="ngram-edit-distance-inline">
                     <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 9cc021f..10bd02f 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -170,7 +170,7 @@
 
                 // Check if there is a filter field.
                 List<String> filterField = null;
-                int filterFieldPos = datasetDetailsRecord.getType().findFieldPosition(
+                int filterFieldPos = datasetDetailsRecord.getType().getFieldIndex(
                         InternalDatasetDetails.FILTER_FIELD_NAME);
                 if (filterFieldPos >= 0) {
                     filterField = new ArrayList<String>();
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index ee172cc..061030d 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -122,7 +122,7 @@
             }
             searchKey.add(nestedFieldName);
         }
-        int indexKeyTypeFieldPos = rec.getType().findFieldPosition(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
+        int indexKeyTypeFieldPos = rec.getType().getFieldIndex(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
         IACursor fieldTypeCursor = new ACollectionCursor();
         if (indexKeyTypeFieldPos > 0)
             fieldTypeCursor = ((AOrderedList) rec.getValueByPos(indexKeyTypeFieldPos)).getCursor();
@@ -141,7 +141,7 @@
                 searchKeyType.add(fieldType);
             }
         }
-        int isEnforcedFieldPos = rec.getType().findFieldPosition(INDEX_ISENFORCED_FIELD_NAME);
+        int isEnforcedFieldPos = rec.getType().getFieldIndex(INDEX_ISENFORCED_FIELD_NAME);
         Boolean isEnforcingKeys = false;
         if (isEnforcedFieldPos > 0)
             isEnforcingKeys = ((ABoolean) rec.getValueByPos(isEnforcedFieldPos)).getBoolean();
@@ -151,7 +151,7 @@
                 .getIntegerValue();
         // Check if there is a gram length as well.
         int gramLength = -1;
-        int gramLenPos = rec.getType().findFieldPosition(GRAM_LENGTH_FIELD_NAME);
+        int gramLenPos = rec.getType().getFieldIndex(GRAM_LENGTH_FIELD_NAME);
         if (gramLenPos >= 0) {
             gramLength = ((AInt32) rec.getValueByPos(gramLenPos)).getIntegerValue();
         }
diff --git a/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 55ff32f..e3ed6fb 100644
--- a/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
 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.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
@@ -72,9 +73,9 @@
     private int[] openFieldNameLengths;
 
     private int numberOfOpenFields;
+    private RuntimeRecordTypeInfo recTypeInfo;
 
     public RecordBuilder() {
-
         this.closedPartOutputStream = new ByteArrayOutputStream();
         this.numberOfClosedFields = 0;
 
@@ -91,6 +92,8 @@
         this.openPartOffsetArray = null;
         this.openPartOffsetArraySize = 0;
         this.offsetPosition = 0;
+
+        this.recTypeInfo = new RuntimeRecordTypeInfo();
     }
 
     @Override
@@ -108,6 +111,7 @@
     @Override
     public void reset(ARecordType recType) {
         this.recType = recType;
+        this.recTypeInfo.reset(recType);
         this.closedPartOutputStream.reset();
         this.openPartOutputStream.reset();
         this.numberOfClosedFields = 0;
@@ -178,8 +182,8 @@
     public void addField(IValueReference name, IValueReference value) throws AsterixException {
         if (numberOfOpenFields == openPartOffsets.length) {
             openPartOffsets = Arrays.copyOf(openPartOffsets, openPartOffsets.length + DEFAULT_NUM_OPEN_FIELDS);
-            openFieldNameLengths = Arrays.copyOf(openFieldNameLengths, openFieldNameLengths.length
-                    + DEFAULT_NUM_OPEN_FIELDS);
+            openFieldNameLengths = Arrays.copyOf(openFieldNameLengths,
+                    openFieldNameLengths.length + DEFAULT_NUM_OPEN_FIELDS);
         }
         int fieldNameHashCode;
         try {
@@ -191,7 +195,7 @@
         if (recType != null) {
             int cFieldPos;
             try {
-                cFieldPos = recType.findFieldPosition(name.getByteArray(), name.getStartOffset() + 1,
+                cFieldPos = recTypeInfo.getFieldIndex(name.getByteArray(), name.getStartOffset() + 1,
                         name.getLength() - 1);
             } catch (HyracksDataException e) {
                 throw new AsterixException(e);
@@ -229,8 +233,8 @@
                             openBytes, (int) openPartOffsets[i], openFieldNameLengths[i]) == 0) {
                         String field = utf8SerDer.deserialize(new DataInputStream(new ByteArrayInputStream(openBytes,
                                 (int) openPartOffsets[i], openFieldNameLengths[i])));
-                        throw new AsterixException("Open fields " + (i - 1) + " and " + i
-                                + " have the same field name \"" + field + "\"");
+                        throw new AsterixException(
+                                "Open fields " + (i - 1) + " and " + i + " have the same field name \"" + field + "\"");
                     }
                 }
             }
@@ -239,11 +243,10 @@
             int fieldNameHashCode;
             for (int i = 0; i < numberOfOpenFields; i++) {
                 fieldNameHashCode = (int) (openPartOffsets[i] >> 32);
-                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray, (int) fieldNameHashCode,
-                        offsetPosition);
+                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray, fieldNameHashCode, offsetPosition);
                 int fieldOffset = (int) openPartOffsets[i];
-                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray, fieldOffset + openPartOffset + 4
-                        + openPartOffsetArraySize, offsetPosition + 4);
+                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray,
+                        fieldOffset + openPartOffset + 4 + openPartOffsetArraySize, offsetPosition + 4);
                 offsetPosition += 8;
             }
             recordLength = openPartOffset + 4 + openPartOffsetArraySize + openPartOutputStream.size();
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 801907c..0d1f63f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -54,8 +54,9 @@
 
     private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
     private AObjectSerializerDeserializer aObjSerDer = null;
+
     private AObjectSerializerDeserializer getObjSerDer() {
-        if (aObjSerDer == null){
+        if (aObjSerDer == null) {
             aObjSerDer = new AObjectSerializerDeserializer();
         }
         return aObjSerDer;
@@ -84,9 +85,8 @@
                 if (t.getTypeTag() == ATypeTag.UNION) {
                     if (((AUnionType) t).isNullableType()) {
                         t2 = ((AUnionType) recordType.getFieldTypes()[i]).getNullableType();
-                        serializers[i] = AqlSerializerDeserializerProvider.INSTANCE
-                                .getSerializerDeserializer(((AUnionType) recordType.getFieldTypes()[i])
-                                        .getNullableType());
+                        serializers[i] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(
+                                ((AUnionType) recordType.getFieldTypes()[i]).getNullableType());
                     } else {
                         // union .. the general case
                         throw new NotImplementedException();
@@ -132,7 +132,7 @@
                 }
                 for (int fieldId = 0; fieldId < numberOfSchemaFields; fieldId++) {
                     if (hasNullableFields && ((nullBitMap[fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)) {
-                        closedFields[fieldId] = (IAObject) ANull.NULL;
+                        closedFields[fieldId] = ANull.NULL;
                         continue;
                     }
                     closedFields[fieldId] = (IAObject) deserializers[fieldId].deserialize(in);
@@ -250,8 +250,7 @@
                             // the field value is null
                             return 0;
                     }
-                    return AInt32SerializerDeserializer.getInt(serRecord,
-                            (int) (14 + offset + nullBitmapSize + (4 * fieldId)));
+                    return AInt32SerializerDeserializer.getInt(serRecord, 14 + offset + nullBitmapSize + (4 * fieldId));
                 } else {
                     if (nullBitmapSize > 0) {
                         // 9 = tag (1) + record Size (4) + isExpanded (1) +
@@ -260,8 +259,7 @@
                             // the field value is null
                             return 0;
                     }
-                    return AInt32SerializerDeserializer.getInt(serRecord,
-                            (int) (10 + offset + nullBitmapSize + (4 * fieldId)));
+                    return AInt32SerializerDeserializer.getInt(serRecord, 10 + offset + nullBitmapSize + (4 * fieldId));
                 }
             } else
                 return -1;
@@ -273,8 +271,7 @@
                     if ((serRecord[9 + offset + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
                         // the field value is null
                         return 0;
-                return AInt32SerializerDeserializer.getInt(serRecord,
-                        (int) (9 + offset + nullBitmapSize + (4 * fieldId)));
+                return AInt32SerializerDeserializer.getInt(serRecord, 9 + offset + nullBitmapSize + (4 * fieldId));
             } else
                 return -1;
         }
@@ -322,9 +319,8 @@
             if (h == fieldNameHashCode) {
                 fieldOffset = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * mid) + 4);
                 // the utf8 comparator do not require to put the precise length, we can just pass a estimated limit.
-                if (utf8BinaryComparator
-                        .compare(serRecord, fieldOffset, serRecord.length,
-                                fieldName, 1, fieldUtflength + fieldUtfMetaLen) == 0)
+                if (utf8BinaryComparator.compare(serRecord, fieldOffset, serRecord.length, fieldName, 1,
+                        fieldUtflength + fieldUtfMetaLen) == 0)
                     // since they are equal, we can directly use the meta length and the utf length.
                     return fieldOffset + fieldUtfMetaLen + fieldUtflength;
                 else { // this else part has not been tested yet
@@ -332,8 +328,8 @@
                         h = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j));
                         if (h == fieldNameHashCode) {
                             fieldOffset = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j) + 4);
-                            if (utf8BinaryComparator.compare(serRecord, fieldOffset, serRecord.length, fieldName,
-                                    1, fieldUtflength) == 0)
+                            if (utf8BinaryComparator.compare(serRecord, fieldOffset, serRecord.length, fieldName, 1,
+                                    fieldUtflength) == 0)
                                 return fieldOffset + fieldUtfMetaLen + fieldUtflength;
                         } else
                             break;
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java b/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
index 41efb3a..be00153 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
@@ -19,7 +19,6 @@
 
 package org.apache.asterix.om.pointables.cast;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -37,7 +36,6 @@
 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.types.hierachy.ITypeConvertComputer;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
@@ -56,6 +54,7 @@
 
     private final Map<IVisitablePointable, ARecordCaster> raccessorToCaster = new HashMap<IVisitablePointable, ARecordCaster>();
     private final Map<IVisitablePointable, AListCaster> laccessorToCaster = new HashMap<IVisitablePointable, AListCaster>();
+    private final ArrayBackedValueStorage castBuffer = new ArrayBackedValueStorage();
 
     @Override
     public Void visit(AListVisitablePointable accessor, Triple<IVisitablePointable, IAType, Boolean> arg)
@@ -89,11 +88,7 @@
                 arg.second = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
             }
             ARecordType resultType = (ARecordType) arg.second;
-            //cloning result type to avoid race conditions during comparison\hash calculation
-            ARecordType clonedResultType = new ARecordType(resultType.getTypeName(), resultType.getFieldNames(),
-                    resultType.getFieldTypes(), resultType.isOpen());
-
-            caster.castRecord(accessor, arg.first, clonedResultType, this);
+            caster.castRecord(accessor, arg.first, resultType, this);
         } catch (Exception e) {
             throw new AsterixException(e);
         }
@@ -109,21 +104,20 @@
             return null;
         }
         // set the pointer for result
-        ATypeTag reqTypeTag = ((IAType) (arg.second)).getTypeTag();
-        ATypeTag inputTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(accessor.getByteArray()[accessor
-                .getStartOffset()]);
+        ATypeTag reqTypeTag = (arg.second).getTypeTag();
+        ATypeTag inputTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(accessor.getByteArray()[accessor.getStartOffset()]);
         if (!needPromote(inputTypeTag, reqTypeTag)) {
             arg.first.set(accessor);
         } else {
-            ArrayBackedValueStorage castBuffer = new ArrayBackedValueStorage();
-
             try {
+                castBuffer.reset();
                 ATypeHierarchy.convertNumericTypeByteArray(accessor.getByteArray(), accessor.getStartOffset(),
                         accessor.getLength(), reqTypeTag, castBuffer.getDataOutput());
                 arg.first.set(castBuffer);
             } catch (IOException e1) {
-                throw new AsterixException("Type mismatch: cannot cast the " + inputTypeTag + " type to the "
-                        + reqTypeTag + " type.");
+                throw new AsterixException(
+                        "Type mismatch: cannot cast the " + inputTypeTag + " type to the " + reqTypeTag + " type.");
             }
 
         }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
index 283037f..2d9cd5d 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
@@ -24,8 +24,6 @@
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.commons.lang3.ArrayUtils;
-
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.types.ARecordType;
@@ -33,6 +31,7 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeHelper;
+import org.apache.commons.lang3.ArrayUtils;
 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;
@@ -74,8 +73,8 @@
         ARecordType recType1 = extractRecordType(t1);
 
         if (recType0 == null || recType1 == null) {
-            throw new AlgebricksException("record-merge expects possibly NULL records as arguments, but got (" + t0
-                    + ", " + t1 + ")");
+            throw new AlgebricksException(
+                    "record-merge expects possibly NULL records as arguments, but got (" + t0 + ", " + t1 + ")");
         }
 
         List<String> resultFieldNames = new ArrayList<>();
@@ -145,21 +144,22 @@
 
         for (int i = 0; i < fieldType1Copy.getFieldTypes().length; i++) {
             try {
-                int pos = returnType.findFieldPosition(fieldType1Copy.getFieldNames()[i]);
+                int pos = returnType.getFieldIndex(fieldType1Copy.getFieldNames()[i]);
                 if (pos >= 0) {
                     if (fieldType1Copy.getFieldTypes()[i].getTypeTag() != ATypeTag.RECORD) {
                         break;
                     }
                     IAType[] oldTypes = returnType.getFieldTypes();
-                    oldTypes[pos] = mergedNestedType(fieldType1Copy.getFieldTypes()[i], returnType.getFieldTypes()[pos]);
+                    oldTypes[pos] = mergedNestedType(fieldType1Copy.getFieldTypes()[i],
+                            returnType.getFieldTypes()[pos]);
                     returnType = new ARecordType(returnType.getTypeName(), returnType.getFieldNames(), oldTypes,
                             returnType.isOpen());
                 } else {
                     IAType[] combinedFieldTypes = ArrayUtils.addAll(returnType.getFieldTypes().clone(),
                             fieldType1Copy.getFieldTypes()[i]);
-                    returnType = new ARecordType(returnType.getTypeName(), ArrayUtils.addAll(
-                            returnType.getFieldNames(), fieldType1Copy.getFieldNames()[i]), combinedFieldTypes,
-                            returnType.isOpen());
+                    returnType = new ARecordType(returnType.getTypeName(),
+                            ArrayUtils.addAll(returnType.getFieldNames(), fieldType1Copy.getFieldNames()[i]),
+                            combinedFieldTypes, returnType.isOpen());
                 }
 
             } catch (IOException | AsterixException e) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index a1e0739..2be1b4e 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -19,18 +19,12 @@
 
 package org.apache.asterix.om.types;
 
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
-
-import org.apache.hyracks.util.string.UTF8StringUtil;
-import org.apache.hyracks.util.string.UTF8StringWriter;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
+import java.util.Map;
 
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
@@ -39,31 +33,20 @@
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
 import org.apache.asterix.om.visitors.IOMVisitor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 
 public class ARecordType extends AbstractComplexType {
 
     private static final long serialVersionUID = 1L;
-    private String[] fieldNames;
-    private IAType[] fieldTypes;
-    private boolean isOpen;
+    private final String[] fieldNames;
+    private final IAType[] fieldTypes;
+    private final Map<String, Integer> fieldNameToIndexMap = new HashMap<String, Integer>();
+    private final boolean isOpen;
     private final List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
-
-    private transient IBinaryHashFunction fieldNameHashFunction;
-    private transient IBinaryComparator fieldNameComparator;
-    private final byte serializedFieldNames[];
-    private final int serializedFieldNameOffsets[];
-    private final long hashCodeIndexPairs[];
-
-    private final UTF8StringSerializerDeserializer utf8SerDer = new UTF8StringSerializerDeserializer();
 
     /**
      * @param typeName
@@ -78,7 +61,6 @@
      *             if there are duplicate field names or if there is an error serializing the field names
      * @throws HyracksDataException
      */
-    @SuppressWarnings("resource")
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen)
             throws AsterixException, HyracksDataException {
         super(typeName);
@@ -86,91 +68,10 @@
         this.fieldTypes = fieldTypes;
         this.isOpen = isOpen;
 
-        fieldNameComparator = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY)
-                .createBinaryComparator();
-        fieldNameHashFunction = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
-                .createBinaryHashFunction();
-        ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
-        DataOutputStream dos = new DataOutputStream(baaos);
-        UTF8StringWriter writer = new UTF8StringWriter();
-        serializedFieldNameOffsets = new int[fieldNames.length];
-        hashCodeIndexPairs = new long[fieldNames.length];
-
-        int length = 0;
-        for (int i = 0; i < fieldNames.length; i++) {
-            serializedFieldNameOffsets[i] = baaos.size();
-            try {
-                writer.writeUTF8(fieldNames[i], dos);
-            } catch (IOException e) {
-                throw new AsterixException(e);
-            }
-            length = baaos.size() - serializedFieldNameOffsets[i];
-            hashCodeIndexPairs[i] = fieldNameHashFunction.hash(baaos.getByteArray(), serializedFieldNameOffsets[i],
-                    length);
-            hashCodeIndexPairs[i] = hashCodeIndexPairs[i] << 32;
-            hashCodeIndexPairs[i] = hashCodeIndexPairs[i] | i;
+        // Puts field names to the field name to field index map.
+        for (int index = 0; index < fieldNames.length; ++index) {
+            fieldNameToIndexMap.put(fieldNames[index], index);
         }
-        try {
-            dos.close();
-        } catch (IOException e) {
-            throw new AsterixException(e);
-        }
-        serializedFieldNames = baaos.getByteArray();
-
-        Arrays.sort(hashCodeIndexPairs);
-        int j;
-        for (int i = 0; i < fieldNames.length; i++) {
-            j = findFieldPosition(serializedFieldNames, serializedFieldNameOffsets[i],
-                    UTF8StringUtil.getStringLength(serializedFieldNames, serializedFieldNameOffsets[i]));
-            if (j != i) {
-                throw new AsterixException("Closed fields " + j + " and " + i + " have the same field name \""
-                        + fieldNames[i] + "\"");
-            }
-        }
-    }
-
-    private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
-        ois.defaultReadObject();
-        fieldNameComparator = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY)
-                .createBinaryComparator();
-        fieldNameHashFunction = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
-                .createBinaryHashFunction();
-    }
-
-    /**
-     * Returns the position of the field in the closed schema or -1 if the field does not exist.
-     *
-     * @param bytes
-     *            the serialized bytes of the field name
-     * @param start
-     *            the starting offset of the field name in bytes
-     * @param length
-     *            the length of the field name in bytes
-     * @return the position of the field in the closed schema or -1 if the field does not exist.
-     * @throws HyracksDataException
-     */
-    public int findFieldPosition(byte[] bytes, int start, int length) throws HyracksDataException {
-        if (hashCodeIndexPairs.length == 0) {
-            return -1;
-        }
-
-        int fIndex;
-        int probeFieldHash = fieldNameHashFunction.hash(bytes, start, length);
-        int i = Arrays.binarySearch(hashCodeIndexPairs, ((long) probeFieldHash) << 32);
-        i = (i < 0) ? (i = -1 * (i + 1)) : i;
-
-        while (i < hashCodeIndexPairs.length && (int) (hashCodeIndexPairs[i] >>> 32) == probeFieldHash) {
-            fIndex = (int) hashCodeIndexPairs[i];
-            int cFieldLength = UTF8StringUtil.getStringLength(serializedFieldNames,
-                    serializedFieldNameOffsets[fIndex]);
-            if (fieldNameComparator.compare(serializedFieldNames, serializedFieldNameOffsets[fIndex], cFieldLength,
-                    bytes, start, length) == 0) {
-                return fIndex;
-            }
-            i++;
-        }
-
-        return -1;
     }
 
     public final String[] getFieldNames() {
@@ -223,11 +124,16 @@
      *            the name of the field whose position is sought
      * @return the position of the field in the closed schema or -1 if the field does not exist.
      */
-    public int findFieldPosition(String fieldName) throws IOException {
-        ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
-        DataOutputStream dos = new DataOutputStream(baaos);
-        utf8SerDer.serialize(fieldName, dos);
-        return findFieldPosition(baaos.getByteArray(), 0, baaos.getByteArray().length);
+    public int getFieldIndex(String fieldName) throws IOException {
+        if (fieldNames == null) {
+            return -1;
+        }
+        Integer index = fieldNameToIndexMap.get(fieldName);
+        if (index == null) {
+            return -1;
+        } else {
+            return index;
+        }
     }
 
     /**
@@ -263,8 +169,8 @@
                 //enforced SubType
                 subRecordType = ((AUnionType) subRecordType).getNullableType();
                 if (subRecordType.getTypeTag().serialize() != ATypeTag.RECORD.serialize()) {
-                    throw new IOException("Field accessor is not defined for values of type "
-                            + subRecordType.getTypeTag());
+                    throw new IOException(
+                            "Field accessor is not defined for values of type " + subRecordType.getTypeTag());
                 }
 
             }
@@ -283,7 +189,7 @@
      *             if an error occurs while serializing the field name
      */
     public IAType getFieldType(String fieldName) throws IOException {
-        int fieldPos = findFieldPosition(fieldName);
+        int fieldPos = getFieldIndex(fieldName);
         if (fieldPos < 0 || fieldPos >= fieldTypes.length) {
             return null;
         }
@@ -299,7 +205,7 @@
      * @throws IOException
      */
     public boolean isClosedField(String fieldName) throws IOException {
-        return findFieldPosition(fieldName) != -1;
+        return getFieldIndex(fieldName) != -1;
     }
 
     /**
@@ -332,6 +238,10 @@
                 List<String> fieldName = partitioningExprs.get(i);
                 IAType fieldType = getSubFieldType(fieldName);
 
+                if (fieldType == null) {
+                    System.out.println();
+                }
+
                 switch (fieldType.getTypeTag()) {
                     case INT8:
                     case INT16:
@@ -358,20 +268,6 @@
             }
         }
         return partitioningExprTypes;
-    }
-
-    private IAType getPartitioningExpressionType(String fieldName, boolean autogenerated) throws AsterixException,
-            IOException {
-        IAType fieldType = getFieldType(fieldName);
-        if (fieldType == null) {
-            if (autogenerated) {
-                throw new AsterixException("Primary key field: " + fieldName
-                        + " should be defined in the type that the dataset is using.");
-            } else {
-                throw new AsterixException("Primary key field: " + fieldName + " could not be found.");
-            }
-        }
-        return fieldType;
     }
 
     /**
@@ -401,8 +297,8 @@
                         + "\" could be only in a prefix part of the composite index, containing opened field.");
             if (keyFieldTypes.get(pos) != BuiltinType.ANULL
                     && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag())
-                throw new AlgebricksException("A field \"" + fieldName + "\" is already defined with the type \""
-                        + fieldType + "\"");
+                throw new AlgebricksException(
+                        "A field \"" + fieldName + "\" is already defined with the type \"" + fieldType + "\"");
             switch (indexType) {
                 case BTREE:
                     switch (fieldType.getTypeTag()) {
@@ -447,9 +343,9 @@
                         case UNION:
                             break;
                         default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag()
-                                    + " cannot be indexed using the Length Partitioned N-Gram index.");
+                            throw new AlgebricksException(
+                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
+                                            + " cannot be indexed using the Length Partitioned N-Gram index.");
                     }
                     break;
                 case LENGTH_PARTITIONED_WORD_INVIX:
@@ -460,9 +356,9 @@
                         case UNION:
                             break;
                         default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag()
-                                    + " cannot be indexed using the Length Partitioned Keyword index.");
+                            throw new AlgebricksException(
+                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
+                                            + " cannot be indexed using the Length Partitioned Keyword index.");
                     }
                     break;
                 case SINGLE_PARTITION_NGRAM_INVIX:
@@ -599,7 +495,7 @@
     public int hash() {
         int h = 0;
         for (int i = 0; i < fieldNames.length; i++) {
-            h += 31 * h + (int) (hashCodeIndexPairs[i] >> 32);
+            h += 31 * h + fieldNames[i].hashCode();
         }
         for (int i = 0; i < fieldTypes.length; i++) {
             h += 31 * h + fieldTypes[i].hashCode();
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java b/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
new file mode 100644
index 0000000..03e88f3
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.types.runtime;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+
+public class RuntimeRecordTypeInfo {
+
+    private final IBinaryHashFunction fieldNameHashFunction;
+    private final IBinaryComparator fieldNameComparator;
+    private final UTF8StringWriter writer;
+    private final ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
+    private final DataOutputStream dos = new DataOutputStream(baaos);
+
+    private int[] serializedFieldNameOffsets;
+    private long[] hashCodeIndexPairs;
+    private ARecordType cachedRecType = null;
+
+    public RuntimeRecordTypeInfo() {
+        fieldNameComparator = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY)
+                .createBinaryComparator();
+        fieldNameHashFunction = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
+                .createBinaryHashFunction();
+        writer = new UTF8StringWriter();
+    }
+
+    /**
+     * Reset the binary artifacts of a runtime type info instance.
+     *
+     * @param recType,
+     *            the record type.
+     */
+    public void reset(ARecordType recType) {
+        if (cachedRecType == recType) {
+            // if the time doesn't change, we just skip the reset.
+            return;
+        }
+        // Sets the record type.
+        cachedRecType = recType;
+        // Resets the bytes for names.
+        baaos.reset();
+
+        if (recType != null) {
+            String[] fieldNames = recType.getFieldNames();
+            if (serializedFieldNameOffsets == null || serializedFieldNameOffsets.length != fieldNames.length) {
+                serializedFieldNameOffsets = new int[fieldNames.length];
+                hashCodeIndexPairs = new long[fieldNames.length];
+            }
+            int length = 0;
+            try {
+                for (int i = 0; i < fieldNames.length; ++i) {
+                    serializedFieldNameOffsets[i] = baaos.size();
+                    writer.writeUTF8(fieldNames[i], dos);
+                    length = baaos.size() - serializedFieldNameOffsets[i];
+                    hashCodeIndexPairs[i] = fieldNameHashFunction.hash(baaos.getByteArray(),
+                            serializedFieldNameOffsets[i], length);
+                    hashCodeIndexPairs[i] = hashCodeIndexPairs[i] << 32;
+                    hashCodeIndexPairs[i] = hashCodeIndexPairs[i] | i;
+                }
+                dos.flush();
+                Arrays.sort(hashCodeIndexPairs);
+                for (int i = 0; i < fieldNames.length; i++) {
+                    int j = getFieldIndex(baaos.getByteArray(), serializedFieldNameOffsets[i],
+                            UTF8StringUtil.getStringLength(baaos.getByteArray(), serializedFieldNameOffsets[i]));
+                    if (j != i) {
+                        throw new IllegalStateException("Closed fields " + j + " and " + i
+                                + " have the same field name \"" + fieldNames[i] + "\"");
+                    }
+                }
+            } catch (IOException e) {
+                throw new IllegalStateException(e);
+            }
+        } else {
+            serializedFieldNameOffsets = new int[0];
+            hashCodeIndexPairs = new long[0];
+        }
+    }
+
+    /**
+     * Returns the position of the field in the closed schema or -1 if the field does not exist.
+     *
+     * @param bytes
+     *            the serialized bytes of the field name
+     * @param start
+     *            the starting offset of the field name in bytes
+     * @param length
+     *            the length of the field name in bytes
+     * @return the position of the field in the closed schema or -1 if the field does not exist.
+     * @throws HyracksDataException
+     */
+    public int getFieldIndex(byte[] bytes, int start, int length) throws HyracksDataException {
+        if (hashCodeIndexPairs.length == 0) {
+            return -1;
+        }
+        int fIndex;
+        int probeFieldHash = fieldNameHashFunction.hash(bytes, start, length);
+        int i = Arrays.binarySearch(hashCodeIndexPairs, ((long) probeFieldHash) << 32);
+        i = (i < 0) ? (i = -1 * (i + 1)) : i;
+
+        while (i < hashCodeIndexPairs.length && (int) (hashCodeIndexPairs[i] >>> 32) == probeFieldHash) {
+            fIndex = (int) hashCodeIndexPairs[i];
+            int cFieldLength = UTF8StringUtil.getStringLength(baaos.getByteArray(), serializedFieldNameOffsets[fIndex]);
+            if (fieldNameComparator.compare(baaos.getByteArray(), serializedFieldNameOffsets[fIndex], cFieldLength,
+                    bytes, start, length) == 0) {
+                return fIndex;
+            }
+            i++;
+        }
+        return -1;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index 883be0a..2f41816 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -22,6 +22,7 @@
 import java.util.List;
 
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
@@ -50,22 +51,27 @@
     public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
         return new ICopyEvaluator() {
 
-            private DataOutput out = output.getDataOutput();
-            private ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
+            private final DataOutput out = output.getDataOutput();
+            private final ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
 
-            private ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
-            private ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
-            private ArrayBackedValueStorage[] abvsFields = new ArrayBackedValueStorage[fieldPath.size()];
-            private DataOutput[] doFields = new DataOutput[fieldPath.size()];
+            private final ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
+            private final ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
+            private final ArrayBackedValueStorage[] abvsFields = new ArrayBackedValueStorage[fieldPath.size()];
+            private final DataOutput[] doFields = new DataOutput[fieldPath.size()];
+            private final RuntimeRecordTypeInfo[] recTypeInfos = new RuntimeRecordTypeInfo[fieldPath.size()];
 
             {
                 FieldAccessUtil.getFieldsAbvs(abvsFields, doFields, fieldPath);
-                recordType = recordType.deepCopy(recordType);
+                for (int index = 0; index < fieldPath.size(); ++index) {
+                    recTypeInfos[index] = new RuntimeRecordTypeInfo();
+                }
+
             }
 
             @Override
             public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
-                FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, recordType);
+                FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, recordType,
+                        recTypeInfos);
             }
         };
     }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
index db8a6b4..a442ef4 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
@@ -34,9 +34,10 @@
 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.runtime.RuntimeRecordTypeInfo;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.commons.lang.NotImplementedException;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -89,8 +90,8 @@
 
     public static void evaluate(IFrameTupleReference tuple, DataOutput out, ICopyEvaluator eval0,
             ArrayBackedValueStorage[] abvsFields, ArrayBackedValueStorage abvsRecord,
-            ByteArrayAccessibleOutputStream subRecordTmpStream, ARecordType recordType) throws AlgebricksException {
-
+            ByteArrayAccessibleOutputStream subRecordTmpStream, ARecordType recordType,
+            RuntimeRecordTypeInfo[] recTypeInfos) throws AlgebricksException {
         try {
             abvsRecord.reset();
             eval0.evaluate(tuple);
@@ -99,7 +100,10 @@
             int subFieldOffset = -1;
             int subFieldLength = -1;
             int nullBitmapSize = -1;
+
             IAType subType = recordType;
+            recTypeInfos[0].reset(recordType);
+
             ATypeTag subTypeTag = ATypeTag.NULL;
             byte[] subRecord = abvsRecord.getByteArray();
             boolean openField = false;
@@ -117,9 +121,11 @@
                     if (subType.getTypeTag().serialize() != SER_RECORD_TYPE_TAG) {
                         throw new AlgebricksException("Field accessor is not defined for values of type " + subTypeTag);
                     }
-
+                    if (subType.getTypeTag() == ATypeTag.RECORD) {
+                        recTypeInfos[i].reset((ARecordType) subType);
+                    }
                 }
-                subFieldIndex = ((ARecordType) subType).findFieldPosition(abvsFields[i].getByteArray(),
+                subFieldIndex = recTypeInfos[i].getFieldIndex(abvsFields[i].getByteArray(),
                         abvsFields[i].getStartOffset() + 1, abvsFields[i].getLength());
                 if (subFieldIndex == -1) {
                     break;
@@ -133,6 +139,10 @@
                     return;
                 }
                 subType = ((ARecordType) subType).getFieldTypes()[subFieldIndex];
+                if (subType.getTypeTag() == ATypeTag.RECORD && i + 1 < abvsFields.length) {
+                    // Move to the next Depth
+                    recTypeInfos[i + 1].reset((ARecordType) subType);
+                }
                 if (subType.getTypeTag().equals(ATypeTag.UNION)) {
                     if (((AUnionType) subType).isNullableType()) {
                         subTypeTag = ((AUnionType) subType).getNullableType().getTypeTag();
@@ -172,7 +182,8 @@
                 }
 
                 subTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(subRecord[subFieldOffset]);
-                subFieldLength = NonTaggedFormatUtil.getFieldValueLength(subRecord, subFieldOffset, subTypeTag, true) + 1;
+                subFieldLength = NonTaggedFormatUtil.getFieldValueLength(subRecord, subFieldOffset, subTypeTag, true)
+                        + 1;
 
                 if (i < abvsFields.length - 1) {
                     //setup next iteration
@@ -187,7 +198,6 @@
                 out.writeByte(subTypeTag.serialize());
             }
             out.write(subRecord, subFieldOffset, subFieldLength);
-
         } catch (IOException e) {
             throw new AlgebricksException(e);
         } catch (AsterixException e) {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 05e7292..8cabee1 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -26,6 +26,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.runtime.RuntimeRecordTypeInfo;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
@@ -56,26 +57,29 @@
     public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
         return new ICopyEvaluator() {
 
-            private DataOutput out = output.getDataOutput();
-            private ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
+            private final DataOutput out = output.getDataOutput();
+            private final ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
 
-            private ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
-            private ArrayBackedValueStorage outInput1 = new ArrayBackedValueStorage();
-            private ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
-            private ICopyEvaluator eval1 = fldNameEvalFactory.createEvaluator(outInput1);
+            private final ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
+            private final ArrayBackedValueStorage outInput1 = new ArrayBackedValueStorage();
+            private final ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
+            private final ICopyEvaluator eval1 = fldNameEvalFactory.createEvaluator(outInput1);
 
-            int size = 1;
-            private ArrayBackedValueStorage abvsFields[] = new ArrayBackedValueStorage[size];
-            private DataOutput[] doFields = new DataOutput[size];
+            private final int size = 1;
+            private final ArrayBackedValueStorage abvsFields[] = new ArrayBackedValueStorage[size];
+            private final DataOutput[] doFields = new DataOutput[size];
 
             @SuppressWarnings("unchecked")
-            private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+            private final ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
                     .getSerializerDeserializer(BuiltinType.ANULL);
+            private final RuntimeRecordTypeInfo[] recTypeInfos = new RuntimeRecordTypeInfo[size];
 
-            private ARecordType mRecordType = recordType.deepCopy(recordType);
             {
                 abvsFields[0] = new ArrayBackedValueStorage();
                 doFields[0] = abvsFields[0].getDataOutput();
+                for (int index = 0; index < size; ++index) {
+                    recTypeInfos[index] = new RuntimeRecordTypeInfo();
+                }
             }
 
             @Override
@@ -92,7 +96,8 @@
                     abvsFields[0].reset();
                     doFields[0].write(serFldName);
 
-                    FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, mRecordType);
+                    FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, recordType,
+                            recTypeInfos);
                 } catch (IOException e) {
                     throw new AlgebricksException(e);
                 }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
index 3fc5db1..d8bafc0 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
@@ -69,8 +69,7 @@
             private DataOutput out = output.getDataOutput();
             private RecordFieldsUtil rfu = new RecordFieldsUtil();
 
-            protected ARecordType mRecordType = recordType.deepCopy(recordType);
-
+            @Override
             public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
                 outInput0.reset();
                 eval0.evaluate(tuple);
@@ -91,7 +90,7 @@
                 recordPointable.set(outInput0.getByteArray(), outInput0.getStartOffset(), outInput0.getLength());
 
                 try {
-                    rfu.processRecord(recordPointable, mRecordType, out, 0);
+                    rfu.processRecord(recordPointable, recordType, out, 0);
                 } catch (IOException e) {
                     e.printStackTrace();
                 } catch (AsterixException e) {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
index 76716b1..bbba625 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
@@ -176,8 +176,8 @@
         orderedListBuilder.write(out, true);
     }
 
-    public void addNameField(IValueReference nameArg, IARecordBuilder fieldRecordBuilder) throws HyracksDataException,
-            AsterixException {
+    public void addNameField(IValueReference nameArg, IARecordBuilder fieldRecordBuilder)
+            throws HyracksDataException, AsterixException {
         ArrayBackedValueStorage fieldAbvs = getTempBuffer();
 
         fieldAbvs.reset();
@@ -185,8 +185,8 @@
         fieldRecordBuilder.addField(fieldAbvs, nameArg);
     }
 
-    public void addFieldType(byte tagId, IARecordBuilder fieldRecordBuilder) throws HyracksDataException,
-            AsterixException {
+    public void addFieldType(byte tagId, IARecordBuilder fieldRecordBuilder)
+            throws HyracksDataException, AsterixException {
         ArrayBackedValueStorage fieldAbvs = getTempBuffer();
         ArrayBackedValueStorage valueAbvs = getTempBuffer();
 
@@ -202,8 +202,8 @@
         fieldRecordBuilder.addField(fieldAbvs, valueAbvs);
     }
 
-    public void addIsOpenField(boolean isOpen, IARecordBuilder fieldRecordBuilder) throws HyracksDataException,
-            AsterixException {
+    public void addIsOpenField(boolean isOpen, IARecordBuilder fieldRecordBuilder)
+            throws HyracksDataException, AsterixException {
         ArrayBackedValueStorage fieldAbvs = getTempBuffer();
         ArrayBackedValueStorage valueAbvs = getTempBuffer();
 
@@ -246,9 +246,9 @@
         valueAbvs.reset();
         ARecordType newType;
         if (fieldType == null) {
-            newType = openType.deepCopy(openType);
+            newType = openType;
         } else {
-            newType = ((ARecordType) fieldType).deepCopy((ARecordType) fieldType);
+            newType = (ARecordType) fieldType;
         }
         ARecordPointable recordP = getRecordPointable();
         recordP.set(recordArg);
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
index df52f90..45e7393 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
@@ -66,6 +66,7 @@
     private static final byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordMergeDescriptor();
         }
@@ -156,8 +157,8 @@
                     }
 
                     private void mergeFields(ARecordType combinedType, ARecordVisitablePointable leftRecord,
-                            ARecordVisitablePointable rightRecord, boolean openFromParent, int nestedLevel) throws IOException,
-                            AsterixException, AlgebricksException {
+                            ARecordVisitablePointable rightRecord, boolean openFromParent, int nestedLevel)
+                                    throws IOException, AsterixException, AlgebricksException {
                         if (rbStack.size() < (nestedLevel + 1)) {
                             rbStack.push(new RecordBuilder());
                         }
@@ -225,13 +226,13 @@
 
                         //Add the merged field
                         if (combinedType.isClosedField(fieldName)) {
-                            int pos = combinedType.findFieldPosition(fieldName);
+                            int pos = combinedType.getFieldIndex(fieldName);
                             if (rightValue == null) {
                                 rbStack.get(nestedLevel).addField(pos, leftValue);
                             } else {
                                 mergeFields((ARecordType) combinedType.getFieldType(fieldName),
-                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue, false,
-                                        nestedLevel + 1);
+                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue,
+                                        false, nestedLevel + 1);
                                 tabvs.reset();
                                 rbStack.get(nestedLevel + 1).write(tabvs.getDataOutput(), true);
                                 rbStack.get(nestedLevel).addField(pos, tabvs);
@@ -241,8 +242,8 @@
                                 rbStack.get(nestedLevel).addField(fieldNamePointable, leftValue);
                             } else {
                                 mergeFields((ARecordType) combinedType.getFieldType(fieldName),
-                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue, false,
-                                        nestedLevel + 1);
+                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue,
+                                        false, nestedLevel + 1);
                                 tabvs.reset();
                                 rbStack.get(nestedLevel + 1).write(tabvs.getDataOutput(), true);
                                 rbStack.get(nestedLevel).addField(fieldNamePointable, tabvs);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/424/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has submitted this change and it was merged.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.

Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Reviewed-on: https://asterix-gerrit.ics.uci.edu/501
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
---
M asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
M asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
M asterix-app/src/test/resources/runtimets/queries_sqlpp/records/RecordsQueries.xml
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
M asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
A asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
22 files changed, 715 insertions(+), 492 deletions(-)

Approvals:
  Till Westmann: Looks good to me, approved
  Jenkins: Verified



diff --git a/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index c90a52a..0d7955d 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -20,18 +20,16 @@
 
 import java.util.List;
 
-import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
 import org.apache.asterix.formats.base.IDataFormat;
-import org.apache.asterix.metadata.declared.AqlMetadataProvider;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.runtime.evaluators.comparisons.ComparisonEvalFactory;
 import org.apache.asterix.runtime.formats.FormatUtils;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
@@ -65,7 +63,7 @@
     @Override
     public ICopyAggregateFunctionFactory createAggregateFunctionFactory(AggregateFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
         switch (fd.getFunctionDescriptorTag()) {
@@ -74,9 +72,9 @@
             case AGGREGATE:
                 return fd.createAggregateFunctionFactory(args);
             default:
-                throw new IllegalStateException("Invalid function descriptor " + fd.getFunctionDescriptorTag()
-                        + " expected " + FunctionDescriptorTag.SERIALAGGREGATE + " or "
-                        + FunctionDescriptorTag.AGGREGATE);
+                throw new IllegalStateException(
+                        "Invalid function descriptor " + fd.getFunctionDescriptorTag() + " expected "
+                                + FunctionDescriptorTag.SERIALAGGREGATE + " or " + FunctionDescriptorTag.AGGREGATE);
         }
     }
 
@@ -91,7 +89,7 @@
     @Override
     public ICopyUnnestingFunctionFactory createUnnestingFunctionFactory(UnnestingFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         return getFunctionDescriptor(expr, env, context).createUnnestingFunctionFactory(args);
     }
@@ -136,7 +134,7 @@
 
     private ICopyEvaluatorFactory createScalarFunctionEvaluatorFactory(AbstractFunctionCallExpression expr,
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
         FunctionIdentifier fi = expr.getFunctionIdentifier();
         ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(fi);
@@ -146,13 +144,12 @@
 
         IFunctionDescriptor fd = null;
         if (!(expr.getFunctionInfo() instanceof IExternalFunctionInfo)) {
-            AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
             IDataFormat format = FormatUtils.getDefaultFormat();
             fd = format.resolveFunction(expr, env);
         } else {
             try {
-                fd = ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor((IExternalFunctionInfo) expr
-                        .getFunctionInfo());
+                fd = ExternalFunctionDescriptorProvider
+                        .getExternalFunctionDescriptor((IExternalFunctionInfo) expr.getFunctionInfo());
             } catch (AsterixException ae) {
                 throw new AlgebricksException(ae);
             }
@@ -162,7 +159,6 @@
 
     private ICopyEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr,
             IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         IDataFormat format = FormatUtils.getDefaultFormat();
         return format.getConstantEvalFactory(expr.getValue());
     }
@@ -204,17 +200,15 @@
             }
 
             default:
-                throw new IllegalStateException("Invalid function descriptor " + fd.getFunctionDescriptorTag()
-                        + " expected " + FunctionDescriptorTag.SERIALAGGREGATE + " or "
-                        + FunctionDescriptorTag.AGGREGATE);
+                throw new IllegalStateException(
+                        "Invalid function descriptor " + fd.getFunctionDescriptorTag() + " expected "
+                                + FunctionDescriptorTag.SERIALAGGREGATE + " or " + FunctionDescriptorTag.AGGREGATE);
         }
     }
 
-    private IFunctionDescriptor getFunctionDescriptor(AbstractFunctionCallExpression expr,
-            IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
-        IFunctionDescriptor fd;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        fd = FormatUtils.getDefaultFormat().resolveFunction(expr, env);
+    private IFunctionDescriptor getFunctionDescriptor(AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,
+            JobGenContext context) throws AlgebricksException {
+        IFunctionDescriptor fd = FormatUtils.getDefaultFormat().resolveFunction(expr, env);
         return fd;
     }
 
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
index 8cdd78b..20bfea5 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
@@ -151,7 +151,7 @@
         if (s == null) {
             return null;
         }
-        int k = recType.findFieldPosition(s);
+        int k = recType.getFieldIndex(s);
         if (k < 0) {
             return null;
         }
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 5def2f8..f51d454 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -196,7 +196,7 @@
                         .getValue()).getValue()).getObject()).getStringValue();
                 int k;
                 try {
-                    k = rt.findFieldPosition(str);
+                    k = rt.getFieldIndex(str);
                 } catch (IOException e) {
                     throw new AlgebricksException(e);
                 }
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index ee30063..f5ef424 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -499,7 +499,7 @@
                         Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
                         ARecordType nestedRecType = nestedTypePair.first;
                         IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
-                        nestedRecTypeFieldTypes[nestedRecType.findFieldPosition(nestedTypePair.second)] = enforcedType;
+                        nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedTypePair.second)] = enforcedType;
                         enforcedType = new ARecordType(nestedRecType.getTypeName(), nestedRecType.getFieldNames(),
                                 nestedRecTypeFieldTypes, nestedRecType.isOpen());
                     }
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
new file mode 100644
index 0000000..1c8db4f
--- /dev/null
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.translator.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.AsterixException;
+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;
+
+/**
+ * A util that can verify if a filter field, a list of partitioning expressions,
+ * or a list of key fields are valid in a record type.
+ */
+public class ValidateUtil {
+
+    /**
+     * Validates the field that will be used as filter for the components of an LSM index.
+     *
+     * @param recType
+     *            the record type
+     * @param keyFieldNames
+     *            a list of key fields that will be validated
+     * @param indexType
+     *            the type of the index that its key fields is being validated
+     * @throws AsterixException
+     *             (if the validation failed), IOException
+     */
+    public static void validateFilterField(ARecordType recType, List<String> filterField)
+            throws AsterixException, IOException {
+        IAType fieldType = recType.getSubFieldType(filterField);
+        if (fieldType == null) {
+            throw new AsterixException("A field with this name  \"" + filterField + "\" could not be found.");
+        }
+        switch (fieldType.getTypeTag()) {
+            case INT8:
+            case INT16:
+            case INT32:
+            case INT64:
+            case FLOAT:
+            case DOUBLE:
+            case STRING:
+            case BINARY:
+            case DATE:
+            case TIME:
+            case DATETIME:
+            case UUID:
+            case YEARMONTHDURATION:
+            case DAYTIMEDURATION:
+                break;
+            case UNION:
+                throw new AsterixException("The filter field \"" + filterField + "\" cannot be nullable");
+            default:
+                throw new AsterixException("The field \"" + filterField + "\" which is of type "
+                        + fieldType.getTypeTag() + " cannot be used as a filter for a dataset.");
+        }
+    }
+
+    /**
+     * Validates the partitioning expression that will be used to partition a dataset and returns expression type.
+     *
+     * @param partitioningExprs
+     *            a list of partitioning expressions that will be validated
+     * @return a list of partitioning expressions types
+     * @throws AsterixException
+     *             (if the validation failed), IOException
+     */
+    public static List<IAType> validatePartitioningExpressions(ARecordType recType,
+            List<List<String>> partitioningExprs, boolean autogenerated) throws AsterixException, IOException {
+        List<IAType> partitioningExprTypes = new ArrayList<IAType>(partitioningExprs.size());
+        if (autogenerated) {
+            if (partitioningExprs.size() > 1) {
+                throw new AsterixException("Cannot autogenerate a composite primary key");
+            }
+            List<String> fieldName = partitioningExprs.get(0);
+            IAType fieldType = recType.getSubFieldType(fieldName);
+            partitioningExprTypes.add(fieldType);
+
+            ATypeTag pkTypeTag = fieldType.getTypeTag();
+            if (pkTypeTag != ATypeTag.UUID) {
+                throw new AsterixException("Cannot autogenerate a primary key for type " + pkTypeTag
+                        + ". Autogenerated primary keys must be of type " + ATypeTag.UUID + ".");
+            }
+        } else {
+            for (int i = 0; i < partitioningExprs.size(); i++) {
+                List<String> fieldName = partitioningExprs.get(i);
+                IAType fieldType = recType.getSubFieldType(fieldName);
+                switch (fieldType.getTypeTag()) {
+                    case INT8:
+                    case INT16:
+                    case INT32:
+                    case INT64:
+                    case FLOAT:
+                    case DOUBLE:
+                    case STRING:
+                    case BINARY:
+                    case DATE:
+                    case TIME:
+                    case UUID:
+                    case DATETIME:
+                    case YEARMONTHDURATION:
+                    case DAYTIMEDURATION:
+                        partitioningExprTypes.add(fieldType);
+                        break;
+                    case UNION:
+                        throw new AsterixException("The partitioning key \"" + fieldName + "\" cannot be nullable");
+                    default:
+                        throw new AsterixException("The partitioning key \"" + fieldName + "\" cannot be of type "
+                                + fieldType.getTypeTag() + ".");
+                }
+            }
+        }
+        return partitioningExprTypes;
+    }
+
+    /**
+     * Validates the key fields that will be used as keys of an index.
+     *
+     * @param recType
+     *            the record type
+     * @param keyFieldNames
+     *            a map of key fields that will be validated
+     * @param keyFieldTypes
+     *            a map of key types (if provided) that will be validated
+     * @param indexType
+     *            the type of the index that its key fields is being validated
+     * @throws AsterixException
+     *             (if the validation failed), IOException
+     */
+    public static void validateKeyFields(ARecordType recType, List<List<String>> keyFieldNames,
+            List<IAType> keyFieldTypes, IndexType indexType) throws AsterixException, IOException {
+        int pos = 0;
+        boolean openFieldCompositeIdx = false;
+        for (List<String> fieldName : keyFieldNames) {
+            IAType fieldType = recType.getSubFieldType(fieldName);
+            if (fieldType == null) {
+                fieldType = keyFieldTypes.get(pos);
+                if (keyFieldTypes.get(pos) == BuiltinType.ANULL)
+                    throw new AsterixException("A field with this name  \"" + fieldName + "\" could not be found.");
+            } else if (openFieldCompositeIdx)
+                throw new AsterixException("A closed field \"" + fieldName
+                        + "\" could be only in a prefix part of the composite index, containing opened field.");
+            if (keyFieldTypes.get(pos) != BuiltinType.ANULL
+                    && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag())
+                throw new AsterixException(
+                        "A field \"" + fieldName + "\" is already defined with the type \"" + fieldType + "\"");
+            switch (indexType) {
+                case BTREE:
+                    switch (fieldType.getTypeTag()) {
+                        case INT8:
+                        case INT16:
+                        case INT32:
+                        case INT64:
+                        case FLOAT:
+                        case DOUBLE:
+                        case STRING:
+                        case BINARY:
+                        case DATE:
+                        case TIME:
+                        case DATETIME:
+                        case UNION:
+                        case UUID:
+                        case YEARMONTHDURATION:
+                        case DAYTIMEDURATION:
+                            break;
+                        default:
+                            throw new AsterixException("The field \"" + fieldName + "\" which is of type "
+                                    + fieldType.getTypeTag() + " cannot be indexed using the BTree index.");
+                    }
+                    break;
+                case RTREE:
+                    switch (fieldType.getTypeTag()) {
+                        case POINT:
+                        case LINE:
+                        case RECTANGLE:
+                        case CIRCLE:
+                        case POLYGON:
+                        case UNION:
+                            break;
+                        default:
+                            throw new AsterixException("The field \"" + fieldName + "\" which is of type "
+                                    + fieldType.getTypeTag() + " cannot be indexed using the RTree index.");
+                    }
+                    break;
+                case LENGTH_PARTITIONED_NGRAM_INVIX:
+                    switch (fieldType.getTypeTag()) {
+                        case STRING:
+                        case UNION:
+                            break;
+                        default:
+                            throw new AsterixException(
+                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
+                                            + " cannot be indexed using the Length Partitioned N-Gram index.");
+                    }
+                    break;
+                case LENGTH_PARTITIONED_WORD_INVIX:
+                    switch (fieldType.getTypeTag()) {
+                        case STRING:
+                        case UNORDEREDLIST:
+                        case ORDEREDLIST:
+                        case UNION:
+                            break;
+                        default:
+                            throw new AsterixException(
+                                    "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
+                                            + " cannot be indexed using the Length Partitioned Keyword index.");
+                    }
+                    break;
+                case SINGLE_PARTITION_NGRAM_INVIX:
+                    switch (fieldType.getTypeTag()) {
+                        case STRING:
+                        case UNION:
+                            break;
+                        default:
+                            throw new AsterixException("The field \"" + fieldName + "\" which is of type "
+                                    + fieldType.getTypeTag() + " cannot be indexed using the N-Gram index.");
+                    }
+                    break;
+                case SINGLE_PARTITION_WORD_INVIX:
+                    switch (fieldType.getTypeTag()) {
+                        case STRING:
+                        case UNORDEREDLIST:
+                        case ORDEREDLIST:
+                        case UNION:
+                            break;
+                        default:
+                            throw new AsterixException("The field \"" + fieldName + "\" which is of type "
+                                    + fieldType.getTypeTag() + " cannot be indexed using the Keyword index.");
+                    }
+                    break;
+                default:
+                    throw new AsterixException("Invalid index type: " + indexType + ".");
+            }
+            pos++;
+        }
+    }
+
+}
diff --git a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
index 1fa610e..fa55a47 100644
--- a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
+++ b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
@@ -168,6 +168,7 @@
 import org.apache.asterix.translator.CompiledStatements.CompiledSubscribeFeedStatement;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.TypeTranslator;
+import org.apache.asterix.translator.util.ValidateUtil;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -586,12 +587,12 @@
                             .getPartitioningExprs();
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
-                    List<IAType> partitioningTypes = aRecordType.validatePartitioningExpressions(partitioningExprs,
-                            autogenerated);
+                    List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
+                            partitioningExprs, autogenerated);
 
                     List<String> filterField = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getFilterField();
                     if (filterField != null) {
-                        aRecordType.validateFilterField(filterField);
+                        ValidateUtil.validateFilterField(aRecordType, filterField);
                     }
                     if (compactionPolicy == null) {
                         if (filterField != null) {
@@ -867,7 +868,7 @@
                 indexFieldTypes.add(fieldType);
             }
 
-            aRecordType.validateKeyFields(indexFields, indexFieldTypes, stmtCreateIndex.getIndexType());
+            ValidateUtil.validateKeyFields(aRecordType, indexFields, indexFieldTypes, stmtCreateIndex.getIndexType());
 
             if (idx != null) {
                 if (stmtCreateIndex.getIfNotExists()) {
diff --git a/asterix-app/src/test/resources/runtimets/queries_sqlpp/records/RecordsQueries.xml b/asterix-app/src/test/resources/runtimets/queries_sqlpp/records/RecordsQueries.xml
index 8b13789..93a3a99 100644
--- a/asterix-app/src/test/resources/runtimets/queries_sqlpp/records/RecordsQueries.xml
+++ b/asterix-app/src/test/resources/runtimets/queries_sqlpp/records/RecordsQueries.xml
@@ -1 +1,131 @@
-
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+        <test-case FilePath="records">
+            <compilation-unit name="access-nested-fields">
+                <output-dir compare="Text">access-nested-fields</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="closed-record-constructor_01">
+                <output-dir compare="Text">closed-record-constructor_01</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="closed-record-constructor_02">
+                <output-dir compare="Text">closed-record-constructor_02</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="closed-record-constructor_03">
+                <output-dir compare="Text">closed-record-constructor_03</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="expFieldName">
+                <output-dir compare="Text">expFieldName</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="field-access-by-index_01">
+                <output-dir compare="Text">field-access-by-index_01</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="field-access-on-open-field">
+                <output-dir compare="Text">field-access-on-open-field</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-fields">
+            <compilation-unit name="documentation-example">
+                <output-dir compare="Text">documentation-example</output-dir>
+            </compilation-unit>
+        </test-case>
+        <!--test-case FilePath="records/get-record-fields">
+            <compilation-unit name="tiny-social-example">
+                <output-dir compare="Text">tiny-social-example</output-dir>
+            </compilation-unit>
+        </test-case!-->
+        <test-case FilePath="records/get-record-fields">
+            <compilation-unit name="tiny-social-example-no-complex-types">
+                <output-dir compare="Text">tiny-social-example-no-complex-types</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-fields">
+            <compilation-unit name="tiny-social-example-only-lists">
+                <output-dir compare="Text">tiny-social-example-only-lists</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-fields">
+            <compilation-unit name="tiny-social-example-only-records">
+                <output-dir compare="Text">tiny-social-example-only-records</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-field-value">
+            <compilation-unit name="documentation-example">
+                <output-dir compare="Text">documentation-example</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-field-value">
+            <compilation-unit name="highly-nested-closed">
+                <output-dir compare="Text">highly-nested-closed</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-field-value">
+            <compilation-unit name="highly-nested-mixed">
+                <output-dir compare="Text">highly-nested-mixed</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-field-value">
+            <compilation-unit name="highly-nested-open">
+                <output-dir compare="Text">highly-nested-open</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records/get-record-field-value">
+            <compilation-unit name="tiny-social-example">
+                <output-dir compare="Text">tiny-social-example</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="open-record-constructor_01">
+                <output-dir compare="Text">open-record-constructor_01</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="open-record-constructor_02">
+                <output-dir compare="Text">open-record-constructor_02</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="closed-closed-fieldname-conflict_issue173">
+                <output-dir compare="Text">closed-closed-fieldname-conflict_issue173</output-dir>
+                <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="open-closed-fieldname-conflict_issue173">
+                <output-dir compare="Text">open-closed-fieldname-conflict_issue173</output-dir>
+                <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="records">
+            <compilation-unit name="open-open-fieldname-conflict_issue173">
+                <output-dir compare="Text">open-open-fieldname-conflict_issue173</output-dir>
+                <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
+            </compilation-unit>
+        </test-case>
diff --git a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 095adbf..81466bc 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -18,9 +18,9 @@
  !-->
 <!DOCTYPE test-suite [
 
-<!ENTITY RecordsQueries SYSTEM "queries/records/RecordsQueries.xml">
+        <!ENTITY RecordsQueries SYSTEM "queries_sqlpp/records/RecordsQueries.xml">
 
-]>
+        ]>
 <test-suite
         xmlns="urn:xml.testframework.asterix.apache.org"
         ResultOffsetPath="results"
@@ -2912,12 +2912,11 @@
                     <output-dir compare="Text">btree-secondary-equi-join</output-dir>
                 </compilation-unit>
             </test-case>
-            <!-- ASTERIXDB-1159 <test-case FilePath="nested-open-index/index-join">
+            <test-case FilePath="nested-open-index/index-join">
                 <compilation-unit name="ngram-edit-distance">
                     <output-dir compare="Text">ngram-edit-distance</output-dir>
                 </compilation-unit>
             </test-case>
-            -->
             <!--  <test-case FilePath="nested-open-index/index-join">
                 <compilation-unit name="ngram-edit-distance-inline">
                     <output-dir compare="Text">ngram-edit-distance-inline</output-dir>
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 9cc021f..10bd02f 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -170,7 +170,7 @@
 
                 // Check if there is a filter field.
                 List<String> filterField = null;
-                int filterFieldPos = datasetDetailsRecord.getType().findFieldPosition(
+                int filterFieldPos = datasetDetailsRecord.getType().getFieldIndex(
                         InternalDatasetDetails.FILTER_FIELD_NAME);
                 if (filterFieldPos >= 0) {
                     filterField = new ArrayList<String>();
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index ee172cc..061030d 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -122,7 +122,7 @@
             }
             searchKey.add(nestedFieldName);
         }
-        int indexKeyTypeFieldPos = rec.getType().findFieldPosition(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
+        int indexKeyTypeFieldPos = rec.getType().getFieldIndex(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
         IACursor fieldTypeCursor = new ACollectionCursor();
         if (indexKeyTypeFieldPos > 0)
             fieldTypeCursor = ((AOrderedList) rec.getValueByPos(indexKeyTypeFieldPos)).getCursor();
@@ -141,7 +141,7 @@
                 searchKeyType.add(fieldType);
             }
         }
-        int isEnforcedFieldPos = rec.getType().findFieldPosition(INDEX_ISENFORCED_FIELD_NAME);
+        int isEnforcedFieldPos = rec.getType().getFieldIndex(INDEX_ISENFORCED_FIELD_NAME);
         Boolean isEnforcingKeys = false;
         if (isEnforcedFieldPos > 0)
             isEnforcingKeys = ((ABoolean) rec.getValueByPos(isEnforcedFieldPos)).getBoolean();
@@ -151,7 +151,7 @@
                 .getIntegerValue();
         // Check if there is a gram length as well.
         int gramLength = -1;
-        int gramLenPos = rec.getType().findFieldPosition(GRAM_LENGTH_FIELD_NAME);
+        int gramLenPos = rec.getType().getFieldIndex(GRAM_LENGTH_FIELD_NAME);
         if (gramLenPos >= 0) {
             gramLength = ((AInt32) rec.getValueByPos(gramLenPos)).getIntegerValue();
         }
diff --git a/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 55ff32f..e3ed6fb 100644
--- a/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
 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.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
@@ -72,9 +73,9 @@
     private int[] openFieldNameLengths;
 
     private int numberOfOpenFields;
+    private RuntimeRecordTypeInfo recTypeInfo;
 
     public RecordBuilder() {
-
         this.closedPartOutputStream = new ByteArrayOutputStream();
         this.numberOfClosedFields = 0;
 
@@ -91,6 +92,8 @@
         this.openPartOffsetArray = null;
         this.openPartOffsetArraySize = 0;
         this.offsetPosition = 0;
+
+        this.recTypeInfo = new RuntimeRecordTypeInfo();
     }
 
     @Override
@@ -108,6 +111,7 @@
     @Override
     public void reset(ARecordType recType) {
         this.recType = recType;
+        this.recTypeInfo.reset(recType);
         this.closedPartOutputStream.reset();
         this.openPartOutputStream.reset();
         this.numberOfClosedFields = 0;
@@ -178,8 +182,8 @@
     public void addField(IValueReference name, IValueReference value) throws AsterixException {
         if (numberOfOpenFields == openPartOffsets.length) {
             openPartOffsets = Arrays.copyOf(openPartOffsets, openPartOffsets.length + DEFAULT_NUM_OPEN_FIELDS);
-            openFieldNameLengths = Arrays.copyOf(openFieldNameLengths, openFieldNameLengths.length
-                    + DEFAULT_NUM_OPEN_FIELDS);
+            openFieldNameLengths = Arrays.copyOf(openFieldNameLengths,
+                    openFieldNameLengths.length + DEFAULT_NUM_OPEN_FIELDS);
         }
         int fieldNameHashCode;
         try {
@@ -191,7 +195,7 @@
         if (recType != null) {
             int cFieldPos;
             try {
-                cFieldPos = recType.findFieldPosition(name.getByteArray(), name.getStartOffset() + 1,
+                cFieldPos = recTypeInfo.getFieldIndex(name.getByteArray(), name.getStartOffset() + 1,
                         name.getLength() - 1);
             } catch (HyracksDataException e) {
                 throw new AsterixException(e);
@@ -229,8 +233,8 @@
                             openBytes, (int) openPartOffsets[i], openFieldNameLengths[i]) == 0) {
                         String field = utf8SerDer.deserialize(new DataInputStream(new ByteArrayInputStream(openBytes,
                                 (int) openPartOffsets[i], openFieldNameLengths[i])));
-                        throw new AsterixException("Open fields " + (i - 1) + " and " + i
-                                + " have the same field name \"" + field + "\"");
+                        throw new AsterixException(
+                                "Open fields " + (i - 1) + " and " + i + " have the same field name \"" + field + "\"");
                     }
                 }
             }
@@ -239,11 +243,10 @@
             int fieldNameHashCode;
             for (int i = 0; i < numberOfOpenFields; i++) {
                 fieldNameHashCode = (int) (openPartOffsets[i] >> 32);
-                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray, (int) fieldNameHashCode,
-                        offsetPosition);
+                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray, fieldNameHashCode, offsetPosition);
                 int fieldOffset = (int) openPartOffsets[i];
-                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray, fieldOffset + openPartOffset + 4
-                        + openPartOffsetArraySize, offsetPosition + 4);
+                SerializerDeserializerUtil.writeIntToByteArray(openPartOffsetArray,
+                        fieldOffset + openPartOffset + 4 + openPartOffsetArraySize, offsetPosition + 4);
                 offsetPosition += 8;
             }
             recordLength = openPartOffset + 4 + openPartOffsetArraySize + openPartOutputStream.size();
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 801907c..0d1f63f 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -54,8 +54,9 @@
 
     private final AStringSerializerDeserializer aStringSerDer = new AStringSerializerDeserializer();
     private AObjectSerializerDeserializer aObjSerDer = null;
+
     private AObjectSerializerDeserializer getObjSerDer() {
-        if (aObjSerDer == null){
+        if (aObjSerDer == null) {
             aObjSerDer = new AObjectSerializerDeserializer();
         }
         return aObjSerDer;
@@ -84,9 +85,8 @@
                 if (t.getTypeTag() == ATypeTag.UNION) {
                     if (((AUnionType) t).isNullableType()) {
                         t2 = ((AUnionType) recordType.getFieldTypes()[i]).getNullableType();
-                        serializers[i] = AqlSerializerDeserializerProvider.INSTANCE
-                                .getSerializerDeserializer(((AUnionType) recordType.getFieldTypes()[i])
-                                        .getNullableType());
+                        serializers[i] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(
+                                ((AUnionType) recordType.getFieldTypes()[i]).getNullableType());
                     } else {
                         // union .. the general case
                         throw new NotImplementedException();
@@ -132,7 +132,7 @@
                 }
                 for (int fieldId = 0; fieldId < numberOfSchemaFields; fieldId++) {
                     if (hasNullableFields && ((nullBitMap[fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)) {
-                        closedFields[fieldId] = (IAObject) ANull.NULL;
+                        closedFields[fieldId] = ANull.NULL;
                         continue;
                     }
                     closedFields[fieldId] = (IAObject) deserializers[fieldId].deserialize(in);
@@ -250,8 +250,7 @@
                             // the field value is null
                             return 0;
                     }
-                    return AInt32SerializerDeserializer.getInt(serRecord,
-                            (int) (14 + offset + nullBitmapSize + (4 * fieldId)));
+                    return AInt32SerializerDeserializer.getInt(serRecord, 14 + offset + nullBitmapSize + (4 * fieldId));
                 } else {
                     if (nullBitmapSize > 0) {
                         // 9 = tag (1) + record Size (4) + isExpanded (1) +
@@ -260,8 +259,7 @@
                             // the field value is null
                             return 0;
                     }
-                    return AInt32SerializerDeserializer.getInt(serRecord,
-                            (int) (10 + offset + nullBitmapSize + (4 * fieldId)));
+                    return AInt32SerializerDeserializer.getInt(serRecord, 10 + offset + nullBitmapSize + (4 * fieldId));
                 }
             } else
                 return -1;
@@ -273,8 +271,7 @@
                     if ((serRecord[9 + offset + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
                         // the field value is null
                         return 0;
-                return AInt32SerializerDeserializer.getInt(serRecord,
-                        (int) (9 + offset + nullBitmapSize + (4 * fieldId)));
+                return AInt32SerializerDeserializer.getInt(serRecord, 9 + offset + nullBitmapSize + (4 * fieldId));
             } else
                 return -1;
         }
@@ -322,9 +319,8 @@
             if (h == fieldNameHashCode) {
                 fieldOffset = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * mid) + 4);
                 // the utf8 comparator do not require to put the precise length, we can just pass a estimated limit.
-                if (utf8BinaryComparator
-                        .compare(serRecord, fieldOffset, serRecord.length,
-                                fieldName, 1, fieldUtflength + fieldUtfMetaLen) == 0)
+                if (utf8BinaryComparator.compare(serRecord, fieldOffset, serRecord.length, fieldName, 1,
+                        fieldUtflength + fieldUtfMetaLen) == 0)
                     // since they are equal, we can directly use the meta length and the utf length.
                     return fieldOffset + fieldUtfMetaLen + fieldUtflength;
                 else { // this else part has not been tested yet
@@ -332,8 +328,8 @@
                         h = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j));
                         if (h == fieldNameHashCode) {
                             fieldOffset = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j) + 4);
-                            if (utf8BinaryComparator.compare(serRecord, fieldOffset, serRecord.length, fieldName,
-                                    1, fieldUtflength) == 0)
+                            if (utf8BinaryComparator.compare(serRecord, fieldOffset, serRecord.length, fieldName, 1,
+                                    fieldUtflength) == 0)
                                 return fieldOffset + fieldUtfMetaLen + fieldUtflength;
                         } else
                             break;
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java b/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
index 41efb3a..be00153 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
@@ -19,7 +19,6 @@
 
 package org.apache.asterix.om.pointables.cast;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -37,7 +36,6 @@
 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.types.hierachy.ITypeConvertComputer;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 
@@ -56,6 +54,7 @@
 
     private final Map<IVisitablePointable, ARecordCaster> raccessorToCaster = new HashMap<IVisitablePointable, ARecordCaster>();
     private final Map<IVisitablePointable, AListCaster> laccessorToCaster = new HashMap<IVisitablePointable, AListCaster>();
+    private final ArrayBackedValueStorage castBuffer = new ArrayBackedValueStorage();
 
     @Override
     public Void visit(AListVisitablePointable accessor, Triple<IVisitablePointable, IAType, Boolean> arg)
@@ -89,11 +88,7 @@
                 arg.second = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
             }
             ARecordType resultType = (ARecordType) arg.second;
-            //cloning result type to avoid race conditions during comparison\hash calculation
-            ARecordType clonedResultType = new ARecordType(resultType.getTypeName(), resultType.getFieldNames(),
-                    resultType.getFieldTypes(), resultType.isOpen());
-
-            caster.castRecord(accessor, arg.first, clonedResultType, this);
+            caster.castRecord(accessor, arg.first, resultType, this);
         } catch (Exception e) {
             throw new AsterixException(e);
         }
@@ -109,21 +104,20 @@
             return null;
         }
         // set the pointer for result
-        ATypeTag reqTypeTag = ((IAType) (arg.second)).getTypeTag();
-        ATypeTag inputTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(accessor.getByteArray()[accessor
-                .getStartOffset()]);
+        ATypeTag reqTypeTag = (arg.second).getTypeTag();
+        ATypeTag inputTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                .deserialize(accessor.getByteArray()[accessor.getStartOffset()]);
         if (!needPromote(inputTypeTag, reqTypeTag)) {
             arg.first.set(accessor);
         } else {
-            ArrayBackedValueStorage castBuffer = new ArrayBackedValueStorage();
-
             try {
+                castBuffer.reset();
                 ATypeHierarchy.convertNumericTypeByteArray(accessor.getByteArray(), accessor.getStartOffset(),
                         accessor.getLength(), reqTypeTag, castBuffer.getDataOutput());
                 arg.first.set(castBuffer);
             } catch (IOException e1) {
-                throw new AsterixException("Type mismatch: cannot cast the " + inputTypeTag + " type to the "
-                        + reqTypeTag + " type.");
+                throw new AsterixException(
+                        "Type mismatch: cannot cast the " + inputTypeTag + " type to the " + reqTypeTag + " type.");
             }
 
         }
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
index 283037f..2d9cd5d 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
@@ -24,8 +24,6 @@
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.commons.lang3.ArrayUtils;
-
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.types.ARecordType;
@@ -33,6 +31,7 @@
 import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeHelper;
+import org.apache.commons.lang3.ArrayUtils;
 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;
@@ -74,8 +73,8 @@
         ARecordType recType1 = extractRecordType(t1);
 
         if (recType0 == null || recType1 == null) {
-            throw new AlgebricksException("record-merge expects possibly NULL records as arguments, but got (" + t0
-                    + ", " + t1 + ")");
+            throw new AlgebricksException(
+                    "record-merge expects possibly NULL records as arguments, but got (" + t0 + ", " + t1 + ")");
         }
 
         List<String> resultFieldNames = new ArrayList<>();
@@ -145,21 +144,22 @@
 
         for (int i = 0; i < fieldType1Copy.getFieldTypes().length; i++) {
             try {
-                int pos = returnType.findFieldPosition(fieldType1Copy.getFieldNames()[i]);
+                int pos = returnType.getFieldIndex(fieldType1Copy.getFieldNames()[i]);
                 if (pos >= 0) {
                     if (fieldType1Copy.getFieldTypes()[i].getTypeTag() != ATypeTag.RECORD) {
                         break;
                     }
                     IAType[] oldTypes = returnType.getFieldTypes();
-                    oldTypes[pos] = mergedNestedType(fieldType1Copy.getFieldTypes()[i], returnType.getFieldTypes()[pos]);
+                    oldTypes[pos] = mergedNestedType(fieldType1Copy.getFieldTypes()[i],
+                            returnType.getFieldTypes()[pos]);
                     returnType = new ARecordType(returnType.getTypeName(), returnType.getFieldNames(), oldTypes,
                             returnType.isOpen());
                 } else {
                     IAType[] combinedFieldTypes = ArrayUtils.addAll(returnType.getFieldTypes().clone(),
                             fieldType1Copy.getFieldTypes()[i]);
-                    returnType = new ARecordType(returnType.getTypeName(), ArrayUtils.addAll(
-                            returnType.getFieldNames(), fieldType1Copy.getFieldNames()[i]), combinedFieldTypes,
-                            returnType.isOpen());
+                    returnType = new ARecordType(returnType.getTypeName(),
+                            ArrayUtils.addAll(returnType.getFieldNames(), fieldType1Copy.getFieldNames()[i]),
+                            combinedFieldTypes, returnType.isOpen());
                 }
 
             } catch (IOException | AsterixException e) {
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index a1e0739..e65f545 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -19,51 +19,39 @@
 
 package org.apache.asterix.om.types;
 
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
-
-import org.apache.hyracks.util.string.UTF8StringUtil;
-import org.apache.hyracks.util.string.UTF8StringWriter;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
+import java.util.Map;
 
 import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
 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 org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 
+/**
+ * ARecordType is read-only and shared by different partitions at runtime.
+ * Note: to check whether a field name is defined in the closed part at runtime,
+ * please use RuntimeRecordTypeInfo which separates the mutable states
+ * from ARecordType and has to be one-per-partition.
+ */
 public class ARecordType extends AbstractComplexType {
 
     private static final long serialVersionUID = 1L;
-    private String[] fieldNames;
-    private IAType[] fieldTypes;
-    private boolean isOpen;
+    private final String[] fieldNames;
+    private final IAType[] fieldTypes;
+    private final Map<String, Integer> fieldNameToIndexMap = new HashMap<String, Integer>();
+    private final boolean isOpen;
     private final List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
-
-    private transient IBinaryHashFunction fieldNameHashFunction;
-    private transient IBinaryComparator fieldNameComparator;
-    private final byte serializedFieldNames[];
-    private final int serializedFieldNameOffsets[];
-    private final long hashCodeIndexPairs[];
-
-    private final UTF8StringSerializerDeserializer utf8SerDer = new UTF8StringSerializerDeserializer();
 
     /**
      * @param typeName
@@ -78,7 +66,6 @@
      *             if there are duplicate field names or if there is an error serializing the field names
      * @throws HyracksDataException
      */
-    @SuppressWarnings("resource")
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen)
             throws AsterixException, HyracksDataException {
         super(typeName);
@@ -86,91 +73,10 @@
         this.fieldTypes = fieldTypes;
         this.isOpen = isOpen;
 
-        fieldNameComparator = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY)
-                .createBinaryComparator();
-        fieldNameHashFunction = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
-                .createBinaryHashFunction();
-        ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
-        DataOutputStream dos = new DataOutputStream(baaos);
-        UTF8StringWriter writer = new UTF8StringWriter();
-        serializedFieldNameOffsets = new int[fieldNames.length];
-        hashCodeIndexPairs = new long[fieldNames.length];
-
-        int length = 0;
-        for (int i = 0; i < fieldNames.length; i++) {
-            serializedFieldNameOffsets[i] = baaos.size();
-            try {
-                writer.writeUTF8(fieldNames[i], dos);
-            } catch (IOException e) {
-                throw new AsterixException(e);
-            }
-            length = baaos.size() - serializedFieldNameOffsets[i];
-            hashCodeIndexPairs[i] = fieldNameHashFunction.hash(baaos.getByteArray(), serializedFieldNameOffsets[i],
-                    length);
-            hashCodeIndexPairs[i] = hashCodeIndexPairs[i] << 32;
-            hashCodeIndexPairs[i] = hashCodeIndexPairs[i] | i;
+        // Puts field names to the field name to field index map.
+        for (int index = 0; index < fieldNames.length; ++index) {
+            fieldNameToIndexMap.put(fieldNames[index], index);
         }
-        try {
-            dos.close();
-        } catch (IOException e) {
-            throw new AsterixException(e);
-        }
-        serializedFieldNames = baaos.getByteArray();
-
-        Arrays.sort(hashCodeIndexPairs);
-        int j;
-        for (int i = 0; i < fieldNames.length; i++) {
-            j = findFieldPosition(serializedFieldNames, serializedFieldNameOffsets[i],
-                    UTF8StringUtil.getStringLength(serializedFieldNames, serializedFieldNameOffsets[i]));
-            if (j != i) {
-                throw new AsterixException("Closed fields " + j + " and " + i + " have the same field name \""
-                        + fieldNames[i] + "\"");
-            }
-        }
-    }
-
-    private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
-        ois.defaultReadObject();
-        fieldNameComparator = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY)
-                .createBinaryComparator();
-        fieldNameHashFunction = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
-                .createBinaryHashFunction();
-    }
-
-    /**
-     * Returns the position of the field in the closed schema or -1 if the field does not exist.
-     *
-     * @param bytes
-     *            the serialized bytes of the field name
-     * @param start
-     *            the starting offset of the field name in bytes
-     * @param length
-     *            the length of the field name in bytes
-     * @return the position of the field in the closed schema or -1 if the field does not exist.
-     * @throws HyracksDataException
-     */
-    public int findFieldPosition(byte[] bytes, int start, int length) throws HyracksDataException {
-        if (hashCodeIndexPairs.length == 0) {
-            return -1;
-        }
-
-        int fIndex;
-        int probeFieldHash = fieldNameHashFunction.hash(bytes, start, length);
-        int i = Arrays.binarySearch(hashCodeIndexPairs, ((long) probeFieldHash) << 32);
-        i = (i < 0) ? (i = -1 * (i + 1)) : i;
-
-        while (i < hashCodeIndexPairs.length && (int) (hashCodeIndexPairs[i] >>> 32) == probeFieldHash) {
-            fIndex = (int) hashCodeIndexPairs[i];
-            int cFieldLength = UTF8StringUtil.getStringLength(serializedFieldNames,
-                    serializedFieldNameOffsets[fIndex]);
-            if (fieldNameComparator.compare(serializedFieldNames, serializedFieldNameOffsets[fIndex], cFieldLength,
-                    bytes, start, length) == 0) {
-                return fIndex;
-            }
-            i++;
-        }
-
-        return -1;
     }
 
     public final String[] getFieldNames() {
@@ -223,11 +129,16 @@
      *            the name of the field whose position is sought
      * @return the position of the field in the closed schema or -1 if the field does not exist.
      */
-    public int findFieldPosition(String fieldName) throws IOException {
-        ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
-        DataOutputStream dos = new DataOutputStream(baaos);
-        utf8SerDer.serialize(fieldName, dos);
-        return findFieldPosition(baaos.getByteArray(), 0, baaos.getByteArray().length);
+    public int getFieldIndex(String fieldName) throws IOException {
+        if (fieldNames == null) {
+            return -1;
+        }
+        Integer index = fieldNameToIndexMap.get(fieldName);
+        if (index == null) {
+            return -1;
+        } else {
+            return index;
+        }
     }
 
     /**
@@ -263,8 +174,8 @@
                 //enforced SubType
                 subRecordType = ((AUnionType) subRecordType).getNullableType();
                 if (subRecordType.getTypeTag().serialize() != ATypeTag.RECORD.serialize()) {
-                    throw new IOException("Field accessor is not defined for values of type "
-                            + subRecordType.getTypeTag());
+                    throw new IOException(
+                            "Field accessor is not defined for values of type " + subRecordType.getTypeTag());
                 }
 
             }
@@ -283,7 +194,7 @@
      *             if an error occurs while serializing the field name
      */
     public IAType getFieldType(String fieldName) throws IOException {
-        int fieldPos = findFieldPosition(fieldName);
+        int fieldPos = getFieldIndex(fieldName);
         if (fieldPos < 0 || fieldPos >= fieldTypes.length) {
             return null;
         }
@@ -299,238 +210,7 @@
      * @throws IOException
      */
     public boolean isClosedField(String fieldName) throws IOException {
-        return findFieldPosition(fieldName) != -1;
-    }
-
-    /**
-     * Validates the partitioning expression that will be used to partition a dataset and returns expression type.
-     *
-     * @param partitioningExprs
-     *            a list of partitioning expressions that will be validated
-     * @return a list of partitioning expressions types
-     * @throws AlgebricksException
-     *             (if the validation failed), IOException
-     */
-    public List<IAType> validatePartitioningExpressions(List<List<String>> partitioningExprs, boolean autogenerated)
-            throws AsterixException, IOException {
-        List<IAType> partitioningExprTypes = new ArrayList<IAType>(partitioningExprs.size());
-        if (autogenerated) {
-            if (partitioningExprs.size() > 1) {
-                throw new AsterixException("Cannot autogenerate a composite primary key");
-            }
-            List<String> fieldName = partitioningExprs.get(0);
-            IAType fieldType = getSubFieldType(fieldName);
-            partitioningExprTypes.add(fieldType);
-
-            ATypeTag pkTypeTag = fieldType.getTypeTag();
-            if (pkTypeTag != ATypeTag.UUID) {
-                throw new AsterixException("Cannot autogenerate a primary key for type " + pkTypeTag
-                        + ". Autogenerated primary keys must be of type " + ATypeTag.UUID + ".");
-            }
-        } else {
-            for (int i = 0; i < partitioningExprs.size(); i++) {
-                List<String> fieldName = partitioningExprs.get(i);
-                IAType fieldType = getSubFieldType(fieldName);
-
-                switch (fieldType.getTypeTag()) {
-                    case INT8:
-                    case INT16:
-                    case INT32:
-                    case INT64:
-                    case FLOAT:
-                    case DOUBLE:
-                    case STRING:
-                    case BINARY:
-                    case DATE:
-                    case TIME:
-                    case UUID:
-                    case DATETIME:
-                    case YEARMONTHDURATION:
-                    case DAYTIMEDURATION:
-                        partitioningExprTypes.add(fieldType);
-                        break;
-                    case UNION:
-                        throw new AsterixException("The partitioning key \"" + fieldName + "\" cannot be nullable");
-                    default:
-                        throw new AsterixException("The partitioning key \"" + fieldName + "\" cannot be of type "
-                                + fieldType.getTypeTag() + ".");
-                }
-            }
-        }
-        return partitioningExprTypes;
-    }
-
-    private IAType getPartitioningExpressionType(String fieldName, boolean autogenerated) throws AsterixException,
-            IOException {
-        IAType fieldType = getFieldType(fieldName);
-        if (fieldType == null) {
-            if (autogenerated) {
-                throw new AsterixException("Primary key field: " + fieldName
-                        + " should be defined in the type that the dataset is using.");
-            } else {
-                throw new AsterixException("Primary key field: " + fieldName + " could not be found.");
-            }
-        }
-        return fieldType;
-    }
-
-    /**
-     * Validates the key fields that will be used as keys of an index.
-     *
-     * @param keyFieldNames
-     *            a map of key fields that will be validated
-     * @param keyFieldTypes
-     *            a map of key types (if provided) that will be validated
-     * @param indexType
-     *            the type of the index that its key fields is being validated
-     * @throws AlgebricksException
-     *             (if the validation failed), IOException
-     */
-    public void validateKeyFields(List<List<String>> keyFieldNames, List<IAType> keyFieldTypes, IndexType indexType)
-            throws AlgebricksException, IOException {
-        int pos = 0;
-        boolean openFieldCompositeIdx = false;
-        for (List<String> fieldName : keyFieldNames) {
-            IAType fieldType = getSubFieldType(fieldName);
-            if (fieldType == null) {
-                fieldType = keyFieldTypes.get(pos);
-                if (keyFieldTypes.get(pos) == BuiltinType.ANULL)
-                    throw new AlgebricksException("A field with this name  \"" + fieldName + "\" could not be found.");
-            } else if (openFieldCompositeIdx)
-                throw new AlgebricksException("A closed field \"" + fieldName
-                        + "\" could be only in a prefix part of the composite index, containing opened field.");
-            if (keyFieldTypes.get(pos) != BuiltinType.ANULL
-                    && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag())
-                throw new AlgebricksException("A field \"" + fieldName + "\" is already defined with the type \""
-                        + fieldType + "\"");
-            switch (indexType) {
-                case BTREE:
-                    switch (fieldType.getTypeTag()) {
-                        case INT8:
-                        case INT16:
-                        case INT32:
-                        case INT64:
-                        case FLOAT:
-                        case DOUBLE:
-                        case STRING:
-                        case BINARY:
-                        case DATE:
-                        case TIME:
-                        case DATETIME:
-                        case UNION:
-                        case UUID:
-                        case YEARMONTHDURATION:
-                        case DAYTIMEDURATION:
-                            break;
-                        default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag() + " cannot be indexed using the BTree index.");
-                    }
-                    break;
-                case RTREE:
-                    switch (fieldType.getTypeTag()) {
-                        case POINT:
-                        case LINE:
-                        case RECTANGLE:
-                        case CIRCLE:
-                        case POLYGON:
-                        case UNION:
-                            break;
-                        default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag() + " cannot be indexed using the RTree index.");
-                    }
-                    break;
-                case LENGTH_PARTITIONED_NGRAM_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case UNION:
-                            break;
-                        default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag()
-                                    + " cannot be indexed using the Length Partitioned N-Gram index.");
-                    }
-                    break;
-                case LENGTH_PARTITIONED_WORD_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case UNORDEREDLIST:
-                        case ORDEREDLIST:
-                        case UNION:
-                            break;
-                        default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag()
-                                    + " cannot be indexed using the Length Partitioned Keyword index.");
-                    }
-                    break;
-                case SINGLE_PARTITION_NGRAM_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case UNION:
-                            break;
-                        default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag() + " cannot be indexed using the N-Gram index.");
-                    }
-                    break;
-                case SINGLE_PARTITION_WORD_INVIX:
-                    switch (fieldType.getTypeTag()) {
-                        case STRING:
-                        case UNORDEREDLIST:
-                        case ORDEREDLIST:
-                        case UNION:
-                            break;
-                        default:
-                            throw new AlgebricksException("The field \"" + fieldName + "\" which is of type "
-                                    + fieldType.getTypeTag() + " cannot be indexed using the Keyword index.");
-                    }
-                    break;
-                default:
-                    throw new AlgebricksException("Invalid index type: " + indexType + ".");
-            }
-            pos++;
-        }
-    }
-
-    /**
-     * Validates the field that will be used as filter for the components of an LSM index.
-     *
-     * @param keyFieldNames
-     *            a list of key fields that will be validated
-     * @param indexType
-     *            the type of the index that its key fields is being validated
-     * @throws AlgebricksException
-     *             (if the validation failed), IOException
-     */
-    public void validateFilterField(List<String> filterField) throws AlgebricksException, IOException {
-        IAType fieldType = getSubFieldType(filterField);
-        if (fieldType == null) {
-            throw new AlgebricksException("A field with this name  \"" + filterField + "\" could not be found.");
-        }
-        switch (fieldType.getTypeTag()) {
-            case INT8:
-            case INT16:
-            case INT32:
-            case INT64:
-            case FLOAT:
-            case DOUBLE:
-            case STRING:
-            case BINARY:
-            case DATE:
-            case TIME:
-            case DATETIME:
-            case UUID:
-            case YEARMONTHDURATION:
-            case DAYTIMEDURATION:
-                break;
-            case UNION:
-                throw new AlgebricksException("The filter field \"" + filterField + "\" cannot be nullable");
-            default:
-                throw new AlgebricksException("The field \"" + filterField + "\" which is of type "
-                        + fieldType.getTypeTag() + " cannot be used as a filter for a dataset.");
-        }
+        return getFieldIndex(fieldName) != -1;
     }
 
     public boolean doesFieldExist(String fieldName) {
@@ -599,7 +279,7 @@
     public int hash() {
         int h = 0;
         for (int i = 0; i < fieldNames.length; i++) {
-            h += 31 * h + (int) (hashCodeIndexPairs[i] >> 32);
+            h += 31 * h + fieldNames[i].hashCode();
         }
         for (int i = 0; i < fieldTypes.length; i++) {
             h += 31 * h + fieldTypes[i].hashCode();
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java b/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
new file mode 100644
index 0000000..07773d8
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.types.runtime;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+
+/**
+ * This class contains the mutable states for a record type
+ * so as to allow a caller to check whether a field name
+ * is in the closed part of the record type.
+ * The RuntimeRecordTypeInfo has to be one-per-partition
+ * to avoid race conditions.
+ */
+public class RuntimeRecordTypeInfo {
+
+    private final IBinaryHashFunction fieldNameHashFunction;
+    private final IBinaryComparator fieldNameComparator;
+    private final UTF8StringWriter writer;
+    private final ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
+    private final DataOutputStream dos = new DataOutputStream(baaos);
+
+    private int[] serializedFieldNameOffsets;
+    private long[] hashCodeIndexPairs;
+    private ARecordType cachedRecType = null;
+
+    public RuntimeRecordTypeInfo() {
+        fieldNameComparator = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY)
+                .createBinaryComparator();
+        fieldNameHashFunction = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
+                .createBinaryHashFunction();
+        writer = new UTF8StringWriter();
+    }
+
+    /**
+     * Reset the binary artifacts of a runtime type info instance.
+     *
+     * @param recType,
+     *            the record type.
+     */
+    public void reset(ARecordType recType) {
+        if (cachedRecType == recType) {
+            // if the type doesn't change, we just skip the reset.
+            return;
+        }
+        // Sets the record type.
+        cachedRecType = recType;
+        // Resets the bytes for names.
+        baaos.reset();
+
+        if (recType != null) {
+            String[] fieldNames = recType.getFieldNames();
+            if (serializedFieldNameOffsets == null || serializedFieldNameOffsets.length != fieldNames.length) {
+                serializedFieldNameOffsets = new int[fieldNames.length];
+                hashCodeIndexPairs = new long[fieldNames.length];
+            }
+            int length = 0;
+            try {
+                for (int i = 0; i < fieldNames.length; ++i) {
+                    serializedFieldNameOffsets[i] = baaos.size();
+                    writer.writeUTF8(fieldNames[i], dos);
+                    length = baaos.size() - serializedFieldNameOffsets[i];
+                    hashCodeIndexPairs[i] = fieldNameHashFunction.hash(baaos.getByteArray(),
+                            serializedFieldNameOffsets[i], length);
+                    hashCodeIndexPairs[i] = hashCodeIndexPairs[i] << 32;
+                    hashCodeIndexPairs[i] = hashCodeIndexPairs[i] | i;
+                }
+                dos.flush();
+                Arrays.sort(hashCodeIndexPairs);
+                for (int i = 0; i < fieldNames.length; i++) {
+                    int j = getFieldIndex(baaos.getByteArray(), serializedFieldNameOffsets[i],
+                            UTF8StringUtil.getStringLength(baaos.getByteArray(), serializedFieldNameOffsets[i]));
+                    if (j != i) {
+                        throw new IllegalStateException("Closed fields " + j + " and " + i
+                                + " have the same field name \"" + fieldNames[i] + "\"");
+                    }
+                }
+            } catch (IOException e) {
+                throw new IllegalStateException(e);
+            }
+        } else {
+            serializedFieldNameOffsets = new int[0];
+            hashCodeIndexPairs = new long[0];
+        }
+    }
+
+    /**
+     * Returns the position of the field in the closed schema or -1 if the field does not exist.
+     *
+     * @param bytes
+     *            the serialized bytes of the field name
+     * @param start
+     *            the starting offset of the field name in bytes
+     * @param length
+     *            the length of the field name in bytes
+     * @return the position of the field in the closed schema or -1 if the field does not exist.
+     * @throws HyracksDataException
+     */
+    public int getFieldIndex(byte[] bytes, int start, int length) throws HyracksDataException {
+        if (hashCodeIndexPairs.length == 0) {
+            return -1;
+        }
+        int fIndex;
+        int probeFieldHash = fieldNameHashFunction.hash(bytes, start, length);
+        int i = Arrays.binarySearch(hashCodeIndexPairs, ((long) probeFieldHash) << 32);
+        i = (i < 0) ? (i = -1 * (i + 1)) : i;
+
+        while (i < hashCodeIndexPairs.length && (int) (hashCodeIndexPairs[i] >>> 32) == probeFieldHash) {
+            fIndex = (int) hashCodeIndexPairs[i];
+            int cFieldLength = UTF8StringUtil.getStringLength(baaos.getByteArray(), serializedFieldNameOffsets[fIndex]);
+            if (fieldNameComparator.compare(baaos.getByteArray(), serializedFieldNameOffsets[fIndex], cFieldLength,
+                    bytes, start, length) == 0) {
+                return fIndex;
+            }
+            i++;
+        }
+        return -1;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index 883be0a..2f41816 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -22,6 +22,7 @@
 import java.util.List;
 
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
@@ -50,22 +51,27 @@
     public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
         return new ICopyEvaluator() {
 
-            private DataOutput out = output.getDataOutput();
-            private ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
+            private final DataOutput out = output.getDataOutput();
+            private final ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
 
-            private ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
-            private ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
-            private ArrayBackedValueStorage[] abvsFields = new ArrayBackedValueStorage[fieldPath.size()];
-            private DataOutput[] doFields = new DataOutput[fieldPath.size()];
+            private final ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
+            private final ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
+            private final ArrayBackedValueStorage[] abvsFields = new ArrayBackedValueStorage[fieldPath.size()];
+            private final DataOutput[] doFields = new DataOutput[fieldPath.size()];
+            private final RuntimeRecordTypeInfo[] recTypeInfos = new RuntimeRecordTypeInfo[fieldPath.size()];
 
             {
                 FieldAccessUtil.getFieldsAbvs(abvsFields, doFields, fieldPath);
-                recordType = recordType.deepCopy(recordType);
+                for (int index = 0; index < fieldPath.size(); ++index) {
+                    recTypeInfos[index] = new RuntimeRecordTypeInfo();
+                }
+
             }
 
             @Override
             public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
-                FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, recordType);
+                FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, recordType,
+                        recTypeInfos);
             }
         };
     }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
index db8a6b4..a442ef4 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
@@ -34,9 +34,10 @@
 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.runtime.RuntimeRecordTypeInfo;
 import org.apache.asterix.om.util.NonTaggedFormatUtil;
+import org.apache.commons.lang.NotImplementedException;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -89,8 +90,8 @@
 
     public static void evaluate(IFrameTupleReference tuple, DataOutput out, ICopyEvaluator eval0,
             ArrayBackedValueStorage[] abvsFields, ArrayBackedValueStorage abvsRecord,
-            ByteArrayAccessibleOutputStream subRecordTmpStream, ARecordType recordType) throws AlgebricksException {
-
+            ByteArrayAccessibleOutputStream subRecordTmpStream, ARecordType recordType,
+            RuntimeRecordTypeInfo[] recTypeInfos) throws AlgebricksException {
         try {
             abvsRecord.reset();
             eval0.evaluate(tuple);
@@ -99,7 +100,10 @@
             int subFieldOffset = -1;
             int subFieldLength = -1;
             int nullBitmapSize = -1;
+
             IAType subType = recordType;
+            recTypeInfos[0].reset(recordType);
+
             ATypeTag subTypeTag = ATypeTag.NULL;
             byte[] subRecord = abvsRecord.getByteArray();
             boolean openField = false;
@@ -117,9 +121,11 @@
                     if (subType.getTypeTag().serialize() != SER_RECORD_TYPE_TAG) {
                         throw new AlgebricksException("Field accessor is not defined for values of type " + subTypeTag);
                     }
-
+                    if (subType.getTypeTag() == ATypeTag.RECORD) {
+                        recTypeInfos[i].reset((ARecordType) subType);
+                    }
                 }
-                subFieldIndex = ((ARecordType) subType).findFieldPosition(abvsFields[i].getByteArray(),
+                subFieldIndex = recTypeInfos[i].getFieldIndex(abvsFields[i].getByteArray(),
                         abvsFields[i].getStartOffset() + 1, abvsFields[i].getLength());
                 if (subFieldIndex == -1) {
                     break;
@@ -133,6 +139,10 @@
                     return;
                 }
                 subType = ((ARecordType) subType).getFieldTypes()[subFieldIndex];
+                if (subType.getTypeTag() == ATypeTag.RECORD && i + 1 < abvsFields.length) {
+                    // Move to the next Depth
+                    recTypeInfos[i + 1].reset((ARecordType) subType);
+                }
                 if (subType.getTypeTag().equals(ATypeTag.UNION)) {
                     if (((AUnionType) subType).isNullableType()) {
                         subTypeTag = ((AUnionType) subType).getNullableType().getTypeTag();
@@ -172,7 +182,8 @@
                 }
 
                 subTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(subRecord[subFieldOffset]);
-                subFieldLength = NonTaggedFormatUtil.getFieldValueLength(subRecord, subFieldOffset, subTypeTag, true) + 1;
+                subFieldLength = NonTaggedFormatUtil.getFieldValueLength(subRecord, subFieldOffset, subTypeTag, true)
+                        + 1;
 
                 if (i < abvsFields.length - 1) {
                     //setup next iteration
@@ -187,7 +198,6 @@
                 out.writeByte(subTypeTag.serialize());
             }
             out.write(subRecord, subFieldOffset, subFieldLength);
-
         } catch (IOException e) {
             throw new AlgebricksException(e);
         } catch (AsterixException e) {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 05e7292..8cabee1 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -26,6 +26,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.runtime.RuntimeRecordTypeInfo;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
@@ -56,26 +57,29 @@
     public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
         return new ICopyEvaluator() {
 
-            private DataOutput out = output.getDataOutput();
-            private ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
+            private final DataOutput out = output.getDataOutput();
+            private final ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
 
-            private ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
-            private ArrayBackedValueStorage outInput1 = new ArrayBackedValueStorage();
-            private ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
-            private ICopyEvaluator eval1 = fldNameEvalFactory.createEvaluator(outInput1);
+            private final ArrayBackedValueStorage outInput0 = new ArrayBackedValueStorage();
+            private final ArrayBackedValueStorage outInput1 = new ArrayBackedValueStorage();
+            private final ICopyEvaluator eval0 = recordEvalFactory.createEvaluator(outInput0);
+            private final ICopyEvaluator eval1 = fldNameEvalFactory.createEvaluator(outInput1);
 
-            int size = 1;
-            private ArrayBackedValueStorage abvsFields[] = new ArrayBackedValueStorage[size];
-            private DataOutput[] doFields = new DataOutput[size];
+            private final int size = 1;
+            private final ArrayBackedValueStorage abvsFields[] = new ArrayBackedValueStorage[size];
+            private final DataOutput[] doFields = new DataOutput[size];
 
             @SuppressWarnings("unchecked")
-            private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+            private final ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
                     .getSerializerDeserializer(BuiltinType.ANULL);
+            private final RuntimeRecordTypeInfo[] recTypeInfos = new RuntimeRecordTypeInfo[size];
 
-            private ARecordType mRecordType = recordType.deepCopy(recordType);
             {
                 abvsFields[0] = new ArrayBackedValueStorage();
                 doFields[0] = abvsFields[0].getDataOutput();
+                for (int index = 0; index < size; ++index) {
+                    recTypeInfos[index] = new RuntimeRecordTypeInfo();
+                }
             }
 
             @Override
@@ -92,7 +96,8 @@
                     abvsFields[0].reset();
                     doFields[0].write(serFldName);
 
-                    FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, mRecordType);
+                    FieldAccessUtil.evaluate(tuple, out, eval0, abvsFields, outInput0, subRecordTmpStream, recordType,
+                            recTypeInfos);
                 } catch (IOException e) {
                     throw new AlgebricksException(e);
                 }
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
index 3fc5db1..d8bafc0 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
@@ -69,8 +69,7 @@
             private DataOutput out = output.getDataOutput();
             private RecordFieldsUtil rfu = new RecordFieldsUtil();
 
-            protected ARecordType mRecordType = recordType.deepCopy(recordType);
-
+            @Override
             public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
                 outInput0.reset();
                 eval0.evaluate(tuple);
@@ -91,7 +90,7 @@
                 recordPointable.set(outInput0.getByteArray(), outInput0.getStartOffset(), outInput0.getLength());
 
                 try {
-                    rfu.processRecord(recordPointable, mRecordType, out, 0);
+                    rfu.processRecord(recordPointable, recordType, out, 0);
                 } catch (IOException e) {
                     e.printStackTrace();
                 } catch (AsterixException e) {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
index 76716b1..bbba625 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
@@ -176,8 +176,8 @@
         orderedListBuilder.write(out, true);
     }
 
-    public void addNameField(IValueReference nameArg, IARecordBuilder fieldRecordBuilder) throws HyracksDataException,
-            AsterixException {
+    public void addNameField(IValueReference nameArg, IARecordBuilder fieldRecordBuilder)
+            throws HyracksDataException, AsterixException {
         ArrayBackedValueStorage fieldAbvs = getTempBuffer();
 
         fieldAbvs.reset();
@@ -185,8 +185,8 @@
         fieldRecordBuilder.addField(fieldAbvs, nameArg);
     }
 
-    public void addFieldType(byte tagId, IARecordBuilder fieldRecordBuilder) throws HyracksDataException,
-            AsterixException {
+    public void addFieldType(byte tagId, IARecordBuilder fieldRecordBuilder)
+            throws HyracksDataException, AsterixException {
         ArrayBackedValueStorage fieldAbvs = getTempBuffer();
         ArrayBackedValueStorage valueAbvs = getTempBuffer();
 
@@ -202,8 +202,8 @@
         fieldRecordBuilder.addField(fieldAbvs, valueAbvs);
     }
 
-    public void addIsOpenField(boolean isOpen, IARecordBuilder fieldRecordBuilder) throws HyracksDataException,
-            AsterixException {
+    public void addIsOpenField(boolean isOpen, IARecordBuilder fieldRecordBuilder)
+            throws HyracksDataException, AsterixException {
         ArrayBackedValueStorage fieldAbvs = getTempBuffer();
         ArrayBackedValueStorage valueAbvs = getTempBuffer();
 
@@ -246,9 +246,9 @@
         valueAbvs.reset();
         ARecordType newType;
         if (fieldType == null) {
-            newType = openType.deepCopy(openType);
+            newType = openType;
         } else {
-            newType = ((ARecordType) fieldType).deepCopy((ARecordType) fieldType);
+            newType = (ARecordType) fieldType;
         }
         ARecordPointable recordP = getRecordPointable();
         recordP.set(recordArg);
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
index df52f90..cdecce5 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
@@ -66,6 +66,7 @@
     private static final byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new RecordMergeDescriptor();
         }
@@ -112,15 +113,13 @@
                 final ICopyEvaluator eval0 = args[0].createEvaluator(abvs0);
                 final ICopyEvaluator eval1 = args[1].createEvaluator(abvs1);
 
-                final Stack<RecordBuilder> rbStack = new Stack<RecordBuilder>();
-
-                final ArrayBackedValueStorage tabvs = new ArrayBackedValueStorage();
-
-                final ByteArrayAccessibleOutputStream nameOutputStream = new ByteArrayAccessibleOutputStream();
-                final ByteArrayInputStream namebais = new ByteArrayInputStream(nameOutputStream.getByteArray());
-                final DataInputStream namedis = new DataInputStream(namebais);
-
                 return new ICopyEvaluator() {
+                    private final Stack<RecordBuilder> rbStack = new Stack<RecordBuilder>();
+                    private final ArrayBackedValueStorage tabvs = new ArrayBackedValueStorage();
+                    private final ByteArrayAccessibleOutputStream nameOutputStream = new ByteArrayAccessibleOutputStream();
+                    private final ByteArrayInputStream namebais = new ByteArrayInputStream(
+                            nameOutputStream.getByteArray());
+                    private final DataInputStream namedis = new DataInputStream(namebais);
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -156,8 +155,8 @@
                     }
 
                     private void mergeFields(ARecordType combinedType, ARecordVisitablePointable leftRecord,
-                            ARecordVisitablePointable rightRecord, boolean openFromParent, int nestedLevel) throws IOException,
-                            AsterixException, AlgebricksException {
+                            ARecordVisitablePointable rightRecord, boolean openFromParent, int nestedLevel)
+                                    throws IOException, AsterixException, AlgebricksException {
                         if (rbStack.size() < (nestedLevel + 1)) {
                             rbStack.push(new RecordBuilder());
                         }
@@ -225,13 +224,13 @@
 
                         //Add the merged field
                         if (combinedType.isClosedField(fieldName)) {
-                            int pos = combinedType.findFieldPosition(fieldName);
+                            int pos = combinedType.getFieldIndex(fieldName);
                             if (rightValue == null) {
                                 rbStack.get(nestedLevel).addField(pos, leftValue);
                             } else {
                                 mergeFields((ARecordType) combinedType.getFieldType(fieldName),
-                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue, false,
-                                        nestedLevel + 1);
+                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue,
+                                        false, nestedLevel + 1);
                                 tabvs.reset();
                                 rbStack.get(nestedLevel + 1).write(tabvs.getDataOutput(), true);
                                 rbStack.get(nestedLevel).addField(pos, tabvs);
@@ -241,8 +240,8 @@
                                 rbStack.get(nestedLevel).addField(fieldNamePointable, leftValue);
                             } else {
                                 mergeFields((ARecordType) combinedType.getFieldType(fieldName),
-                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue, false,
-                                        nestedLevel + 1);
+                                        (ARecordVisitablePointable) leftValue, (ARecordVisitablePointable) rightValue,
+                                        false, nestedLevel + 1);
                                 tabvs.reset();
                                 rbStack.get(nestedLevel + 1).write(tabvs.getDataOutput(), true);
                                 rbStack.get(nestedLevel).addField(fieldNamePointable, tabvs);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Heri Ramampiaro <he...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/501

to look at the new patch set (#5).

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................

ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.

Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
---
M asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
M asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
M asterix-app/src/test/resources/runtimets/queries_sqlpp/records/RecordsQueries.xml
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
M asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
A asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
22 files changed, 715 insertions(+), 492 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/01/501/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has uploaded a new patch set (#2).

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................

ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.

Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
---
M asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
M asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
A asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
19 files changed, 307 insertions(+), 268 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/01/501/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 1: -Verified

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/421/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 3: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/423/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/532/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/429/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Heri Ramampiaro <he...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/423/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 2: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/422/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 6: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/429/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Heri Ramampiaro <he...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Preston Carman has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 4:

(1 comment)

Does this change also allow runtimets/queries/records/get-record-fields test to be enabled? The test had sporadic failures on jenkins. If so, it would be nice to include it in the patch.

https://asterix-gerrit.ics.uci.edu/#/c/501/4/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
File asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java:

Line 71:             // if the time doesn't change, we just skip the reset.
time = type


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 5: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/426/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Heri Ramampiaro <he...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 1: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/532/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/422/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 6: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Heri Ramampiaro <he...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 1: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/421/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/501

to look at the new patch set (#4).

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................

ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.

Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
---
M asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
M asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
M asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
A asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
21 files changed, 582 insertions(+), 489 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/01/501/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 4:

(1 comment)

>>Does this change also allow runtimets/queries/records/get-record-fields test to be enabled? The test had sporadic failures on jenkins. If so, it would be nice to include it in the patch.

@Preston, with this patch, runtimets/queries/records/get-record-fields is not sporadic --- it deterministically fails.  I think you can continue to investigate that test case once this patch is merged.

https://asterix-gerrit.ics.uci.edu/#/c/501/4/asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
File asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java:

Line 71:             // if the time doesn't change, we just skip the reset.
> time = type
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/426/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................


Patch Set 4: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-topic/424/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/501

to look at the new patch set (#6).

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................

ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.

Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
---
M asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
M asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
M asterix-app/src/test/resources/runtimets/queries_sqlpp/records/RecordsQueries.xml
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
M asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
A asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
22 files changed, 715 insertions(+), 492 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/01/501/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Heri Ramampiaro <he...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1159: make ARecordType read-only and separate its ...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/501

to look at the new patch set (#3).

Change subject: ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.
......................................................................

ASTERIXDB-1159: make ARecordType read-only and separate its mutable state to RuntimeRecordTypeInfo.

Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
---
M asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ByNameToByIndexFieldAccessRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
M asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
M asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
M asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
M asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterix-om/src/main/java/org/apache/asterix/om/pointables/cast/ACastVisitor.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordMergeTypeComputer.java
M asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
A asterix-om/src/main/java/org/apache/asterix/om/types/runtime/RuntimeRecordTypeInfo.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldsEvalFactory.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordMergeDescriptor.java
21 files changed, 576 insertions(+), 481 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/01/501/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/501
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbfa73907ae9422a4ad85dc4db5fa568855beb17
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>