You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by wy...@apache.org on 2023/03/09 00:40:01 UTC

[asterixdb] branch master updated: [ASTERIXDB-3029][RT] Introduce LazyVisitablePointable

This is an automated email from the ASF dual-hosted git repository.

wyk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 4b5a6a7f07 [ASTERIXDB-3029][RT] Introduce LazyVisitablePointable
4b5a6a7f07 is described below

commit 4b5a6a7f071d1540d913d40ff1c24d9d72cabfd8
Author: Wail Alkowaileet <wa...@gmail.com>
AuthorDate: Tue Mar 7 16:33:41 2023 -0800

    [ASTERIXDB-3029][RT] Introduce LazyVisitablePointable
    
    - user mode changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    - The lazy visitable pointable allows for traversing a record and
      processing it in a single pass compared to the IVisitablePointable,
      which requires two passes - one for calling set() and another when
      calling accept().
    - Does not require writing the nested values into temporary buffers.
    - However, it is suitable when traversing the records in DFS.
    - Traversing and processing records efficiently is crucial for
      ASTERIXDB-3002 to infer the schema and covert rows into columns.
    
    Change-Id: Iaf2c827d80a2a7e77f5f6eb5438cea417387e132
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/17412
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Wail Alkowaileet <wa...@gmail.com>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
---
 .../test/om/lazy/LazyVisitablePointableTest.java   | 122 +++++++++++
 .../asterix/test/om/lazy/RecordTypeInference.java  |  96 +++++++++
 .../lazy/AbstractLazyNestedVisitablePointable.java | 139 +++++++++++++
 .../om/lazy/AbstractLazyVisitablePointable.java    | 101 +++++++++
 .../lazy/AbstractListLazyVisitablePointable.java   |  73 +++++++
 .../om/lazy/FixedListLazyVisitablePointable.java   |  63 ++++++
 .../om/lazy/FlatLazyVisitablePointable.java        |  60 ++++++
 .../om/lazy/GenericLazyVisitablePointable.java     |  91 +++++++++
 .../om/lazy/ILazyVisitablePointableVisitor.java    |  62 ++++++
 .../om/lazy/RecordLazyVisitablePointable.java      | 125 ++++++++++++
 .../om/lazy/TypedRecordLazyVisitablePointable.java | 226 +++++++++++++++++++++
 .../lazy/VariableListLazyVisitablePointable.java   |  73 +++++++
 12 files changed, 1231 insertions(+)

diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/LazyVisitablePointableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/LazyVisitablePointableTest.java
new file mode 100644
index 0000000000..d981be1ed6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/LazyVisitablePointableTest.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.om.lazy;
+
+import static org.apache.hyracks.util.file.FileUtil.joinPath;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.apache.asterix.external.parser.JSONDataParser;
+import org.apache.asterix.om.lazy.AbstractLazyVisitablePointable;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.lazy.TypedRecordLazyVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.pointables.cast.ACastVisitor;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.comparisons.DeepEqualAssessor;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.fasterxml.jackson.core.JsonFactory;
+
+/**
+ * Test the operations of {@link AbstractLazyVisitablePointable}
+ */
+public class LazyVisitablePointableTest {
+    private static final String BASE_DIR;
+    private static final String[] FILE_PATHS;
+    private final JSONDataParser parser;
+    private final ACastVisitor castVisitor;
+    private final RecordTypeInference schemaInference;
+    private final DeepEqualAssessor deepEqualAssessor;
+    private final RecordLazyVisitablePointable openLazyPointable;
+    private final ARecordVisitablePointable openPointable;
+    private final ArrayBackedValueStorage recordStorage;
+    private final Triple<IVisitablePointable, IAType, Boolean> arg;
+
+    static {
+        BASE_DIR = "data";
+        FILE_PATHS = new String[] { joinPath(BASE_DIR, "hdfs", "parquet", "dummy_tweet.json"),
+                joinPath(BASE_DIR, "nested01", "person2.adm"), joinPath(BASE_DIR, "yelp-checkin", "use-case-1.json"),
+                joinPath(BASE_DIR, "yelp-checkin", "use-case-2.json"),
+                joinPath(BASE_DIR, "yelp-checkin", "use-case-3.json"),
+                joinPath(BASE_DIR, "yelp-checkin", "use-case-4.json") };
+    }
+
+    public LazyVisitablePointableTest() {
+        parser = new JSONDataParser(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, new JsonFactory());
+        castVisitor = new ACastVisitor();
+        schemaInference = new RecordTypeInference();
+        deepEqualAssessor = new DeepEqualAssessor();
+        openLazyPointable = new RecordLazyVisitablePointable(true);
+        openPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        recordStorage = new ArrayBackedValueStorage();
+        arg = new Triple<>(null, null, null);
+        arg.third = Boolean.FALSE;
+    }
+
+    private void prepareParser(String path) throws IOException {
+        FileInputStream inputStream = new FileInputStream(path);
+        parser.setInputStream(inputStream);
+    }
+
+    private void inferCastAndCompare() throws HyracksDataException {
+        recordStorage.reset();
+        while (parser.parse(recordStorage.getDataOutput())) {
+            openLazyPointable.set(recordStorage);
+
+            //Infer the schema
+            ARecordType inferredFromOpen = (ARecordType) openLazyPointable.accept(schemaInference, "fromOpen");
+            ARecordVisitablePointable closedPointable = new ARecordVisitablePointable(inferredFromOpen);
+            arg.first = closedPointable;
+            arg.second = inferredFromOpen;
+
+            //Cast to closed using the inferred type
+            openPointable.set(recordStorage);
+            openPointable.accept(castVisitor, arg);
+            //Ensure both closed and open records are the same
+            Assert.assertTrue(deepEqualAssessor.isEqual(openPointable, closedPointable));
+
+            //Ensure lazy pointable can handle closed types
+            TypedRecordLazyVisitablePointable closedLazyPointable =
+                    new TypedRecordLazyVisitablePointable(inferredFromOpen);
+            closedLazyPointable.set(closedPointable);
+            //Infer the type (again) but from a closed type
+            ARecordType inferredFromClosed = (ARecordType) closedLazyPointable.accept(schemaInference, "fromClosed");
+            //Ensure both inferred types are the same
+            Assert.assertTrue(inferredFromOpen.deepEqual(inferredFromClosed));
+            recordStorage.reset();
+        }
+    }
+
+    @Test
+    public void runTest() throws IOException {
+        for (String path : FILE_PATHS) {
+            prepareParser(path);
+            inferCastAndCompare();
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/RecordTypeInference.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/RecordTypeInference.java
new file mode 100644
index 0000000000..4a1a4a3589
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/om/lazy/RecordTypeInference.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.om.lazy;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.asterix.om.lazy.AbstractListLazyVisitablePointable;
+import org.apache.asterix.om.lazy.FlatLazyVisitablePointable;
+import org.apache.asterix.om.lazy.ILazyVisitablePointableVisitor;
+import org.apache.asterix.om.lazy.RecordLazyVisitablePointable;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+/**
+ * Infers the type of single record using lazy visitable pointable
+ */
+public class RecordTypeInference implements ILazyVisitablePointableVisitor<IAType, String> {
+    private final ByteArrayAccessibleInputStream in;
+    private final DataInputStream dataIn;
+    private final UTF8StringReader utf8Reader;
+
+    public RecordTypeInference() {
+        in = new ByteArrayAccessibleInputStream(new byte[] {}, 0, 0);
+        dataIn = new DataInputStream(in);
+        utf8Reader = new UTF8StringReader();
+    }
+
+    @Override
+    public IAType visit(RecordLazyVisitablePointable pointable, String arg) throws HyracksDataException {
+        String[] fieldNames = new String[pointable.getNumberOfChildren()];
+        IAType[] fieldTypes = new IAType[pointable.getNumberOfChildren()];
+        for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+            pointable.nextChild();
+            fieldNames[i] = deserializeString(pointable.getFieldName());
+            fieldTypes[i] = pointable.getChildVisitablePointable().accept(this, fieldNames[i]);
+        }
+        // isOpen has to be false here to ensure that every field go to the closed part
+        return new ARecordType(arg, fieldNames, fieldTypes, false);
+    }
+
+    @Override
+    public IAType visit(AbstractListLazyVisitablePointable pointable, String arg) throws HyracksDataException {
+        IAType itemType = BuiltinType.ANY;
+        String itemTypeName = arg + "Item";
+        for (int i = 0; i < pointable.getNumberOfChildren(); i++) {
+            pointable.nextChild();
+            IAType ithItemType = pointable.getChildVisitablePointable().accept(this, itemTypeName);
+            if (itemType.getTypeTag() != ATypeTag.ANY && itemType.getTypeTag() != ithItemType.getTypeTag()) {
+                throw new UnsupportedOperationException("Union types are not supported");
+            }
+            itemType = ithItemType;
+        }
+        return pointable.getTypeTag() == ATypeTag.ARRAY ? new AOrderedListType(itemType, arg)
+                : new AUnorderedListType(itemType, arg);
+    }
+
+    @Override
+    public IAType visit(FlatLazyVisitablePointable pointable, String arg) throws HyracksDataException {
+        return BuiltinType.getBuiltinType(pointable.getTypeTag());
+    }
+
+    private String deserializeString(IValueReference stringValue) throws HyracksDataException {
+        in.setContent(stringValue.getByteArray(), stringValue.getStartOffset(), stringValue.getLength());
+        try {
+            return UTF8StringUtil.readUTF8(dataIn, utf8Reader);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyNestedVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyNestedVisitablePointable.java
new file mode 100644
index 0000000000..e9f8e8a8fc
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyNestedVisitablePointable.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+/**
+ * A common implementation for nested values (i.e., {@link ATypeTag#OBJECT}, {@link ATypeTag#ARRAY}, and
+ * {@link ATypeTag#MULTISET}).
+ * <p>
+ * Contract:
+ * <p>
+ * 1- A child's value may or may not contain a type tag. Thus, it is the responsibility of the caller to check if the
+ * child's value contains a type tag by calling {@link #isTaggedChild()}.
+ * 2- The returned objects from {@link #getChildVisitablePointable()} and {@link #getChildValue()}, are reused
+ * when possible. Thus, when the caller does the following for example:
+ * <p>
+ * AbstractLazyVisitablePointable child1 = visitablePointable.getChildVisitablePointable();
+ * visitablePointable.nextChild();
+ * AbstractLazyVisitablePointable child2 = visitablePointable.getChildVisitablePointable();
+ * <p>
+ * both child1 and child2 may have the same value, which is the value of the second child.
+ */
+public abstract class AbstractLazyNestedVisitablePointable extends AbstractLazyVisitablePointable {
+    private final ATypeTag typeTag;
+    protected final VoidPointable currentValue;
+    protected byte currentChildTypeTag;
+
+    AbstractLazyNestedVisitablePointable(boolean tagged, ATypeTag typeTag) {
+        super(tagged);
+        this.typeTag = typeTag;
+        currentValue = new VoidPointable();
+    }
+
+    /**
+     * Prepare the value and the tag of the next child
+     */
+    public abstract void nextChild() throws HyracksDataException;
+
+    /**
+     * If the child contains a tag
+     *
+     * @return true if the child is tagged (open value), false otherwise
+     */
+    public abstract boolean isTaggedChild();
+
+    /**
+     * @return number of children
+     */
+    public abstract int getNumberOfChildren();
+
+    /**
+     * Gets a child visitable-pointable.
+     */
+    public abstract AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException;
+
+    /**
+     * Returns a value reference of the child. Note that this is not a visitable-pointable reference.
+     */
+    public final IValueReference getChildValue() {
+        return currentValue;
+    }
+
+    /**
+     * The serialized type tag of a child
+     */
+    public final byte getChildSerializedTypeTag() {
+        return currentChildTypeTag;
+    }
+
+    /**
+     * The type tag of a child
+     */
+    public final ATypeTag getChildTypeTag() {
+        return ATypeTag.VALUE_TYPE_MAPPING[currentChildTypeTag];
+    }
+
+    /**
+     * @return The type tag that corresponds to {@code this} visitable-pointable
+     */
+    @Override
+    public final ATypeTag getTypeTag() {
+        return ATypeTag.VALUE_TYPE_MAPPING[getSerializedTypeTag()];
+    }
+
+    /**
+     * @return The serialized type tag that corresponds to {@code this} visitable-pointable
+     */
+    @Override
+    public final byte getSerializedTypeTag() {
+        return typeTag.serialize();
+    }
+
+    /**
+     * Helper method to create a typed (i.e., non-tagged) visitable-pointable
+     *
+     * @param type the required type
+     * @return a visitable pointable that corresponds to {@code type}
+     */
+    static AbstractLazyVisitablePointable createVisitable(IAType type) {
+        ATypeTag typeTag = type.getTypeTag();
+        switch (typeTag) {
+            case OBJECT:
+                return new TypedRecordLazyVisitablePointable(false, (ARecordType) type);
+            case ARRAY:
+            case MULTISET:
+                AbstractCollectionType listType = (AbstractCollectionType) type;
+                return NonTaggedFormatUtil.isFixedSizedCollection(listType.getItemType())
+                        ? new FixedListLazyVisitablePointable(false, listType)
+                        : new VariableListLazyVisitablePointable(false, listType);
+            default:
+                return new FlatLazyVisitablePointable(false, typeTag);
+
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyVisitablePointable.java
new file mode 100644
index 0000000000..1d6425c337
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractLazyVisitablePointable.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import org.apache.asterix.om.pointables.base.IVisitablePointable;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+/**
+ * Similar to {@link IVisitablePointable}. The difference is, upon calling {@link #set(byte[], int, int)}, the nested
+ * values (children) will not call {@link #set(byte[], int, int)} recursively. Instead, it will wait until the
+ * child is accessed. Thus, when a processor (a class that implements {@link ILazyVisitablePointableVisitor}) wants
+ * to traverse the object in a <b>DFS mode</b>, the traversal will be done in a single pass - compared to the two passes
+ * when using the {@link IVisitablePointable}, where one pass is done when calling
+ * {@link IVisitablePointable#set(byte[], int, int)} and another pass is done by the processor (e.g., the result
+ * printer). Also, the lazy visitable-pointable requires less memory as we do not allocate any temporary buffers.
+ */
+public abstract class AbstractLazyVisitablePointable implements IPointable {
+    private final boolean tagged;
+    private byte[] data;
+    private int offset;
+    private int length;
+
+    AbstractLazyVisitablePointable(boolean tagged) {
+        this.tagged = tagged;
+    }
+
+    @Override
+    public final void set(byte[] data, int offset, int length) {
+        this.data = data;
+        this.offset = offset;
+        this.length = length;
+        init(data, offset, length);
+    }
+
+    @Override
+    public final void set(IValueReference pointer) {
+        set(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public final byte[] getByteArray() {
+        return data;
+    }
+
+    @Override
+    public final int getStartOffset() {
+        return offset;
+    }
+
+    @Override
+    public final int getLength() {
+        return length;
+    }
+
+    /**
+     * @return The serialized type tag
+     */
+    public abstract byte getSerializedTypeTag();
+
+    /**
+     * @return The type tag
+     */
+    public abstract ATypeTag getTypeTag();
+
+    public abstract <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException;
+
+    /**
+     * @return true if the value contains tag, false otherwise
+     */
+    public final boolean isTagged() {
+        return tagged;
+    }
+
+    /**
+     * Called by {@link #set(byte[], int, int)} to initialize the visitable-pointable
+     *
+     * @param data   value's data
+     * @param offset value's start offset
+     * @param length value's length
+     */
+    abstract void init(byte[] data, int offset, int length);
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractListLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractListLazyVisitablePointable.java
new file mode 100644
index 0000000000..68d558cb01
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/AbstractListLazyVisitablePointable.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import java.util.Objects;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Common implementation for both {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET}
+ */
+public abstract class AbstractListLazyVisitablePointable extends AbstractLazyNestedVisitablePointable {
+    private final int headerSize;
+    private final AbstractLazyVisitablePointable itemVisitablePointable;
+    private int numberOfItems;
+    protected int currentIndex;
+    protected int itemsOffset;
+
+    AbstractListLazyVisitablePointable(boolean tagged, AbstractCollectionType listType) {
+        super(tagged, listType.getTypeTag());
+        Objects.requireNonNull(listType);
+        Objects.requireNonNull(listType.getItemType());
+        //1 for typeTag if tagged, 1 for itemTypeTag, 4 for length
+        headerSize = (isTagged() ? 1 : 0) + 1 + 4;
+        itemVisitablePointable = createVisitablePointable(listType.getItemType());
+    }
+
+    @Override
+    public final int getNumberOfChildren() {
+        return numberOfItems;
+    }
+
+    @Override
+    final void init(byte[] data, int offset, int length) {
+        int pointer = headerSize + offset;
+        numberOfItems = AInt32SerializerDeserializer.getInt(data, pointer);
+        itemsOffset = pointer + 4;
+        currentIndex = 0;
+    }
+
+    @Override
+    public AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException {
+        itemVisitablePointable.set(getChildValue());
+        return itemVisitablePointable;
+    }
+
+    @Override
+    public <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+        return visitor.visit(this, arg);
+    }
+
+    abstract AbstractLazyVisitablePointable createVisitablePointable(IAType itemType);
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FixedListLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FixedListLazyVisitablePointable.java
new file mode 100644
index 0000000000..f3153b206c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FixedListLazyVisitablePointable.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.lazy;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This implementation is only for closed {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET} with fixed-length
+ * items.
+ */
+public class FixedListLazyVisitablePointable extends AbstractListLazyVisitablePointable {
+    private final int itemSize;
+
+    public FixedListLazyVisitablePointable(boolean tagged, AbstractCollectionType listType) {
+        super(tagged, listType);
+        ATypeTag itemTag = listType.getItemType().getTypeTag();
+        currentChildTypeTag = itemTag.serialize();
+        try {
+            itemSize = NonTaggedFormatUtil.getFieldValueLength(null, -1, itemTag, false);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+
+    }
+
+    @Override
+    public void nextChild() {
+        byte[] data = getByteArray();
+        int itemOffset = getStartOffset() + itemsOffset + currentIndex * itemSize;
+        currentValue.set(data, itemOffset, itemSize);
+        currentIndex++;
+    }
+
+    @Override
+    public boolean isTaggedChild() {
+        return false;
+    }
+
+    @Override
+    AbstractLazyVisitablePointable createVisitablePointable(IAType itemType) {
+        return createVisitable(itemType);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FlatLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FlatLazyVisitablePointable.java
new file mode 100644
index 0000000000..f441c7b28c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/FlatLazyVisitablePointable.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import java.util.Objects;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class FlatLazyVisitablePointable extends AbstractLazyVisitablePointable {
+    private final ATypeTag typeTag;
+
+    public FlatLazyVisitablePointable(boolean tagged, ATypeTag typeTag) {
+        super(tagged);
+        Objects.requireNonNull(typeTag);
+        this.typeTag = typeTag;
+    }
+
+    @Override
+    public byte getSerializedTypeTag() {
+        if (isTagged()) {
+            return getByteArray()[getStartOffset()];
+        }
+        return typeTag.serialize();
+    }
+
+    @Override
+    public ATypeTag getTypeTag() {
+        if (isTagged()) {
+            return ATypeTag.VALUE_TYPE_MAPPING[getSerializedTypeTag()];
+        }
+        return typeTag;
+    }
+
+    @Override
+    public <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    void init(byte[] data, int offset, int length) {
+        //noOp
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/GenericLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/GenericLazyVisitablePointable.java
new file mode 100644
index 0000000000..f6d06eef22
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/GenericLazyVisitablePointable.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This is a generic lazy visitable-pointable for tagged values (a.k.a. open values). Each nested visitable-pointable
+ * should only allocate a single instance of this class and reuse it for every open value.
+ */
+public class GenericLazyVisitablePointable extends AbstractLazyVisitablePointable {
+    private RecordLazyVisitablePointable object;
+    private VariableListLazyVisitablePointable array;
+    private VariableListLazyVisitablePointable multiset;
+    private FlatLazyVisitablePointable flat;
+
+    private AbstractLazyVisitablePointable current;
+
+    public GenericLazyVisitablePointable() {
+        super(true);
+    }
+
+    @Override
+    public final byte getSerializedTypeTag() {
+        return current.getSerializedTypeTag();
+    }
+
+    @Override
+    public final ATypeTag getTypeTag() {
+        return current.getTypeTag();
+    }
+
+    @Override
+    public <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+        switch (current.getTypeTag()) {
+            case OBJECT:
+                return visitor.visit(object, arg);
+            case ARRAY:
+                return visitor.visit(array, arg);
+            case MULTISET:
+                return visitor.visit(multiset, arg);
+            default:
+                return visitor.visit(flat, arg);
+        }
+    }
+
+    @Override
+    void init(byte[] data, int offset, int length) {
+        ATypeTag typeTag = ATypeTag.VALUE_TYPE_MAPPING[data[offset]];
+        AbstractLazyVisitablePointable visitable = getOrCreateVisitablePointable(typeTag);
+        visitable.set(data, offset, length);
+        current = visitable;
+    }
+
+    private AbstractLazyVisitablePointable getOrCreateVisitablePointable(ATypeTag typeTag) {
+        switch (typeTag) {
+            case OBJECT:
+                object = object == null ? new RecordLazyVisitablePointable(true) : object;
+                return object;
+            case ARRAY:
+                array = array == null ? new VariableListLazyVisitablePointable(true,
+                        DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE) : array;
+                return array;
+            case MULTISET:
+                multiset = multiset == null ? new VariableListLazyVisitablePointable(true,
+                        DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE) : multiset;
+                return multiset;
+            default:
+                flat = flat == null ? new FlatLazyVisitablePointable(true, ATypeTag.ANY) : flat;
+                return flat;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/ILazyVisitablePointableVisitor.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/ILazyVisitablePointableVisitor.java
new file mode 100644
index 0000000000..50116cd106
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/ILazyVisitablePointableVisitor.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * A visitor for ADM values which utilizes the lazy visitable:
+ *
+ * @param <R> return type
+ * @param <T> argument type
+ * @see AbstractLazyVisitablePointable
+ * @see AbstractLazyNestedVisitablePointable
+ */
+public interface ILazyVisitablePointableVisitor<R, T> {
+    /**
+     * Visit record value
+     *
+     * @param pointable either typed {@link TypedRecordLazyVisitablePointable} or
+     *                  untyped {@link RecordLazyVisitablePointable}
+     * @param arg       visitor argument
+     * @return return value
+     */
+    R visit(RecordLazyVisitablePointable pointable, T arg) throws HyracksDataException;
+
+    /**
+     * Visit list value
+     *
+     * @param pointable either a list with fixed-length items {@link FixedListLazyVisitablePointable} or
+     *                  a list with variable-length items {@link VariableListLazyVisitablePointable}
+     * @param arg       visitor argument
+     * @return return value
+     */
+
+    R visit(AbstractListLazyVisitablePointable pointable, T arg) throws HyracksDataException;
+
+    /**
+     * Atomic values
+     *
+     * @param pointable any flat item (e.g., {@link org.apache.asterix.om.types.ATypeTag#BIGINT}
+     * @param arg       visitor argument
+     * @return return value
+     * @throws HyracksDataException
+     */
+    R visit(FlatLazyVisitablePointable pointable, T arg) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/RecordLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/RecordLazyVisitablePointable.java
new file mode 100644
index 0000000000..154b9f2fe7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/RecordLazyVisitablePointable.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+/**
+ * This implementation is to handle {@link ATypeTag#OBJECT} with open fields only
+ */
+public class RecordLazyVisitablePointable extends AbstractLazyNestedVisitablePointable {
+    protected final VoidPointable currentFieldName;
+    protected final AbstractLazyVisitablePointable openVisitable;
+    private int openValuesOffset;
+    private int numberOfOpenChildren;
+
+    public RecordLazyVisitablePointable(boolean tagged) {
+        super(tagged, ATypeTag.OBJECT);
+        currentFieldName = new VoidPointable();
+        openVisitable = new GenericLazyVisitablePointable();
+    }
+
+    @Override
+    public void nextChild() throws HyracksDataException {
+        byte[] data = getByteArray();
+
+        //set field name
+        int fieldNameLength = NonTaggedFormatUtil.getFieldValueLength(data, openValuesOffset, ATypeTag.STRING, false);
+        currentFieldName.set(data, openValuesOffset, fieldNameLength);
+        openValuesOffset += fieldNameLength;
+
+        //set Type tag
+        currentChildTypeTag = data[openValuesOffset];
+
+        //set value
+        int valueLength = NonTaggedFormatUtil.getFieldValueLength(data, openValuesOffset, getChildTypeTag(), true) + 1;
+        currentValue.set(data, openValuesOffset, valueLength);
+        openValuesOffset += valueLength;
+    }
+
+    @Override
+    public boolean isTaggedChild() {
+        return true;
+    }
+
+    @Override
+    public int getNumberOfChildren() {
+        return numberOfOpenChildren;
+    }
+
+    public IValueReference getFieldName() {
+        return currentFieldName;
+    }
+
+    @Override
+    public AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException {
+        openVisitable.set(getChildValue());
+        return openVisitable;
+    }
+
+    @Override
+    public final <R, T> R accept(ILazyVisitablePointableVisitor<R, T> visitor, T arg) throws HyracksDataException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    void init(byte[] data, int offset, int length) {
+        initOpenPart(data, offset);
+    }
+
+    /* ******************************************************
+     * Init Open part
+     * ******************************************************
+     */
+    protected int initOpenPart(byte[] data, int pointer) {
+        //+1 for type tag and +4 for the length
+        int skipTypeTag = isTagged() ? 1 : 0;
+        int currentPointer = pointer + skipTypeTag + 4;
+
+        boolean isExpanded = data[currentPointer] == 1;
+        //Advance to numberOfClosedChildren or
+        currentPointer++;
+
+        if (isExpanded) {
+            /*
+             * -(isTagged() ? 0 : 1) because the open part is computed - in the record builder - with the
+             * assumption that the record type tag is always there. Hence, if the record contains a type tag at the
+             * beginning of the record, we subtract 0 (noop). However, if the record doesn't contain a tag, then we
+             * need to subtract by 1 to get the correct offset of the open part (i.e., as if the tag existed).
+             */
+            int openPartStart =
+                    pointer + AInt32SerializerDeserializer.getInt(data, currentPointer) - (isTagged() ? 0 : 1);
+            //Skip open part offset to the beginning of closed part
+            currentPointer += 4;
+            //Number of children in the open part
+            numberOfOpenChildren = AInt32SerializerDeserializer.getInt(data, openPartStart);
+            //Skip the numberOfOpenChildren and the hashOffsetPair to the first open value
+            openValuesOffset = openPartStart + 4 + 8 * numberOfOpenChildren;
+        } else {
+            numberOfOpenChildren = 0;
+        }
+
+        return currentPointer;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/TypedRecordLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/TypedRecordLazyVisitablePointable.java
new file mode 100644
index 0000000000..19eb076d2c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/TypedRecordLazyVisitablePointable.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import java.util.Objects;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.util.string.UTF8StringWriter;
+
+/**
+ * This implementation extends {@link RecordLazyVisitablePointable} to handle {@link ATypeTag#OBJECT} with open and
+ * closed fields
+ */
+public class TypedRecordLazyVisitablePointable extends RecordLazyVisitablePointable {
+    private static final IPointable MISSING_POINTABLE = createConstantPointable(ATypeTag.MISSING);
+    private static final IPointable NULL_POINTABLE = createConstantPointable(ATypeTag.NULL);
+    private final ARecordType recordType;
+
+    //Closed values
+    private final IValueReference[] closedFieldNames;
+    private final AbstractLazyVisitablePointable[] closedVisitables;
+    private final int numberOfClosedChildren;
+    private final ATypeTag[] closedChildTags;
+    //Record builder computes the fields' offset as if the type tag exists
+    private final int actualChildOffset;
+    private int currentIndex;
+    private int closedValuesOffset;
+
+    /**
+     * A constructor for the root record
+     *
+     * @param rootRecordType root record type
+     */
+    public TypedRecordLazyVisitablePointable(ARecordType rootRecordType) {
+        this(true, rootRecordType);
+    }
+
+    public TypedRecordLazyVisitablePointable(boolean tagged, ARecordType recordType) {
+        super(tagged);
+        Objects.requireNonNull(recordType);
+        this.recordType = recordType;
+        numberOfClosedChildren = this.recordType.getFieldTypes().length;
+        closedFieldNames = createSerializedClosedFieldNames(this.recordType);
+        closedVisitables = createClosedVisitables(this.recordType);
+        closedChildTags = createInitialClosedTypeTags(this.recordType);
+        //-1 if not tagged. The offsets were calculated as if the tag exists.
+        actualChildOffset = isTagged() ? 0 : -1;
+    }
+
+    @Override
+    public void nextChild() throws HyracksDataException {
+        currentIndex++;
+        if (isTaggedChild()) {
+            super.nextChild();
+        } else {
+            setClosedValueInfo();
+        }
+    }
+
+    @Override
+    public boolean isTaggedChild() {
+        return currentIndex >= numberOfClosedChildren;
+    }
+
+    @Override
+    public int getNumberOfChildren() {
+        return numberOfClosedChildren + super.getNumberOfChildren();
+    }
+
+    @Override
+    public AbstractLazyVisitablePointable getChildVisitablePointable() throws HyracksDataException {
+        AbstractLazyVisitablePointable visitablePointable;
+        if (isTaggedChild()) {
+            visitablePointable = openVisitable;
+        } else {
+            visitablePointable = closedVisitables[currentIndex];
+        }
+        visitablePointable.set(getChildValue());
+        return visitablePointable;
+    }
+
+    private void setClosedValueInfo() throws HyracksDataException {
+        ATypeTag typeTag = closedChildTags[currentIndex];
+        if (typeTag == ATypeTag.NULL) {
+            currentValue.set(NULL_POINTABLE);
+        } else if (typeTag == ATypeTag.MISSING) {
+            currentValue.set(MISSING_POINTABLE);
+        } else {
+            byte[] data = getByteArray();
+            int offset =
+                    getStartOffset() + AInt32SerializerDeserializer.getInt(data, closedValuesOffset + 4 * currentIndex)
+                            + actualChildOffset;
+            int length = NonTaggedFormatUtil.getFieldValueLength(data, offset, typeTag, false);
+            currentValue.set(data, offset, length);
+        }
+        currentFieldName.set(closedFieldNames[currentIndex]);
+        currentChildTypeTag = typeTag.serialize();
+    }
+
+    /* ******************************************************
+     * Init Open part
+     * ******************************************************
+     */
+    @Override
+    void init(byte[] data, int offset, int length) {
+        /*
+         * Skip length and the type tag if the current record contains a tag. Only the root can be tagged and typed
+         * at the same time. Nested typed records will not be tagged.
+         */
+        int skipTag = isTagged() ? 1 : 0;
+        currentIndex = -1;
+        //initOpenPart first. It will skip type tag and length.
+        int pointer = recordType.isOpen() ? initOpenPart(data, offset) : offset + skipTag + 4;
+        initClosedPart(pointer, data);
+    }
+
+    private void initClosedPart(int pointer, byte[] data) {
+        //+4 to skip the number of closed children
+        int currentPointer = pointer + 4;
+        if (NonTaggedFormatUtil.hasOptionalField(recordType)) {
+            initClosedChildrenTags(data, currentPointer);
+            currentPointer =
+                    (numberOfClosedChildren % 4 == 0 ? numberOfClosedChildren / 4 : numberOfClosedChildren / 4 + 1);
+        }
+        closedValuesOffset = currentPointer;
+    }
+
+    private static IPointable createConstantPointable(ATypeTag tag) {
+        byte[] data = { tag.serialize() };
+        IPointable value = new VoidPointable();
+        value.set(data, 0, 1);
+        return value;
+    }
+
+    private void initClosedChildrenTags(byte[] data, int nullBitMapOffset) {
+        IAType[] types = recordType.getFieldTypes();
+        for (int i = 0; i < numberOfClosedChildren; i++) {
+            byte nullMissingOrValue = data[nullBitMapOffset + i / 4];
+            if (RecordUtil.isNull(nullMissingOrValue, i)) {
+                closedChildTags[i] = ATypeTag.NULL;
+            } else if (RecordUtil.isMissing(nullMissingOrValue, i)) {
+                closedChildTags[i] = ATypeTag.MISSING;
+            } else {
+                IAType type = types[i];
+                type = type.getTypeTag() == ATypeTag.UNION ? ((AUnionType) type).getActualType() : type;
+                closedChildTags[i] = type.getTypeTag();
+            }
+        }
+    }
+
+    private static ATypeTag[] createInitialClosedTypeTags(ARecordType recordType) {
+        IAType[] types = recordType.getFieldTypes();
+        ATypeTag[] typeTags = new ATypeTag[types.length];
+        for (int i = 0; i < types.length; i++) {
+            IAType type = types[i];
+            if (type.getTypeTag() == ATypeTag.UNION) {
+                type = ((AUnionType) type).getActualType();
+            }
+            typeTags[i] = type.getTypeTag();
+        }
+        return typeTags;
+    }
+
+    private static IValueReference[] createSerializedClosedFieldNames(ARecordType recordType) {
+        UTF8StringWriter writer = new UTF8StringWriter();
+        AMutableString mutableString = new AMutableString("");
+        AStringSerializerDeserializer serDer = new AStringSerializerDeserializer(writer, null);
+
+        String[] fieldNames = recordType.getFieldNames();
+        IValueReference[] fieldNameReferences = new IValueReference[fieldNames.length];
+        for (int i = 0; i < fieldNameReferences.length; i++) {
+            mutableString.setValue(fieldNames[i]);
+            fieldNameReferences[i] = createFieldName(mutableString, serDer);
+        }
+        return fieldNameReferences;
+    }
+
+    private static IValueReference createFieldName(AMutableString mutableString, AStringSerializerDeserializer serDer) {
+        ArrayBackedValueStorage storage = new ArrayBackedValueStorage();
+        try {
+            serDer.serialize(mutableString, storage.getDataOutput());
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+        return storage;
+    }
+
+    private static AbstractLazyVisitablePointable[] createClosedVisitables(ARecordType recordType) {
+        IAType[] types = recordType.getFieldTypes();
+        AbstractLazyVisitablePointable[] visitables = new AbstractLazyVisitablePointable[types.length];
+        for (int i = 0; i < types.length; i++) {
+            visitables[i] = createVisitable(types[i]);
+        }
+        return visitables;
+    }
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/VariableListLazyVisitablePointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/VariableListLazyVisitablePointable.java
new file mode 100644
index 0000000000..9e4ab9fc56
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/lazy/VariableListLazyVisitablePointable.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.lazy;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This implementation is for {@link ATypeTag#ARRAY} and {@link ATypeTag#MULTISET} with variable-length items.
+ */
+public class VariableListLazyVisitablePointable extends AbstractListLazyVisitablePointable {
+    private final ATypeTag itemTag;
+    //List builder computes the items' offset as if the type tag exists
+    private final int actualChildOffset;
+
+    public VariableListLazyVisitablePointable(boolean tagged, AbstractCollectionType listType) {
+        super(tagged, listType);
+        itemTag = listType.getItemType().getTypeTag();
+        //-1 if not tagged. The offsets were calculated as if the tag exists.
+        actualChildOffset = isTagged() ? 0 : -1;
+    }
+
+    @Override
+    public void nextChild() throws HyracksDataException {
+        byte[] data = getByteArray();
+        int itemOffset = getStartOffset() + AInt32SerializerDeserializer.getInt(data, itemsOffset + currentIndex * 4)
+                + actualChildOffset;
+        ATypeTag itemTypeTag = processTypeTag(data, itemOffset);
+        int itemSize = NonTaggedFormatUtil.getFieldValueLength(data, itemOffset, itemTypeTag, isTaggedChild());
+        currentValue.set(data, itemOffset, itemSize);
+        currentIndex++;
+    }
+
+    private ATypeTag processTypeTag(byte[] data, int itemOffset) {
+        if (itemTag == ATypeTag.ANY) {
+            currentChildTypeTag = data[itemOffset];
+        }
+        return itemTag;
+    }
+
+    @Override
+    public boolean isTaggedChild() {
+        return itemTag == ATypeTag.ANY;
+    }
+
+    @Override
+    AbstractLazyVisitablePointable createVisitablePointable(IAType itemType) {
+        if (itemType.getTypeTag() != ATypeTag.ANY) {
+            return createVisitable(itemType);
+        }
+        return new GenericLazyVisitablePointable();
+    }
+}