You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org> on 2019/01/08 18:07:37 UTC

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

Murtadha Hubail has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................

[ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Reuse UTF8 hash function and comparator from
  evaluators when extracting a field's offset.

Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
4 files changed, 33 insertions(+), 15 deletions(-)


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

diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 97c49e0..2685e48 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -305,8 +305,8 @@
         return offset + AInt32SerializerDeserializer.getInt(serRecord, pointer + nullBitmapSize + (4 * fieldId));
     }
 
-    public static int getFieldOffsetByName(byte[] serRecord, int start, int len, byte[] fieldName, int nstart)
-            throws HyracksDataException {
+    public static int getFieldOffsetByName(byte[] serRecord, int start, int len, byte[] fieldName, int nstart,
+            IBinaryHashFunction nameHashFunction, IBinaryComparator nameComparator) throws HyracksDataException {
         // 5 is the index of the byte that determines whether the record is expanded or not, i.e. it has an open part.
         if (hasNoFields(serRecord, start, len) || serRecord[start + 5] != 1) {
             return -1;
@@ -316,14 +316,7 @@
         int numberOfOpenField = AInt32SerializerDeserializer.getInt(serRecord, openPartOffset);
         int fieldUtflength = UTF8StringUtil.getUTFLength(fieldName, nstart + 1);
         int fieldUtfMetaLen = UTF8StringUtil.getNumBytesToStoreLength(fieldUtflength);
-
-        IBinaryHashFunction utf8HashFunction =
-                BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
-
-        IBinaryComparator utf8BinaryComparator =
-                BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
-
-        int fieldNameHashCode = utf8HashFunction.hash(fieldName, nstart + 1, fieldUtflength + fieldUtfMetaLen);
+        int fieldNameHashCode = nameHashFunction.hash(fieldName, nstart + 1, fieldUtflength + fieldUtfMetaLen);
 
         int offset = openPartOffset + 4;
         int fieldOffset = -1;
@@ -337,7 +330,7 @@
             if (h == fieldNameHashCode) {
                 fieldOffset = start + 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, len, fieldName, nstart + 1,
+                if (nameComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
                         fieldUtflength + fieldUtfMetaLen) == 0) {
                     // since they are equal, we can directly use the meta length and the utf length.
                     return fieldOffset + fieldUtfMetaLen + fieldUtflength;
@@ -346,7 +339,7 @@
                         h = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j));
                         if (h == fieldNameHashCode) {
                             fieldOffset = start + AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j) + 4);
-                            if (utf8BinaryComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
+                            if (nameComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
                                     fieldUtflength) == 0) {
                                 return fieldOffset + fieldUtfMetaLen + fieldUtflength;
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
index 3deef73..e66cc52 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
@@ -22,6 +22,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
@@ -29,6 +31,8 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -55,6 +59,10 @@
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private DataOutput out = resultStorage.getDataOutput();
 
@@ -83,7 +91,7 @@
                     byte[] serFldName = inputArg1.getByteArray();
                     int serFldNameOffset = inputArg1.getStartOffset();
                     fieldValueOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, serRecordOffset,
-                            serRecordLen, serFldName, serFldNameOffset);
+                            serRecordLen, serFldName, serFldNameOffset, fieldNameHashFunction, fieldNameComparator);
                     if (fieldValueOffset < 0) {
                         out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index 632a698..f885f5a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -23,6 +23,8 @@
 import java.util.List;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.ANull;
@@ -40,6 +42,8 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -70,6 +74,10 @@
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private final DataOutput out = resultStorage.getDataOutput();
             private final ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
@@ -204,7 +212,8 @@
                     for (; pathIndex < fieldPointables.length; pathIndex++) {
                         openField = true;
                         subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, start, len,
-                                fieldPointables[pathIndex].getByteArray(), fieldPointables[pathIndex].getStartOffset());
+                                fieldPointables[pathIndex].getByteArray(), fieldPointables[pathIndex].getStartOffset(),
+                                fieldNameHashFunction, fieldNameComparator);
                         if (subFieldOffset < 0) {
                             out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                             result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 53dbaad..7cc3aa3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -22,6 +22,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -33,6 +35,8 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -61,6 +65,10 @@
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private final DataOutput out = resultStorage.getDataOutput();
 
@@ -120,7 +128,7 @@
 
                     // Look at open fields.
                     subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, serRecordOffset,
-                            serRecordLen, serFldName, serFldNameOffset);
+                            serRecordLen, serFldName, serFldNameOffset, fieldNameHashFunction, fieldNameComparator);
                     if (subFieldOffset < 0) {
                         out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                         result.set(resultStorage);

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4929/ (8/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2939/ (3/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


[ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Reuse UTF8 hash function and comparator from
  evaluators when extracting a field's offset.

Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3109
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
4 files changed, 33 insertions(+), 15 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Jenkins: Verified; No violations found; Verified
  Michael Blow: Looks good to me, approved



diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index 97c49e0..2685e48 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -305,8 +305,8 @@
         return offset + AInt32SerializerDeserializer.getInt(serRecord, pointer + nullBitmapSize + (4 * fieldId));
     }
 
-    public static int getFieldOffsetByName(byte[] serRecord, int start, int len, byte[] fieldName, int nstart)
-            throws HyracksDataException {
+    public static int getFieldOffsetByName(byte[] serRecord, int start, int len, byte[] fieldName, int nstart,
+            IBinaryHashFunction nameHashFunction, IBinaryComparator nameComparator) throws HyracksDataException {
         // 5 is the index of the byte that determines whether the record is expanded or not, i.e. it has an open part.
         if (hasNoFields(serRecord, start, len) || serRecord[start + 5] != 1) {
             return -1;
@@ -316,14 +316,7 @@
         int numberOfOpenField = AInt32SerializerDeserializer.getInt(serRecord, openPartOffset);
         int fieldUtflength = UTF8StringUtil.getUTFLength(fieldName, nstart + 1);
         int fieldUtfMetaLen = UTF8StringUtil.getNumBytesToStoreLength(fieldUtflength);
-
-        IBinaryHashFunction utf8HashFunction =
-                BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
-
-        IBinaryComparator utf8BinaryComparator =
-                BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
-
-        int fieldNameHashCode = utf8HashFunction.hash(fieldName, nstart + 1, fieldUtflength + fieldUtfMetaLen);
+        int fieldNameHashCode = nameHashFunction.hash(fieldName, nstart + 1, fieldUtflength + fieldUtfMetaLen);
 
         int offset = openPartOffset + 4;
         int fieldOffset = -1;
@@ -337,7 +330,7 @@
             if (h == fieldNameHashCode) {
                 fieldOffset = start + 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, len, fieldName, nstart + 1,
+                if (nameComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
                         fieldUtflength + fieldUtfMetaLen) == 0) {
                     // since they are equal, we can directly use the meta length and the utf length.
                     return fieldOffset + fieldUtfMetaLen + fieldUtflength;
@@ -346,7 +339,7 @@
                         h = AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j));
                         if (h == fieldNameHashCode) {
                             fieldOffset = start + AInt32SerializerDeserializer.getInt(serRecord, offset + (8 * j) + 4);
-                            if (utf8BinaryComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
+                            if (nameComparator.compare(serRecord, fieldOffset, len, fieldName, nstart + 1,
                                     fieldUtflength) == 0) {
                                 return fieldOffset + fieldUtfMetaLen + fieldUtflength;
                             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
index 3deef73..e66cc52 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByNameEvalFactory.java
@@ -22,6 +22,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
@@ -29,6 +31,8 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -55,6 +59,10 @@
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private DataOutput out = resultStorage.getDataOutput();
 
@@ -83,7 +91,7 @@
                     byte[] serFldName = inputArg1.getByteArray();
                     int serFldNameOffset = inputArg1.getStartOffset();
                     fieldValueOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, serRecordOffset,
-                            serRecordLen, serFldName, serFldNameOffset);
+                            serRecordLen, serFldName, serFldNameOffset, fieldNameHashFunction, fieldNameComparator);
                     if (fieldValueOffset < 0) {
                         out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                         result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
index 632a698..f885f5a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessNestedEvalFactory.java
@@ -23,6 +23,8 @@
 import java.util.List;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.ANull;
@@ -40,6 +42,8 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -70,6 +74,10 @@
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private final DataOutput out = resultStorage.getDataOutput();
             private final ByteArrayAccessibleOutputStream subRecordTmpStream = new ByteArrayAccessibleOutputStream();
@@ -204,7 +212,8 @@
                     for (; pathIndex < fieldPointables.length; pathIndex++) {
                         openField = true;
                         subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, start, len,
-                                fieldPointables[pathIndex].getByteArray(), fieldPointables[pathIndex].getStartOffset());
+                                fieldPointables[pathIndex].getByteArray(), fieldPointables[pathIndex].getStartOffset(),
+                                fieldNameHashFunction, fieldNameComparator);
                         if (subFieldOffset < 0) {
                             out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                             result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 53dbaad..7cc3aa3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -22,6 +22,8 @@
 import java.io.IOException;
 
 import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
@@ -33,6 +35,8 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -61,6 +65,10 @@
     public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
 
+            private final IBinaryHashFunction fieldNameHashFunction =
+                    BinaryHashFunctionFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryHashFunction();
+            private final IBinaryComparator fieldNameComparator =
+                    BinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE.createBinaryComparator();
             private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
             private final DataOutput out = resultStorage.getDataOutput();
 
@@ -120,7 +128,7 @@
 
                     // Look at open fields.
                     subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetByName(serRecord, serRecordOffset,
-                            serRecordLen, serFldName, serFldNameOffset);
+                            serRecordLen, serFldName, serFldNameOffset, fieldNameHashFunction, fieldNameComparator);
                     if (subFieldOffset < 0) {
                         out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
                         result.set(resultStorage);

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

Gerrit-MessageType: merged
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/ZaCQkA : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5268/ (2/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/iL9bn1 : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7753/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5152/ (5/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5313/ (10/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4895/ (7/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8878/ (4/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/95/ (1/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7753/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4937/ (12/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10411/ (9/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4841/ (11/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

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

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5499/ (6/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDe...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2507][RT] Objects Creation in ARecordSerializerDeserializer
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/rZZxpy : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb47db9e909e49d65ebf1cd667e27e2467e4085
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: No