You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/07/31 23:53:54 UTC

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )


Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................

[ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees

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

Details:
When writing an index entry into RTree, set the corresponding null flags
for keys that are considered null. When reading an index entry,
examine the null flags to read the index entry properly.
Also, pass the null type traits and null introspector to inverted indexes.

Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
64 files changed, 478 insertions(+), 241 deletions(-)



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

diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 86d8ed4..f4d3dbe 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -107,7 +107,7 @@
         TestUtils.redirectLoggingToConsole();
         AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
         try {
-            integrationUtil.run(Boolean.getBoolean("cleanup.start"), Boolean.getBoolean("cleanup.shutdown"),
+            integrationUtil.run(true, Boolean.getBoolean("cleanup.shutdown"),
                     System.getProperty("conf.path", DEFAULT_CONF_FILE));
         } catch (Exception e) {
             LOGGER.fatal("Unexpected exception", e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index 8b69652..970c5a8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -31,15 +31,18 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
 import org.apache.asterix.metadata.api.IResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.data.std.accessors.ShortBinaryComparatorFactory;
@@ -129,13 +132,15 @@
                 FullTextUtil.fetchFilterAndCreateConfigEvaluator(mdProvider, index.getDataverseName(),
                         indexDetails.getFullTextConfigName());
 
+        ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
         return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                 filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
                 secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, typeTraitProvider.getTypeTrait(BuiltinType.ANULL),
+                NullIntrospector.INSTANCE);
     }
 
     // Returns an array of the type traits of the inverted list elements
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index e00d7b6..22f968a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
 import org.apache.asterix.metadata.api.IResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -35,6 +36,7 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.StringUtils;
@@ -155,6 +157,7 @@
                 dataset.getDatasetType() == DatasetType.EXTERNAL ? IndexingConstants.getBuddyBtreeComparatorFactories()
                         : getComparatorFactoriesForDeletedKeyBTree(secondaryTypeTraits, primaryComparatorFactories,
                                 secondaryComparatorFactories);
+        ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             AsterixVirtualBufferCacheProvider vbcProvider =
                     new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
@@ -162,14 +165,16 @@
                     filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                     pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                     mergePolicyFactory, mergePolicyProperties, true, valueProviderFactories, rTreePolicyType,
-                    linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories);
+                    linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories,
+                    typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
         } else {
             return new ExternalRTreeLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
                     filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                     pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
                     mergePolicyProperties, true, btreeCompFactories, valueProviderFactories, rTreePolicyType,
                     linearizeCmpFactory, rtreeFields, new int[] { numNestedSecondaryKeyFields }, isPointMBR,
-                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate());
+                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
+                    typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index 3b3fc1e..1725ea3 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -54,6 +54,6 @@
                 NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory, getVirtualBufferCacheProvider(),
                 SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE,
                 btreeComparatorFactories, valueProviderFactories, rtreePolicyType, linearizerCmpFactory, null,
-                btreeFields, IS_POINT_MBR, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
+                btreeFields, IS_POINT_MBR, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, null, null);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
index 0bb483a..28b9e88 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -54,6 +54,6 @@
                 NoOpIOOperationCallbackFactory.INSTANCE, NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory,
                 getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
                 MERGE_POLICY_PROPERTIES, DURABLE, valueProviderFactories, rtreePolicyType, linearizerCmpFactory,
-                btreeFields, IS_POINT_MBR, btreeComparatorFactories);
+                btreeFields, IS_POINT_MBR, btreeComparatorFactories, null, null);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
index 43d685e..1981c1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
@@ -38,6 +38,6 @@
             int[] btreeFields, ITypeTraits[] secondaryTypeTraits,
             IBinaryComparatorFactory[] secondaryComparatorFactories, IMetadataPageManagerFactory pageManagerFactory) {
         return new RTreeResourceFactory(storageManager, secondaryTypeTraits, secondaryComparatorFactories,
-                pageManagerFactory, valueProviderFactories, rtreePolicyType);
+                pageManagerFactory, valueProviderFactories, rtreePolicyType, null, null);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
index 8eff435..084c89f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
@@ -131,7 +131,7 @@
         return dataStartOff - tupleStartOff + decodedFieldSlots[fieldCount - 1];
     }
 
-    private boolean isNull(byte[] flags, int flagsOffset, int fieldIdx) {
+    protected boolean isNull(byte[] flags, int flagsOffset, int fieldIdx) {
         int adjustedFieldIdx = getAdjustedFieldIdx(fieldIdx);
         int flagByteIdx = adjustedFieldIdx / 8;
         int flagBitIdx = 7 - (adjustedFieldIdx % 8);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
index 2528518..882be88 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
@@ -182,7 +182,7 @@
      * @param flagsOffset start of the null flags data
      * @param fieldIdx logical field index
      */
-    private void setNullFlag(byte[] flags, int flagsOffset, int fieldIdx) {
+    protected void setNullFlag(byte[] flags, int flagsOffset, int fieldIdx) {
         int adjustedFieldIdx = getAdjustedFieldIdx(fieldIdx);
         int flagByteIdx = adjustedFieldIdx / 8;
         int flagBitIdx = 7 - (adjustedFieldIdx % 8);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
index a693a99..ed44dfa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
@@ -21,6 +21,7 @@
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.common.IIndex;
 
 public interface IInvertedIndex extends IIndex {
@@ -32,4 +33,8 @@
     ITypeTraits[] getTokenTypeTraits();
 
     IBinaryComparatorFactory[] getTokenCmpFactories();
+
+    ITypeTraits getNullTypeTraits();
+
+    INullIntrospector getNullIntrospector();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 8550385..8a107e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -90,11 +91,13 @@
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
             boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
 
         super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
@@ -141,7 +144,7 @@
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory, serviceCtx.getTracer());
+                    metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
         } else {
             return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
                     tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, bufferCache,
@@ -149,7 +152,7 @@
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory, serviceCtx.getTracer());
+                    metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
index 79c6123..395331f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -62,10 +63,12 @@
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
             boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         // ToDo: totally replace tokenizerFactory with full-text config
@@ -85,7 +88,8 @@
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, tokenTypeTraits, tokenCmpFactories,
                 tokenizerFactory, fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate,
+                nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
index 7ec6ce2..9886da4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
 import org.apache.hyracks.dataflow.std.buffermanager.SingleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -68,6 +69,8 @@
     protected int bufferNumLoadedPages;
 
     protected final IInvertedListTupleReference tuple;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
     protected final ITypeTraits[] invListFields;
     protected ICachedPage page;
     // buffer manager to conform to the memory budget
@@ -79,19 +82,22 @@
     protected final IIndexCursorStats stats;
 
     protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
-        this(bufferCache, fileId, invListFields, ctx, stats, false);
+            IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        this(bufferCache, fileId, invListFields, ctx, stats, false, nullTypeTraits, nullIntrospector);
     }
 
     protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IIndexCursorStats stats) throws HyracksDataException {
-        this(bufferCache, fileId, invListFields, null, stats, true);
+            IIndexCursorStats stats, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
+        this(bufferCache, fileId, invListFields, null, stats, true, nullTypeTraits, nullIntrospector);
     }
 
     // If isScan, use the SingleFrameBufferManager to minimize memory cost by allocating only one memory frame,
     // elsewhere use a regular buffer manager
     protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IHyracksTaskContext ctx, IIndexCursorStats stats, boolean isScan) throws HyracksDataException {
+            IHyracksTaskContext ctx, IIndexCursorStats stats, boolean isScan, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         this.bufferCache = bufferCache;
         this.fileId = fileId;
 
@@ -104,7 +110,7 @@
         this.lastRandomSearchedElementIx = 0;
         this.moreBlocksToRead = true;
         this.invListFields = invListFields;
-        this.tuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields);
+        this.tuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields, nullTypeTraits);
         this.buffers = new ArrayList<ByteBuffer>();
         if (ctx == null && !isScan) {
             throw HyracksDataException.create(ErrorCode.CANNOT_CONTINUE_TEXT_SEARCH_HYRACKS_TASK_IS_NULL);
@@ -118,6 +124,8 @@
             this.bufferManagerForSearch = new SingleFrameBufferManager();
         }
         this.stats = stats;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index f2d5a37..a828d22 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
@@ -92,6 +93,8 @@
     protected final IBinaryComparatorFactory[] tokenCmpFactories;
     private final int[] filterFieldsForNonBulkLoadOps;
     private final int[] invertedIndexFieldsForNonBulkLoadOps;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
 
     public LSMInvertedIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
@@ -103,7 +106,8 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
                 opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
                 componentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
@@ -116,14 +120,17 @@
         this.tokenCmpFactories = tokenCmpFactories;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
         this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
+        this.nullIntrospector = nullIntrospector;
+        this.nullTypeTraits = nullTypeTraits;
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             InMemoryInvertedIndex memInvIndex =
                     createInMemoryInvertedIndex(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache), i);
-            BTree deleteKeysBTree = BTreeUtils.createBTree(virtualBufferCache,
-                    new VirtualFreePageManager(virtualBufferCache), invListTypeTraits, invListCmpFactories,
-                    BTreeLeafFrameType.REGULAR_NSM,
-                    ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false, null, null);
+            BTree deleteKeysBTree =
+                    BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
+                            invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM,
+                            ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false,
+                            nullTypeTraits, nullIntrospector);
             LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(this, memInvIndex,
                     deleteKeysBTree, virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
@@ -410,7 +417,8 @@
         return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory,
-                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id), nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
@@ -444,6 +452,16 @@
         return tokenCmpFactories;
     }
 
+    @Override
+    public ITypeTraits getNullTypeTraits() {
+        return nullTypeTraits;
+    }
+
+    @Override
+    public INullIntrospector getNullIntrospector() {
+        return nullIntrospector;
+    }
+
     public IBinaryTokenizerFactory getTokenizerFactory() {
         return tokenizerFactory;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index de1fc49..a801146 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
@@ -56,12 +57,14 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
                 bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
                 tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, mergePolicy,
                 opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
-                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer,
+                nullTypeTraits, nullIntrospector);
     }
 
     @Override
@@ -70,7 +73,8 @@
         return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(virtualBufferCache,
                 virtualFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
                 tokenizerFactory, fullTextConfigEvaluatorFactory,
-                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id), nullTypeTraits,
+                nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index c2e07b3..7ee6af8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
@@ -55,12 +56,15 @@
 
     protected final ITypeTraits[] btreeTypeTraits;
     protected final IBinaryComparatorFactory[] btreeCmpFactories;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
 
     public InMemoryInvertedIndex(IBufferCache virtualBufferCache, IPageManager virtualFreePageManager,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
-            FileReference btreeFileRef) throws HyracksDataException {
+            FileReference btreeFileRef, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.invListTypeTraits = invListTypeTraits;
@@ -79,8 +83,11 @@
             btreeTypeTraits[tokenTypeTraits.length + i] = invListTypeTraits[i];
             btreeCmpFactories[tokenTypeTraits.length + i] = invListCmpFactories[i];
         }
-        this.btree = BTreeUtils.createBTree(virtualBufferCache, virtualFreePageManager, btreeTypeTraits,
-                btreeCmpFactories, BTreeLeafFrameType.REGULAR_NSM, btreeFileRef, false, null, null);
+        this.btree =
+                BTreeUtils.createBTree(virtualBufferCache, virtualFreePageManager, btreeTypeTraits, btreeCmpFactories,
+                        BTreeLeafFrameType.REGULAR_NSM, btreeFileRef, false, nullTypeTraits, nullIntrospector);
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -219,6 +226,16 @@
     }
 
     @Override
+    public ITypeTraits getNullTypeTraits() {
+        return nullTypeTraits;
+    }
+
+    @Override
+    public INullIntrospector getNullIntrospector() {
+        return nullIntrospector;
+    }
+
+    @Override
     public int getNumOfFilterFields() {
         return 0;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index bd9ce60..e3bce41 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
@@ -49,9 +50,11 @@
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
-            FileReference btreeFileRef) throws HyracksDataException {
+            FileReference btreeFileRef, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         super(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
+                tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef, nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
index 0d00ed3..f4086cc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
@@ -20,6 +20,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.fixedsize.FixedSizeElementInvertedListBuilder;
@@ -31,10 +32,15 @@
     protected final ITypeTraits[] invListFields;
     protected final ITypeTraits[] tokenTypeTraits;
     private final boolean isFixedSize;
+    private final ITypeTraits nullTypeTraits;
+    private final INullIntrospector nullIntrospector;
 
-    public InvertedListBuilderFactory(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields) {
+    public InvertedListBuilderFactory(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         this.tokenTypeTraits = tokenTypeTraits;
         this.invListFields = invListFields;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
 
         isFixedSize = InvertedIndexUtils.checkTypeTraitsAllFixed(invListFields);
     }
@@ -44,7 +50,8 @@
         if (isFixedSize) {
             return new FixedSizeElementInvertedListBuilder(invListFields);
         } else {
-            return new VariableSizeElementInvertedListBuilder(tokenTypeTraits, invListFields);
+            return new VariableSizeElementInvertedListBuilder(tokenTypeTraits, invListFields, nullTypeTraits,
+                    nullIntrospector);
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 7a45c5d..48a173e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
@@ -103,6 +104,8 @@
     protected int rootPageId = 0;
     protected IBufferCache bufferCache;
     protected int fileId = -1;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
     protected final ITypeTraits[] invListTypeTraits;
     protected final IBinaryComparatorFactory[] invListCmpFactories;
     protected final ITypeTraits[] tokenTypeTraits;
@@ -119,7 +122,8 @@
     public OnDiskInvertedIndex(IBufferCache bufferCache, IInvertedListBuilder invListBuilder,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile,
-            FileReference invListsFile, IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            FileReference invListsFile, IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         this.bufferCache = bufferCache;
         this.invListBuilder = invListBuilder;
         this.invListTypeTraits = invListTypeTraits;
@@ -128,7 +132,7 @@
         this.tokenCmpFactories = tokenCmpFactories;
         this.btree = BTreeUtils.createDiskBTree(bufferCache, getBTreeTypeTraits(tokenTypeTraits), tokenCmpFactories,
                 BTreeLeafFrameType.REGULAR_NSM, btreeFile, pageManagerFactory.createPageManager(bufferCache), false,
-                null, null);
+                nullTypeTraits, nullIntrospector);
         this.numTokenFields = btree.getComparatorFactories().length;
         this.numInvListKeys = invListCmpFactories.length;
         this.invListsFile = invListsFile;
@@ -136,6 +140,8 @@
         this.invListEndPageIdField = numTokenFields + 1;
         this.invListStartOffField = numTokenFields + 2;
         this.invListNumElementsField = numTokenFields + 3;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -197,10 +203,10 @@
     public IInvertedListCursor createInvertedListCursor(IHyracksTaskContext ctx) throws HyracksDataException {
         if (InvertedIndexUtils.checkTypeTraitsAllFixed(invListTypeTraits)) {
             return new FixedSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, ctx,
-                    NoOpIndexCursorStats.INSTANCE);
+                    NoOpIndexCursorStats.INSTANCE, nullTypeTraits, nullIntrospector);
         } else {
             return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, ctx,
-                    NoOpIndexCursorStats.INSTANCE);
+                    NoOpIndexCursorStats.INSTANCE, nullTypeTraits, nullIntrospector);
         }
     }
 
@@ -210,7 +216,8 @@
         if (InvertedIndexUtils.checkTypeTraitsAllFixed(invListTypeTraits)) {
             return new FixedSizeElementInvertedListScanCursor(bufferCache, fileId, invListTypeTraits, stats);
         } else {
-            return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, stats);
+            return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, stats,
+                    nullTypeTraits, nullIntrospector);
         }
     }
 
@@ -700,6 +707,16 @@
     }
 
     @Override
+    public ITypeTraits getNullTypeTraits() {
+        return nullTypeTraits;
+    }
+
+    @Override
+    public INullIntrospector getNullIntrospector() {
+        return nullIntrospector;
+    }
+
+    @Override
     public int getNumOfFilterFields() {
         return 0;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
index f058e4a..780f36d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.IndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
@@ -39,12 +40,14 @@
     protected final ITypeTraits[] tokenTypeTraits;
     protected final IBinaryComparatorFactory[] tokenCmpFactories;
     protected final IInvertedIndexFileNameMapper fileNameMapper;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
 
     public OnDiskInvertedIndexFactory(IIOManager ioManager, IBufferCache bufferCache,
             IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper,
-            IPageManagerFactory pageManagerFactory) {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         super(ioManager, bufferCache, pageManagerFactory);
         this.invListBuilderFactory = invListBuilderFactory;
         this.invListTypeTraits = invListTypeTraits;
@@ -52,6 +55,8 @@
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.fileNameMapper = fileNameMapper;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -60,6 +65,7 @@
         FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
         IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
         return new OnDiskInvertedIndex(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory);
+                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index fa5c365..ca4d94c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.data.std.primitive.ShortPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
@@ -46,9 +47,10 @@
     public PartitionedOnDiskInvertedIndex(IBufferCache bufferCache, IInvertedListBuilder invListBuilder,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile,
-            FileReference invListsFile, IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            FileReference invListsFile, IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         super(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                btreeFile, invListsFile, pageManagerFactory);
+                btreeFile, invListsFile, pageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     public class PartitionedOnDiskInvertedIndexAccessor extends OnDiskInvertedIndexAccessor {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
index 7f10658..f9f4012 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
@@ -35,9 +36,9 @@
             IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper,
-            IPageManagerFactory pageManagerFactory) {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         super(ioManager, bufferCache, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, fileNameMapper, pageManagerFactory);
+                tokenCmpFactories, fileNameMapper, pageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     @Override
@@ -46,6 +47,7 @@
         FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
         IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
         return new PartitionedOnDiskInvertedIndex(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory);
+                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
index 11c0901..ce6daa8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.AbstractOnDiskInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -49,11 +50,12 @@
     private int[] elementIndexes = new int[10];
 
     public FixedSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
-        super(bufferCache, fileId, invListFields, ctx, stats);
+            IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        super(bufferCache, fileId, invListFields, ctx, stats, nullTypeTraits, nullIntrospector);
 
         this.bufferEndElementIx = 0;
-        this.bufferEndElementTuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields);
+        this.bufferEndElementTuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields, nullTypeTraits);
 
         int tmpSize = 0;
         for (int i = 0; i < invListFields.length; i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
index 3ce26c5..c667487 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
@@ -22,6 +22,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.AbstractInvertedListBuilder;
@@ -35,8 +36,8 @@
 
     // The tokenTypeTraits is necessary because the underlying TypeAwareTupleWriter requires all the type traits of the tuple
     // even if the first a few fields in the tuple are never accessed by the writer
-    public VariableSizeElementInvertedListBuilder(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields)
-            throws HyracksDataException {
+    public VariableSizeElementInvertedListBuilder(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         super(invListFields);
 
         this.allFields = new ITypeTraits[invListFields.length + tokenTypeTraits.length];
@@ -46,7 +47,7 @@
         for (int i = 0; i < invListFields.length; i++) {
             allFields[i + tokenTypeTraits.length] = invListFields[i];
         }
-        this.writer = new TypeAwareTupleWriter(allFields, null, null);
+        this.writer = new TypeAwareTupleWriter(allFields, nullTypeTraits, nullIntrospector);
 
         InvertedIndexUtils.verifyHasVarSizeTypeTrait(invListFields);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
index a5515b3..8d863b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
@@ -49,16 +50,18 @@
     private ITreeIndexTupleWriter tupleWriter;
 
     public VariableSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId,
-            ITypeTraits[] invListFields, IIndexCursorStats stats) throws HyracksDataException {
-        super(bufferCache, fileId, invListFields, stats);
+            ITypeTraits[] invListFields, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        super(bufferCache, fileId, invListFields, stats, nullTypeTraits, nullIntrospector);
         this.isInit = true;
-        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields);
-        this.tupleWriter = new TypeAwareTupleWriter(invListFields, null, null);
+        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields, nullTypeTraits);
+        this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
     }
 
     public VariableSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId,
-            ITypeTraits[] invListFields, IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
-        super(bufferCache, fileId, invListFields, ctx, stats);
+            ITypeTraits[] invListFields, IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        super(bufferCache, fileId, invListFields, ctx, stats, nullTypeTraits, nullIntrospector);
         isInit = true;
     }
 
@@ -69,8 +72,8 @@
         // Note that the cursors can be re-used in the upper-layer callers so we need to reset the state variables when open()
         currentElementIxForScan = 0;
         isInit = true;
-        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields);
-        this.tupleWriter = new TypeAwareTupleWriter(invListFields, null, null);
+        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields, nullTypeTraits);
+        this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
index 75d0cfe..f71f347 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
@@ -54,12 +55,12 @@
     private IInvertedListTupleReference tupleReference;
     private ITreeIndexTupleWriter tupleWriter;
 
-    public VariableSizeInvertedListSearchResultFrameTupleAccessor(int frameSize, ITypeTraits[] fields)
-            throws HyracksDataException {
+    public VariableSizeInvertedListSearchResultFrameTupleAccessor(int frameSize, ITypeTraits[] fields,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         super(frameSize, fields);
 
-        this.tupleWriter = new TypeAwareTupleWriter(fields, null, null);
-        this.tupleReference = new VariableSizeInvertedListTupleReference(fields);
+        this.tupleWriter = new TypeAwareTupleWriter(fields, nullTypeTraits, nullIntrospector);
+        this.tupleReference = new VariableSizeInvertedListTupleReference(fields, nullTypeTraits);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
index 1e69076..997c93a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
@@ -28,17 +28,18 @@
 
 public class VariableSizeInvertedListTupleReference extends AbstractInvertedListTupleReference {
 
-    private ITreeIndexTupleReference tupleReference;
+    private final ITreeIndexTupleReference tupleReference;
 
     @Override
     protected void verifyTypeTrait() throws HyracksDataException {
         InvertedIndexUtils.verifyHasVarSizeTypeTrait(typeTraits);
     }
 
-    public VariableSizeInvertedListTupleReference(ITypeTraits[] typeTraits) throws HyracksDataException {
+    public VariableSizeInvertedListTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits)
+            throws HyracksDataException {
         super(typeTraits);
 
-        this.tupleReference = new TypeAwareTupleReference(typeTraits, null);
+        this.tupleReference = new TypeAwareTupleReference(typeTraits, nullTypeTraits);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 2fb620c..aa6f7c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -103,8 +103,8 @@
         if (bufferManager == null) {
             throw HyracksDataException.create(ErrorCode.CANNOT_CONTINUE_TEXT_SEARCH_BUFFER_MANAGER_IS_NULL);
         }
-        this.finalSearchResult =
-                new InvertedIndexFinalSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
+        this.finalSearchResult = new InvertedIndexFinalSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+                invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
         this.invListMerger = new InvertedListMerger(ctx, invIndex, bufferManager);
         this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
         this.invListCursorFactory = new InvertedListCursorFactory(invIndex, ctx);
@@ -118,9 +118,10 @@
         this.queryTokenAppender = new FrameTupleAppenderAccessor(QUERY_TOKEN_REC_DESC);
         this.queryTokenAppender.reset(queryTokenFrame, true);
         this.isSingleInvertedList = false;
-        this.searchResultTuple = InvertedIndexUtils.createInvertedListTupleReference(invIndex.getInvListTypeTraits());
+        this.searchResultTuple = InvertedIndexUtils.createInvertedListTupleReference(invIndex.getInvListTypeTraits(),
+                invIndex.getNullTypeTraits());
         this.searchResultFta = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(),
-                invIndex.getInvListTypeTraits());
+                invIndex.getInvListTypeTraits(), invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
     }
 
     protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
index e5f4622..f07983f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 
 /**
  * This is an in-memory based storage for final results of inverted-index searches.
@@ -34,8 +35,9 @@
 public class InvertedIndexFinalSearchResult extends InvertedIndexSearchResult {
 
     public InvertedIndexFinalSearchResult(ITypeTraits[] invListFields, IHyracksTaskContext ctx,
-            ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
-        super(invListFields, ctx, bufferManager);
+            ISimpleFrameBufferManager bufferManager, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
+        super(invListFields, ctx, bufferManager, nullTypeTraits, nullIntrospector);
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
index 6958fd5..35d0dd8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
 import org.apache.hyracks.dataflow.std.buffermanager.BufferManagerBackedVSizeFrame;
 import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListSearchResultFrameTupleAppender;
@@ -84,14 +85,16 @@
     protected byte[] tempBytes;
 
     public InvertedIndexSearchResult(ITypeTraits[] invListFields, IHyracksTaskContext ctx,
-            ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
+            ISimpleFrameBufferManager bufferManager, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         this.invListFields = invListFields;
-        this.tupleWriter = new TypeAwareTupleWriter(invListFields, null, null);
+        this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
         initTypeTraits(invListFields);
         this.ctx = ctx;
         appender = new InvertedListSearchResultFrameTupleAppender(ctx.getInitialFrameSize());
-        accessor = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits);
-        tuple = InvertedIndexUtils.createInvertedListTupleReference(typeTraits);
+        accessor = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits,
+                nullTypeTraits, nullIntrospector);
+        tuple = InvertedIndexUtils.createInvertedListTupleReference(typeTraits, nullTypeTraits);
         this.bufferManager = bufferManager;
         this.isInReadMode = false;
         this.isWriteFinished = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
index deea12a..8bf16fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
@@ -92,8 +92,10 @@
     public InvertedListMerger(IHyracksTaskContext ctx, IInvertedIndex invIndex, ISimpleFrameBufferManager bufferManager)
             throws HyracksDataException {
         this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
-        this.prevSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
-        this.newSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
+        this.prevSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+                invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
+        this.newSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+                invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index bee6be6..aefb461 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -84,41 +85,45 @@
             IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
-            throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef,
+                nullTypeTraits, nullIntrospector);
     }
 
     public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
             IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
-            throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         return new PartitionedInMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits,
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                fullTextConfigEvaluatorFactory, btreeFileRef);
+                fullTextConfigEvaluatorFactory, btreeFileRef, nullTypeTraits, nullIntrospector);
     }
 
     public static OnDiskInvertedIndex createOnDiskInvertedIndex(IIOManager ioManager, IBufferCache bufferCache,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile,
-            IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
         FileReference btreeFile = getBTreeFile(ioManager, invListsFile);
         return new OnDiskInvertedIndex(bufferCache, builder, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory);
+                tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     public static PartitionedOnDiskInvertedIndex createPartitionedOnDiskInvertedIndex(IIOManager ioManager,
             IBufferCache bufferCache, ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile,
-            IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
         FileReference btreeFile = getBTreeFile(ioManager, invListsFile);
         return new PartitionedOnDiskInvertedIndex(bufferCache, builder, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory);
+                tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 
     public static FileReference getBTreeFile(IIOManager ioManager, FileReference invListsFile)
@@ -128,9 +133,10 @@
 
     public static BTreeFactory createDeletedKeysBTreeFactory(IIOManager ioManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, IBufferCache diskBufferCache,
-            IPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
-                new BTreeTypeAwareTupleWriterFactory(invListTypeTraits, false, null, null);
+                new BTreeTypeAwareTupleWriterFactory(invListTypeTraits, false, nullTypeTraits, nullIntrospector);
         ITreeIndexFrameFactory leafFrameFactory =
                 BTreeUtils.getLeafFrameFactory(tupleWriterFactory, BTreeLeafFrameType.REGULAR_NSM);
         ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
@@ -148,11 +154,11 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
-            boolean durable, IMetadataPageManagerFactory pageManagerFactory, ITracer tracer)
-            throws HyracksDataException {
+            boolean durable, IMetadataPageManagerFactory pageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
-                invListCmpFactories, diskBufferCache, pageManagerFactory);
+                invListCmpFactories, diskBufferCache, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
         for (int i = 0; i < invListCmpFactories.length; i++) {
@@ -165,17 +171,17 @@
                 new LSMInvertedIndexFileManager(ioManager, onDiskDirFileRef, deletedKeysBTreeFactory);
 
         IInvertedListBuilderFactory invListBuilderFactory =
-                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits);
-        OnDiskInvertedIndexFactory invIndexFactory =
-                new OnDiskInvertedIndexFactory(ioManager, diskBufferCache, invListBuilderFactory, invListTypeTraits,
-                        invListCmpFactories, tokenTypeTraits, tokenCmpFactories, fileManager, pageManagerFactory);
+                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits, nullTypeTraits, nullIntrospector);
+        OnDiskInvertedIndexFactory invIndexFactory = new OnDiskInvertedIndexFactory(ioManager, diskBufferCache,
+                invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                fileManager, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         ComponentFilterHelper filterHelper = null;
         LSMComponentFilterFrameFactory filterFrameFactory = null;
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -188,7 +194,7 @@
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
     }
 
     public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -201,10 +207,11 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
-            boolean durable, IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
+            boolean durable, IPageManagerFactory pageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
-                invListCmpFactories, diskBufferCache, pageManagerFactory);
+                invListCmpFactories, diskBufferCache, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
         for (int i = 0; i < invListCmpFactories.length; i++) {
@@ -217,17 +224,17 @@
                 new LSMInvertedIndexFileManager(ioManager, onDiskDirFileRef, deletedKeysBTreeFactory);
 
         IInvertedListBuilderFactory invListBuilderFactory =
-                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits);
+                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits, nullTypeTraits, nullIntrospector);
         PartitionedOnDiskInvertedIndexFactory invIndexFactory = new PartitionedOnDiskInvertedIndexFactory(ioManager,
                 diskBufferCache, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, fileManager, pageManagerFactory);
+                tokenCmpFactories, fileManager, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         ComponentFilterHelper filterHelper = null;
         LSMComponentFilterFrameFactory filterFrameFactory = null;
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -240,7 +247,7 @@
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
     }
 
     public static boolean checkTypeTraitsAllFixed(ITypeTraits[] typeTraits) {
@@ -266,21 +273,22 @@
         }
     }
 
-    public static IInvertedListTupleReference createInvertedListTupleReference(ITypeTraits[] typeTraits)
-            throws HyracksDataException {
+    public static IInvertedListTupleReference createInvertedListTupleReference(ITypeTraits[] typeTraits,
+            ITypeTraits nullTypeTraits) throws HyracksDataException {
         if (checkTypeTraitsAllFixed(typeTraits)) {
             return new FixedSizeInvertedListTupleReference(typeTraits);
         } else {
-            return new VariableSizeInvertedListTupleReference(typeTraits);
+            return new VariableSizeInvertedListTupleReference(typeTraits, nullTypeTraits);
         }
     }
 
-    public static IFrameTupleAccessor createInvertedListFrameTupleAccessor(int frameSize, ITypeTraits[] typeTraits)
-            throws HyracksDataException {
+    public static IFrameTupleAccessor createInvertedListFrameTupleAccessor(int frameSize, ITypeTraits[] typeTraits,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         if (checkTypeTraitsAllFixed(typeTraits)) {
             return new FixedSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits);
         } else {
-            return new VariableSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits);
+            return new VariableSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits, nullTypeTraits,
+                    nullIntrospector);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
index 9404d91..e6d50d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -61,12 +62,13 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
     }
 
     private ExternalRTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json,
@@ -90,7 +92,7 @@
                 opTrackerProvider.getOperationTracker(ncServiceCtx, this),
                 ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
                 linearizeCmpFactory, buddyBTreeFields, durable, isPointMBR, metadataPageManagerFactory,
-                ncServiceCtx.getTracer());
+                ncServiceCtx.getTracer(), nullTypeTraits, nullIntrospector);
 
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
index 1f4971d..ca8caec 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -49,12 +50,13 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
     }
 
     @Override
@@ -63,7 +65,8 @@
                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
                 mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
+                linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate,
+                nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
index c7a59f4..dd21f7c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -73,10 +74,12 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -117,7 +120,7 @@
                 opTrackerProvider.getOperationTracker(ncServiceCtx, this),
                 ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
                 linearizeCmpFactory, rtreeFields, buddyBTreeFields, filterTypeTraits, filterCmpFactories, filterFields,
-                durable, isPointMBR, metadataPageManagerFactory);
+                durable, isPointMBR, metadataPageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
index 31f78d8..873a023 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -60,10 +61,12 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -81,6 +84,6 @@
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories,
                 rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
index a0d0220..33b9da6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -70,10 +71,12 @@
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable) {
+            ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -108,7 +111,8 @@
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, pageWriteCallbackFactory, linearizeCmpFactory, rtreeFields, filterTypeTraits,
-                filterCmpFactories, filterFields, durable, isPointMBR, metadataPageManagerFactory);
+                filterCmpFactories, filterFields, durable, isPointMBR, metadataPageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
index ed4c5b5..e9b4c90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -57,10 +58,12 @@
             Map<String, String> mergePolicyProperties, boolean durable,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR,
-            IBinaryComparatorFactory[] btreeComparatorFactories) {
+            IBinaryComparatorFactory[] btreeComparatorFactories, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, rteeTypeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
         this.linearizeCmpFactory = linearizeCmpFactory;
@@ -75,6 +78,7 @@
                 cmpFactories, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider,
                 ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, btreeComparatorFactories,
-                valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR, durable);
+                valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR, durable,
+                nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
index 3b7c150..4f3b49b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
@@ -21,11 +21,13 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 
 public class LSMRTreeCopyTupleWriter extends LSMRTreeTupleWriter {
-    public LSMRTreeCopyTupleWriter(ITypeTraits[] typeTraits) {
+    public LSMRTreeCopyTupleWriter(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         // Third parameter is never used locally, just give false.
-        super(typeTraits, false);
+        super(typeTraits, false, nullTypeTraits, nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
index 6585ffe..41fa508 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
@@ -20,18 +20,20 @@
 package org.apache.hyracks.storage.am.lsm.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 
 public class LSMRTreeCopyTupleWriterFactory extends RTreeTypeAwareTupleWriterFactory {
     private static final long serialVersionUID = 1L;
 
-    public LSMRTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits) {
-        super(typeTraits);
+    public LSMRTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
     }
 
     @Override
     public RTreeTypeAwareTupleWriter createTupleWriter() {
-        return new LSMRTreeCopyTupleWriter(typeTraits);
+        return new LSMRTreeCopyTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
index 86a6a39..3e25e8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
@@ -26,8 +26,8 @@
 
 public class LSMRTreeTupleReference extends RTreeTypeAwareTupleReference implements ILSMTreeTupleReference {
 
-    public LSMRTreeTupleReference(ITypeTraits[] typeTraits) {
-        super(typeTraits);
+    public LSMRTreeTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
+        super(typeTraits, nullTypeTraits);
     }
 
     @Override
@@ -45,4 +45,10 @@
     public int getTupleStart() {
         return tupleStartOff;
     }
+
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        // 1 for antimatter
+        return fieldIdx + 1;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
index ef9eb47..7c05daa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
@@ -34,8 +34,8 @@
     private final boolean antimatterAware;
 
     public LSMRTreeTupleReferenceForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
-            boolean antimatterAware) {
-        super(typeTraits);
+            boolean antimatterAware, ITypeTraits nullTypeTraits) {
+        super(typeTraits, nullTypeTraits);
         this.inputKeyFieldCount = keyFieldCount;
         this.inputTotalFieldCount = keyFieldCount + valueFieldCount;
         this.storedKeyFieldCount = keyFieldCount / 2;
@@ -55,8 +55,15 @@
         int cumul = 0;
         //step1. decode field slots for stored key
         for (int i = 0; i < storedKeyFieldCount; i++) {
+            if (!typeTraits[i].isFixedLength()) {
+                cumul += encDec.decode();
+            } else if (nullTypeTraits != null && isNull(buf, tupleStartOff, i)) {
+                cumul += nullTypeTraits.getFixedLength();
+            } else {
+                cumul += typeTraits[i].getFixedLength();
+            }
             //key or value fields
-            cumul += typeTraits[i].getFixedLength();
+            //            cumul += typeTraits[i].getFixedLength();
             decodedFieldSlots[field++] = cumul;
         }
         //step2. decode field slots for non-stored (duplicated point) key
@@ -68,14 +75,22 @@
         encDec.reset(buf, tupleStartOff + nullFlagsBytes);
         for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++) {
             if (!typeTraits[i].isFixedLength()) {
-                //value fields
                 cumul += encDec.decode();
-                decodedFieldSlots[field++] = cumul;
+            } else if (nullTypeTraits != null && isNull(buf, tupleStartOff, i)) {
+                cumul += nullTypeTraits.getFixedLength();
             } else {
-                //key or value fields
                 cumul += typeTraits[i].getFixedLength();
-                decodedFieldSlots[field++] = cumul;
             }
+            decodedFieldSlots[field++] = cumul;
+            //            if (!typeTraits[i].isFixedLength()) {
+            //                //value fields
+            //                cumul += encDec.decode();
+            //                decodedFieldSlots[field++] = cumul;
+            //            } else {
+            //                //key or value fields
+            //                cumul += typeTraits[i].getFixedLength();
+            //                decodedFieldSlots[field++] = cumul;
+            //            }
         }
 
         dataStartOff = encDec.getPos();
@@ -145,4 +160,10 @@
         // Check antimatter bit.
         return BitOperationUtils.getBit(buf, tupleStartOff, ANTIMATTER_BIT_OFFSET);
     }
+
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        // 1 for antimatter
+        return antimatterAware ? fieldIdx + 1 : fieldIdx;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
index 5650cbb..6417839 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
@@ -30,14 +31,15 @@
 public class LSMRTreeTupleWriter extends RTreeTypeAwareTupleWriter implements ILSMTreeTupleWriter {
     private boolean isAntimatter;
 
-    public LSMRTreeTupleWriter(ITypeTraits[] typeTraits, boolean isAntimatter) {
-        super(typeTraits);
+    public LSMRTreeTupleWriter(ITypeTraits[] typeTraits, boolean isAntimatter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.isAntimatter = isAntimatter;
     }
 
     @Override
     public LSMRTreeTupleReference createTupleReference() {
-        return new LSMRTreeTupleReference(typeTraits);
+        return new LSMRTreeTupleReference(typeTraits, nullTypeTraits);
     }
 
     @Override
@@ -72,4 +74,9 @@
         this.isAntimatter = isAntimatter;
     }
 
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        // 1 for antimatter
+        return fieldIdx + 1;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
index 4ad5e3a..040c805 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 
 public class LSMRTreeTupleWriterFactory extends RTreeTypeAwareTupleWriterFactory {
@@ -27,14 +28,15 @@
     private static final long serialVersionUID = 1L;
     private final boolean isAntimatter;
 
-    public LSMRTreeTupleWriterFactory(ITypeTraits[] typeTraits, boolean isAntimatter) {
-        super(typeTraits);
+    public LSMRTreeTupleWriterFactory(ITypeTraits[] typeTraits, boolean isAntimatter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.isAntimatter = isAntimatter;
     }
 
     @Override
     public LSMRTreeTupleWriter createTupleWriter() {
-        return new LSMRTreeTupleWriter(typeTraits, isAntimatter);
+        return new LSMRTreeTupleWriter(typeTraits, isAntimatter, nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
index 6a87af2..625883f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 
@@ -32,8 +33,8 @@
     private final boolean isAntimatter;
 
     public LSMRTreeTupleWriterFactoryForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
-            boolean antimatterAware, boolean isDelete) {
-        super(typeTraits);
+            boolean antimatterAware, boolean isDelete, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.keyFieldCount = keyFieldCount;
         this.valueFieldCount = valueFieldCount;
         this.antimatterAware = antimatterAware;
@@ -43,7 +44,7 @@
     @Override
     public RTreeTypeAwareTupleWriter createTupleWriter() {
         return new LSMRTreeTupleWriterForPointMBR(typeTraits, keyFieldCount, valueFieldCount, antimatterAware,
-                isAntimatter);
+                isAntimatter, nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
index ae2d642..7542524 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
@@ -54,8 +55,9 @@
     private boolean isAntimatter;
 
     public LSMRTreeTupleWriterForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
-            boolean antimatterAware, boolean isAntimatter) {
-        super(typeTraits);
+            boolean antimatterAware, boolean isAntimatter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.inputKeyFieldCount = keyFieldCount;
         this.valueFieldCount = valueFieldCount;
         this.inputTotalFieldCount = keyFieldCount + valueFieldCount;
@@ -81,14 +83,15 @@
 
     @Override
     public LSMRTreeTupleReferenceForPointMBR createTupleReference() {
-        return new LSMRTreeTupleReferenceForPointMBR(typeTraits, inputKeyFieldCount, valueFieldCount, antimatterAware);
+        return new LSMRTreeTupleReferenceForPointMBR(typeTraits, inputKeyFieldCount, valueFieldCount, antimatterAware,
+                nullTypeTraits);
     }
 
     @Override
     public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
         int runner = targetOff;
         int nullFlagsBytes = getNullFlagsBytes(tuple);
-        // write null indicator bits
+        // reset null indicator bits
         for (int i = 0; i < nullFlagsBytes; i++) {
             targetBuf[runner++] = (byte) 0;
         }
@@ -102,8 +105,14 @@
 
         // write key fields
         for (int i = 0; i < storedKeyFieldCount; i++) {
-            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
-            runner += tuple.getFieldLength(i);
+            byte[] fieldData = tuple.getFieldData(i);
+            int fieldOffset = tuple.getFieldStart(i);
+            int fieldLength = tuple.getFieldLength(i);
+            if (nullIntrospector != null && nullIntrospector.isNull(fieldData, fieldOffset, fieldLength)) {
+                setNullFlag(targetBuf, targetOff, i);
+            }
+            System.arraycopy(fieldData, fieldOffset, targetBuf, runner, fieldLength);
+            runner += fieldLength;
         }
         // write value fields
         for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++) {
@@ -138,7 +147,7 @@
 
     @Override
     protected int getNullFlagsBytes(ITupleReference tuple) {
-        return BitOperationUtils.getFlagBytes(storedTotalFieldCount + (antimatterAware ? 1 : 0));
+        return BitOperationUtils.getFlagBytes(storedTotalFieldCount + valueFieldCount + (antimatterAware ? 1 : 0));
     }
 
     @Override
@@ -167,4 +176,9 @@
     public void setAntimatter(boolean isAntimatter) {
         this.isAntimatter = isAntimatter;
     }
+
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        return antimatterAware ? fieldIdx + 1 : fieldIdx;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index cf44b2f..a436c43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
@@ -85,7 +86,8 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
             int[] rtreeFields, int[] buddyBTreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
-            IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+            IMetadataPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         int valueFieldCount = buddyBTreeFields.length;
         int keyFieldCount = typeTraits.length - valueFieldCount;
         ITypeTraits[] btreeTypeTraits = new ITypeTraits[valueFieldCount];
@@ -93,16 +95,16 @@
             btreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
         }
         RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
-                new RTreeTypeAwareTupleWriterFactory(typeTraits);
+                new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory = null;
         if (isPointMBR) {
-            rtreeLeafFrameTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, false, false);
+            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, false, false, nullTypeTraits, nullIntrospector);
         } else {
             rtreeLeafFrameTupleWriterFactory = rtreeInteriorFrameTupleWriterFactory;
         }
         BTreeTypeAwareTupleWriterFactory btreeTupleWriterFactory =
-                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, null, null);
+                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, nullTypeTraits, nullIntrospector);
         RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
                 rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
         RTreeFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeLeafFrameTupleWriterFactory,
@@ -128,7 +130,7 @@
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -155,28 +157,33 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             ILinearizeComparatorFactory linearizerCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
-            IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+            IMetadataPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
-                new LSMRTreeTupleWriterFactory(typeTraits, false);
+                new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory;
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameCopyTupleWriterFactory;
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameBulkLoadWriterFactory;
         if (isPointMBR) {
             int keyFieldCount = rtreeCmpFactories.length;
             int valueFieldCount = btreeComparatorFactories.length - keyFieldCount;
-            rtreeLeafFrameTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
-            rtreeLeafFrameCopyTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
-            rtreeLeafFrameBulkLoadWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
+            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameCopyTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameBulkLoadWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
         } else {
-            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
-            rtreeLeafFrameCopyTupleWriterFactory = new LSMRTreeCopyTupleWriterFactory(typeTraits);
-            rtreeLeafFrameBulkLoadWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
+            rtreeLeafFrameTupleWriterFactory =
+                    new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameCopyTupleWriterFactory =
+                    new LSMRTreeCopyTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameBulkLoadWriterFactory =
+                    new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
         }
 
-        LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
+        LSMRTreeTupleWriterFactory btreeTupleWriterFactory =
+                new LSMRTreeTupleWriterFactory(typeTraits, true, nullTypeTraits, nullIntrospector);
 
         RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
                 rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
@@ -219,7 +226,7 @@
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -247,8 +254,8 @@
             ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] buddyBTreeFields, boolean durable,
-            boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer)
-            throws HyracksDataException {
+            boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
 
         int keyFieldCount = rtreeCmpFactories.length;
         int valueFieldCount = typeTraits.length - keyFieldCount;
@@ -257,16 +264,16 @@
             btreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
         }
         RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
-                new RTreeTypeAwareTupleWriterFactory(typeTraits);
+                new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory = null;
         if (isPointMBR) {
-            rtreeLeafFrameTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, false, false);
+            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, false, false, nullTypeTraits, nullIntrospector);
         } else {
             rtreeLeafFrameTupleWriterFactory = rtreeInteriorFrameTupleWriterFactory;
         }
         BTreeTypeAwareTupleWriterFactory btreeTupleWriterFactory =
-                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, null, null);
+                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, nullTypeTraits, nullIntrospector);
         RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
                 rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
         RTreeFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeLeafFrameTupleWriterFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
index f9eb844..9195982 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -42,10 +43,13 @@
     private final IPageManagerFactory pageManagerFactory;
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
     private final RTreePolicyType rtreePolicyType;
+    private final ITypeTraits nullTypeTraits;
+    private final INullIntrospector nullIntrospector;
 
     public RTreeResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType) {
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         this.path = path;
         this.storageManager = storageManager;
         this.typeTraits = typeTraits;
@@ -53,6 +57,8 @@
         this.pageManagerFactory = pageManagerFactory;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -60,7 +66,8 @@
         IIOManager ioManager = ctx.getIoManager();
         FileReference resourceRef = ioManager.resolve(path);
         return RTreeUtils.createRTree(storageManager.getBufferCache(ctx), typeTraits, valueProviderFactories,
-                comparatorFactories, rtreePolicyType, resourceRef, false, pageManagerFactory);
+                comparatorFactories, rtreePolicyType, resourceRef, false, pageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
index 1a3bb0b..9ef59a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
@@ -21,6 +21,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -37,22 +38,27 @@
     private final IPageManagerFactory pageManagerFactory;
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
     private final RTreePolicyType rtreePolicyType;
+    private final ITypeTraits nullTypeTraits;
+    private final INullIntrospector nullIntrospector;
 
     public RTreeResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType) {
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         this.storageManager = storageManager;
         this.typeTraits = typeTraits;
         this.comparatorFactories = comparatorFactories;
         this.pageManagerFactory = pageManagerFactory;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
     public IResource createResource(FileReference fileRef) {
         return new RTreeResource(fileRef.getRelativePath(), storageManager, typeTraits, comparatorFactories,
-                pageManagerFactory, valueProviderFactories, rtreePolicyType);
+                pageManagerFactory, valueProviderFactories, rtreePolicyType, nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
index 8f7f3f0..8f1b660 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
@@ -26,8 +26,8 @@
 
 public class RTreeTypeAwareTupleReference extends TypeAwareTupleReference implements ITreeIndexTupleReference {
 
-    public RTreeTypeAwareTupleReference(ITypeTraits[] typeTraits) {
-        super(typeTraits, null);
+    public RTreeTypeAwareTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
+        super(typeTraits, nullTypeTraits);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
index 42a9662..02bf4a2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
@@ -22,14 +22,16 @@
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
 
 public class RTreeTypeAwareTupleWriter extends TypeAwareTupleWriter {
 
-    public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits) {
-        super(typeTraits, null, null);
+    public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
     }
 
     public int writeTupleFields(ITreeIndexTupleReference[] refs, int startField, ByteBuffer targetBuf, int targetOff) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
index 0069cc3..56241e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
@@ -20,18 +20,20 @@
 package org.apache.hyracks.storage.am.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 
 public class RTreeTypeAwareTupleWriterFactory extends TypeAwareTupleWriterFactory {
 
     private static final long serialVersionUID = 1L;
 
-    public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
-        super(typeTraits, null, null);
+    public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
     }
 
     @Override
     public RTreeTypeAwareTupleWriter createTupleWriter() {
-        return new RTreeTypeAwareTupleWriter(typeTraits);
+        return new RTreeTypeAwareTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
index 204248f..7df5802 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -45,9 +46,10 @@
     public static RTree createRTree(IBufferCache bufferCache, ITypeTraits[] typeTraits,
             IPrimitiveValueProviderFactory[] valueProviderFactories, IBinaryComparatorFactory[] cmpFactories,
             RTreePolicyType rtreePolicyType, FileReference file, boolean isPointMBR,
-            IPageManagerFactory pageManagerFactory) {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
 
-        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
+        RTreeTypeAwareTupleWriterFactory tupleWriterFactory =
+                new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
         ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
                 valueProviderFactories, rtreePolicyType, isPointMBR);
         ITreeIndexFrameFactory leafFrameFactory =
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
index 0c626df..1b17ef9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
@@ -65,7 +65,7 @@
                 .resolveAbsolutePath(harness.getInvListsFileRef().getFile().getAbsolutePath() + "_btree");
         index = new OnDiskInvertedIndex(harness.getDiskBufferCache(), invListBuilder, invListTypeTraits,
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, harness.getInvListsFileRef(), btreeFile,
-                harness.getMetadataPageManagerFactory());
+                harness.getMetadataPageManagerFactory(), null, null);
 
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 3e8db34..e745f5d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -159,7 +159,8 @@
                 invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCaches().get(0),
                         new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()), null,
+                        null);
                 break;
             }
             case PARTITIONED_INMEMORY: {
@@ -167,47 +168,53 @@
                         harness.getVirtualBufferCaches().get(0),
                         new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()), null,
+                        null);
                 break;
             }
             case ONDISK: {
                 invIndex = InvertedIndexUtils.createOnDiskInvertedIndex(ioManager, harness.getDiskBufferCache(),
                         invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                        harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory());
+                        harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory(), null, null);
                 break;
             }
             case PARTITIONED_ONDISK: {
-                invIndex = InvertedIndexUtils.createPartitionedOnDiskInvertedIndex(ioManager,
-                        harness.getDiskBufferCache(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory());
+                invIndex =
+                        InvertedIndexUtils.createPartitionedOnDiskInvertedIndex(ioManager, harness.getDiskBufferCache(),
+                                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                                harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory(), null, null);
                 break;
             }
             case LSM: {
-                invIndex = InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
-                        invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
-                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                        harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
-                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
-                        harness.getMetadataPageManagerFactory(),
-                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
-                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
+                invIndex =
+                        InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+                                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                                tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
+                                harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+                                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                                invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+                                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+                                harness.getMetadataPageManagerFactory(),
+                                new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                        ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
+                                null, null);
                 break;
             }
             case PARTITIONED_LSM: {
-                invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager,
-                        harness.getVirtualBufferCaches(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
-                        harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
-                        harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
-                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
-                        harness.getMetadataPageManagerFactory(),
-                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
-                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
+                invIndex = InvertedIndexUtils
+                        .createPartitionedLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+                                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                                tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
+                                harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+                                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                                invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+                                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+                                harness.getMetadataPageManagerFactory(),
+                                new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                        ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
+                                null, null);
                 break;
             }
             default: {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index 3ca25ab..9e6fb83 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -50,7 +50,7 @@
                 harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields, btreeFields,
                 filterTypeTraits, filterCmpFactories, filterFields, true, false,
-                harness.getMetadataPageManagerFactory());
+                harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Before
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 75f656d..baf54f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -50,7 +50,7 @@
                 harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields,
                 filterTypeTraits, filterCmpFactories, filterFields, true, false,
-                harness.getMetadataPageManagerFactory());
+                harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Before
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index bbf6263..980f21f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -66,7 +66,7 @@
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, btreeFields, null,
-                null, null, true, false, harness.getMetadataPageManagerFactory());
+                null, null, true, false, harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index 7d6e2e6..f40add9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -67,7 +67,7 @@
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, null, null, null, true,
-                false, harness.getMetadataPageManagerFactory());
+                false, harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index 2140558..02742a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -109,7 +109,7 @@
                 typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
-                null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory);
+                null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory, null, null);
         LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index f9d9046..ff953f1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -90,7 +90,7 @@
                 virtualBufferCaches, file, diskBufferCache, typeTraits, rtreeCmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
-                null, null, null, null, true, false, metadataPageManagerFactory);
+                null, null, null, null, true, false, metadataPageManagerFactory, null, null);
         LSMRTreeWithAntiMatterTuplesTestContext testCtx =
                 new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes, lsmTree);
         return testCtx;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
index 999fd65..cb68d77 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
@@ -54,7 +54,7 @@
             RTreePolicyType rtreePolicyType, int[] rtreeFields, int[] btreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException {
         return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits, valueProviderFactories, rtreeCmpFactories,
-                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory());
+                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory(), null, null);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index bef7ac7..19672dd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -76,7 +76,7 @@
     public static final IPrimitiveValueProviderFactory[] VALUE_PROVIDER_FACTORY =
             RTreeUtils.createPrimitiveValueProviderFactories(CMP_FACTORIES.length, IntegerPointable.FACTORY);
     public static final RTreeTypeAwareTupleWriterFactory TUPLE_WRITER_FACTORY =
-            new RTreeTypeAwareTupleWriterFactory(TYPE_TRAITS);
+            new RTreeTypeAwareTupleWriterFactory(TYPE_TRAITS, null, null);
     public static final ITreeIndexMetadataFrameFactory META_FRAME_FACTORY = new LIFOMetaDataFrameFactory();
     public static final ITreeIndexFrameFactory INTERIOR_FRAME_FACTORY = new RTreeNSMInteriorFrameFactory(
             TUPLE_WRITER_FACTORY, VALUE_PROVIDER_FACTORY, RTreePolicyType.RTREE, false);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
index faa63c5..02c267d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
@@ -61,7 +61,7 @@
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, int[] btreeFields) throws HyracksDataException {
         return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits, valueProviderFactories, rtreeCmpFactories,
-                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory());
+                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory(), null, null);
 
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
index bf70dc1..06ee100 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
@@ -60,7 +60,7 @@
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
         RTree rtree = RTreeUtils.createRTree(bufferCache, typeTraits, valueProviderFactories, cmpFactories,
-                rtreePolicyType, file, false, pageManagerFactory);
+                rtreePolicyType, file, false, pageManagerFactory, null, null);
         RTreeTestContext testCtx = new RTreeTestContext(fieldSerdes, rtree);
         return testCtx;
     }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/ptCbmTBkLqr7Lnsf8 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-CC: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sat, 31 Jul 2021 23:58:52 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )


Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................

[ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees

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

Details:
When writing an index entry into RTree, set the corresponding null flags
for keys that are considered null. When reading an index entry,
examine the null flags to read the index entry properly.
Also, pass the null type traits and null introspector to inverted indexes.

Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
64 files changed, 478 insertions(+), 241 deletions(-)



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

diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 86d8ed4..f4d3dbe 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -107,7 +107,7 @@
         TestUtils.redirectLoggingToConsole();
         AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
         try {
-            integrationUtil.run(Boolean.getBoolean("cleanup.start"), Boolean.getBoolean("cleanup.shutdown"),
+            integrationUtil.run(true, Boolean.getBoolean("cleanup.shutdown"),
                     System.getProperty("conf.path", DEFAULT_CONF_FILE));
         } catch (Exception e) {
             LOGGER.fatal("Unexpected exception", e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index 8b69652..970c5a8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -31,15 +31,18 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
 import org.apache.asterix.metadata.api.IResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.data.std.accessors.ShortBinaryComparatorFactory;
@@ -129,13 +132,15 @@
                 FullTextUtil.fetchFilterAndCreateConfigEvaluator(mdProvider, index.getDataverseName(),
                         indexDetails.getFullTextConfigName());
 
+        ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
         return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
                 filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
                 secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, typeTraitProvider.getTypeTrait(BuiltinType.ANULL),
+                NullIntrospector.INSTANCE);
     }
 
     // Returns an array of the type traits of the inverted list elements
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index e00d7b6..22f968a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.nontagged.NullIntrospector;
 import org.apache.asterix.metadata.api.IResourceFactoryProvider;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -35,6 +36,7 @@
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.StringUtils;
@@ -155,6 +157,7 @@
                 dataset.getDatasetType() == DatasetType.EXTERNAL ? IndexingConstants.getBuddyBtreeComparatorFactories()
                         : getComparatorFactoriesForDeletedKeyBTree(secondaryTypeTraits, primaryComparatorFactories,
                                 secondaryComparatorFactories);
+        ITypeTraitProvider typeTraitProvider = mdProvider.getDataFormat().getTypeTraitProvider();
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             AsterixVirtualBufferCacheProvider vbcProvider =
                     new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
@@ -162,14 +165,16 @@
                     filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                     pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                     mergePolicyFactory, mergePolicyProperties, true, valueProviderFactories, rTreePolicyType,
-                    linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories);
+                    linearizeCmpFactory, rtreeFields, isPointMBR, btreeCompFactories,
+                    typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
         } else {
             return new ExternalRTreeLocalResourceFactory(storageManager, typeTraits, rtreeCmpFactories,
                     filterTypeTraits, filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
                     pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
                     mergePolicyProperties, true, btreeCompFactories, valueProviderFactories, rTreePolicyType,
                     linearizeCmpFactory, rtreeFields, new int[] { numNestedSecondaryKeyFields }, isPointMBR,
-                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate());
+                    mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate(),
+                    typeTraitProvider.getTypeTrait(BuiltinType.ANULL), NullIntrospector.INSTANCE);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index 3b3fc1e..1725ea3 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -54,6 +54,6 @@
                 NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory, getVirtualBufferCacheProvider(),
                 SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY, MERGE_POLICY_PROPERTIES, DURABLE,
                 btreeComparatorFactories, valueProviderFactories, rtreePolicyType, linearizerCmpFactory, null,
-                btreeFields, IS_POINT_MBR, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
+                btreeFields, IS_POINT_MBR, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, null, null);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
index 0bb483a..28b9e88 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -54,6 +54,6 @@
                 NoOpIOOperationCallbackFactory.INSTANCE, NoOpPageWriteCallbackFactory.INSTANCE, pageManagerFactory,
                 getVirtualBufferCacheProvider(), SynchronousSchedulerProvider.INSTANCE, MERGE_POLICY_FACTORY,
                 MERGE_POLICY_PROPERTIES, DURABLE, valueProviderFactories, rtreePolicyType, linearizerCmpFactory,
-                btreeFields, IS_POINT_MBR, btreeComparatorFactories);
+                btreeFields, IS_POINT_MBR, btreeComparatorFactories, null, null);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
index 43d685e..1981c1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
@@ -38,6 +38,6 @@
             int[] btreeFields, ITypeTraits[] secondaryTypeTraits,
             IBinaryComparatorFactory[] secondaryComparatorFactories, IMetadataPageManagerFactory pageManagerFactory) {
         return new RTreeResourceFactory(storageManager, secondaryTypeTraits, secondaryComparatorFactories,
-                pageManagerFactory, valueProviderFactories, rtreePolicyType);
+                pageManagerFactory, valueProviderFactories, rtreePolicyType, null, null);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
index 8eff435..084c89f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
@@ -131,7 +131,7 @@
         return dataStartOff - tupleStartOff + decodedFieldSlots[fieldCount - 1];
     }
 
-    private boolean isNull(byte[] flags, int flagsOffset, int fieldIdx) {
+    protected boolean isNull(byte[] flags, int flagsOffset, int fieldIdx) {
         int adjustedFieldIdx = getAdjustedFieldIdx(fieldIdx);
         int flagByteIdx = adjustedFieldIdx / 8;
         int flagBitIdx = 7 - (adjustedFieldIdx % 8);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
index 2528518..882be88 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
@@ -182,7 +182,7 @@
      * @param flagsOffset start of the null flags data
      * @param fieldIdx logical field index
      */
-    private void setNullFlag(byte[] flags, int flagsOffset, int fieldIdx) {
+    protected void setNullFlag(byte[] flags, int flagsOffset, int fieldIdx) {
         int adjustedFieldIdx = getAdjustedFieldIdx(fieldIdx);
         int flagByteIdx = adjustedFieldIdx / 8;
         int flagBitIdx = 7 - (adjustedFieldIdx % 8);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
index a693a99..ed44dfa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
@@ -21,6 +21,7 @@
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.common.IIndex;
 
 public interface IInvertedIndex extends IIndex {
@@ -32,4 +33,8 @@
     ITypeTraits[] getTokenTypeTraits();
 
     IBinaryComparatorFactory[] getTokenCmpFactories();
+
+    ITypeTraits getNullTypeTraits();
+
+    INullIntrospector getNullIntrospector();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 8550385..8a107e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -90,11 +91,13 @@
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
             boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
 
         super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.tokenizerFactory = tokenizerFactory;
@@ -141,7 +144,7 @@
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory, serviceCtx.getTracer());
+                    metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
         } else {
             return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
                     tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, bufferCache,
@@ -149,7 +152,7 @@
                     opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
                     pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
                     filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
-                    metadataPageManagerFactory, serviceCtx.getTracer());
+                    metadataPageManagerFactory, serviceCtx.getTracer(), nullTypeTraits, nullIntrospector);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
index 79c6123..395331f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
@@ -62,10 +63,12 @@
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
             boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+            int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         // ToDo: totally replace tokenizerFactory with full-text config
@@ -85,7 +88,8 @@
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, tokenTypeTraits, tokenCmpFactories,
                 tokenizerFactory, fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
-                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate,
+                nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
index 7ec6ce2..9886da4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.dataflow.common.utils.TaskUtil;
 import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
 import org.apache.hyracks.dataflow.std.buffermanager.SingleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -68,6 +69,8 @@
     protected int bufferNumLoadedPages;
 
     protected final IInvertedListTupleReference tuple;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
     protected final ITypeTraits[] invListFields;
     protected ICachedPage page;
     // buffer manager to conform to the memory budget
@@ -79,19 +82,22 @@
     protected final IIndexCursorStats stats;
 
     protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
-        this(bufferCache, fileId, invListFields, ctx, stats, false);
+            IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        this(bufferCache, fileId, invListFields, ctx, stats, false, nullTypeTraits, nullIntrospector);
     }
 
     protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IIndexCursorStats stats) throws HyracksDataException {
-        this(bufferCache, fileId, invListFields, null, stats, true);
+            IIndexCursorStats stats, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
+        this(bufferCache, fileId, invListFields, null, stats, true, nullTypeTraits, nullIntrospector);
     }
 
     // If isScan, use the SingleFrameBufferManager to minimize memory cost by allocating only one memory frame,
     // elsewhere use a regular buffer manager
     protected AbstractOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IHyracksTaskContext ctx, IIndexCursorStats stats, boolean isScan) throws HyracksDataException {
+            IHyracksTaskContext ctx, IIndexCursorStats stats, boolean isScan, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         this.bufferCache = bufferCache;
         this.fileId = fileId;
 
@@ -104,7 +110,7 @@
         this.lastRandomSearchedElementIx = 0;
         this.moreBlocksToRead = true;
         this.invListFields = invListFields;
-        this.tuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields);
+        this.tuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields, nullTypeTraits);
         this.buffers = new ArrayList<ByteBuffer>();
         if (ctx == null && !isScan) {
             throw HyracksDataException.create(ErrorCode.CANNOT_CONTINUE_TEXT_SEARCH_HYRACKS_TASK_IS_NULL);
@@ -118,6 +124,8 @@
             this.bufferManagerForSearch = new SingleFrameBufferManager();
         }
         this.stats = stats;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index f2d5a37..a828d22 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -35,6 +35,7 @@
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
@@ -92,6 +93,8 @@
     protected final IBinaryComparatorFactory[] tokenCmpFactories;
     private final int[] filterFieldsForNonBulkLoadOps;
     private final int[] invertedIndexFieldsForNonBulkLoadOps;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
 
     public LSMInvertedIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
             ILSMDiskComponentFactory componentFactory, IComponentFilterHelper filterHelper,
@@ -103,7 +106,8 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
                 opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
                 componentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
@@ -116,14 +120,17 @@
         this.tokenCmpFactories = tokenCmpFactories;
         this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
         this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
+        this.nullIntrospector = nullIntrospector;
+        this.nullTypeTraits = nullTypeTraits;
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             InMemoryInvertedIndex memInvIndex =
                     createInMemoryInvertedIndex(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache), i);
-            BTree deleteKeysBTree = BTreeUtils.createBTree(virtualBufferCache,
-                    new VirtualFreePageManager(virtualBufferCache), invListTypeTraits, invListCmpFactories,
-                    BTreeLeafFrameType.REGULAR_NSM,
-                    ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false, null, null);
+            BTree deleteKeysBTree =
+                    BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
+                            invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM,
+                            ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false,
+                            nullTypeTraits, nullIntrospector);
             LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(this, memInvIndex,
                     deleteKeysBTree, virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
             memoryComponents.add(mutableComponent);
@@ -410,7 +417,8 @@
         return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory,
-                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id), nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
@@ -444,6 +452,16 @@
         return tokenCmpFactories;
     }
 
+    @Override
+    public ITypeTraits getNullTypeTraits() {
+        return nullTypeTraits;
+    }
+
+    @Override
+    public INullIntrospector getNullIntrospector() {
+        return nullIntrospector;
+    }
+
     public IBinaryTokenizerFactory getTokenizerFactory() {
         return tokenizerFactory;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index de1fc49..a801146 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
@@ -56,12 +57,14 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
-            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
+            int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
                 bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
                 tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, mergePolicy,
                 opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
-                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer,
+                nullTypeTraits, nullIntrospector);
     }
 
     @Override
@@ -70,7 +73,8 @@
         return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(virtualBufferCache,
                 virtualFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
                 tokenizerFactory, fullTextConfigEvaluatorFactory,
-                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+                ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id), nullTypeTraits,
+                nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index c2e07b3..7ee6af8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -31,6 +31,7 @@
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
@@ -55,12 +56,15 @@
 
     protected final ITypeTraits[] btreeTypeTraits;
     protected final IBinaryComparatorFactory[] btreeCmpFactories;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
 
     public InMemoryInvertedIndex(IBufferCache virtualBufferCache, IPageManager virtualFreePageManager,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
-            FileReference btreeFileRef) throws HyracksDataException {
+            FileReference btreeFileRef, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.invListTypeTraits = invListTypeTraits;
@@ -79,8 +83,11 @@
             btreeTypeTraits[tokenTypeTraits.length + i] = invListTypeTraits[i];
             btreeCmpFactories[tokenTypeTraits.length + i] = invListCmpFactories[i];
         }
-        this.btree = BTreeUtils.createBTree(virtualBufferCache, virtualFreePageManager, btreeTypeTraits,
-                btreeCmpFactories, BTreeLeafFrameType.REGULAR_NSM, btreeFileRef, false, null, null);
+        this.btree =
+                BTreeUtils.createBTree(virtualBufferCache, virtualFreePageManager, btreeTypeTraits, btreeCmpFactories,
+                        BTreeLeafFrameType.REGULAR_NSM, btreeFileRef, false, nullTypeTraits, nullIntrospector);
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -219,6 +226,16 @@
     }
 
     @Override
+    public ITypeTraits getNullTypeTraits() {
+        return nullTypeTraits;
+    }
+
+    @Override
+    public INullIntrospector getNullIntrospector() {
+        return nullIntrospector;
+    }
+
+    @Override
     public int getNumOfFilterFields() {
         return 0;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index bd9ce60..e3bce41 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
@@ -49,9 +50,11 @@
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
-            FileReference btreeFileRef) throws HyracksDataException {
+            FileReference btreeFileRef, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         super(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
+                tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef, nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
index 0d00ed3..f4086cc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
@@ -20,6 +20,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.fixedsize.FixedSizeElementInvertedListBuilder;
@@ -31,10 +32,15 @@
     protected final ITypeTraits[] invListFields;
     protected final ITypeTraits[] tokenTypeTraits;
     private final boolean isFixedSize;
+    private final ITypeTraits nullTypeTraits;
+    private final INullIntrospector nullIntrospector;
 
-    public InvertedListBuilderFactory(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields) {
+    public InvertedListBuilderFactory(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         this.tokenTypeTraits = tokenTypeTraits;
         this.invListFields = invListFields;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
 
         isFixedSize = InvertedIndexUtils.checkTypeTraitsAllFixed(invListFields);
     }
@@ -44,7 +50,8 @@
         if (isFixedSize) {
             return new FixedSizeElementInvertedListBuilder(invListFields);
         } else {
-            return new VariableSizeElementInvertedListBuilder(tokenTypeTraits, invListFields);
+            return new VariableSizeElementInvertedListBuilder(tokenTypeTraits, invListFields, nullTypeTraits,
+                    nullIntrospector);
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 7a45c5d..48a173e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
@@ -103,6 +104,8 @@
     protected int rootPageId = 0;
     protected IBufferCache bufferCache;
     protected int fileId = -1;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
     protected final ITypeTraits[] invListTypeTraits;
     protected final IBinaryComparatorFactory[] invListCmpFactories;
     protected final ITypeTraits[] tokenTypeTraits;
@@ -119,7 +122,8 @@
     public OnDiskInvertedIndex(IBufferCache bufferCache, IInvertedListBuilder invListBuilder,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile,
-            FileReference invListsFile, IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            FileReference invListsFile, IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         this.bufferCache = bufferCache;
         this.invListBuilder = invListBuilder;
         this.invListTypeTraits = invListTypeTraits;
@@ -128,7 +132,7 @@
         this.tokenCmpFactories = tokenCmpFactories;
         this.btree = BTreeUtils.createDiskBTree(bufferCache, getBTreeTypeTraits(tokenTypeTraits), tokenCmpFactories,
                 BTreeLeafFrameType.REGULAR_NSM, btreeFile, pageManagerFactory.createPageManager(bufferCache), false,
-                null, null);
+                nullTypeTraits, nullIntrospector);
         this.numTokenFields = btree.getComparatorFactories().length;
         this.numInvListKeys = invListCmpFactories.length;
         this.invListsFile = invListsFile;
@@ -136,6 +140,8 @@
         this.invListEndPageIdField = numTokenFields + 1;
         this.invListStartOffField = numTokenFields + 2;
         this.invListNumElementsField = numTokenFields + 3;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -197,10 +203,10 @@
     public IInvertedListCursor createInvertedListCursor(IHyracksTaskContext ctx) throws HyracksDataException {
         if (InvertedIndexUtils.checkTypeTraitsAllFixed(invListTypeTraits)) {
             return new FixedSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, ctx,
-                    NoOpIndexCursorStats.INSTANCE);
+                    NoOpIndexCursorStats.INSTANCE, nullTypeTraits, nullIntrospector);
         } else {
             return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, ctx,
-                    NoOpIndexCursorStats.INSTANCE);
+                    NoOpIndexCursorStats.INSTANCE, nullTypeTraits, nullIntrospector);
         }
     }
 
@@ -210,7 +216,8 @@
         if (InvertedIndexUtils.checkTypeTraitsAllFixed(invListTypeTraits)) {
             return new FixedSizeElementInvertedListScanCursor(bufferCache, fileId, invListTypeTraits, stats);
         } else {
-            return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, stats);
+            return new VariableSizeElementOnDiskInvertedListCursor(bufferCache, fileId, invListTypeTraits, stats,
+                    nullTypeTraits, nullIntrospector);
         }
     }
 
@@ -700,6 +707,16 @@
     }
 
     @Override
+    public ITypeTraits getNullTypeTraits() {
+        return nullTypeTraits;
+    }
+
+    @Override
+    public INullIntrospector getNullIntrospector() {
+        return nullIntrospector;
+    }
+
+    @Override
     public int getNumOfFilterFields() {
         return 0;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
index f058e4a..780f36d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.common.impls.IndexFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
@@ -39,12 +40,14 @@
     protected final ITypeTraits[] tokenTypeTraits;
     protected final IBinaryComparatorFactory[] tokenCmpFactories;
     protected final IInvertedIndexFileNameMapper fileNameMapper;
+    protected final ITypeTraits nullTypeTraits;
+    protected final INullIntrospector nullIntrospector;
 
     public OnDiskInvertedIndexFactory(IIOManager ioManager, IBufferCache bufferCache,
             IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper,
-            IPageManagerFactory pageManagerFactory) {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         super(ioManager, bufferCache, pageManagerFactory);
         this.invListBuilderFactory = invListBuilderFactory;
         this.invListTypeTraits = invListTypeTraits;
@@ -52,6 +55,8 @@
         this.tokenTypeTraits = tokenTypeTraits;
         this.tokenCmpFactories = tokenCmpFactories;
         this.fileNameMapper = fileNameMapper;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -60,6 +65,7 @@
         FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
         IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
         return new OnDiskInvertedIndex(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory);
+                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index fa5c365..ca4d94c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.data.std.primitive.ShortPointable;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
@@ -46,9 +47,10 @@
     public PartitionedOnDiskInvertedIndex(IBufferCache bufferCache, IInvertedListBuilder invListBuilder,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile,
-            FileReference invListsFile, IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            FileReference invListsFile, IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         super(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                btreeFile, invListsFile, pageManagerFactory);
+                btreeFile, invListsFile, pageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     public class PartitionedOnDiskInvertedIndexAccessor extends OnDiskInvertedIndexAccessor {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
index 7f10658..f9f4012 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
@@ -35,9 +36,9 @@
             IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper,
-            IPageManagerFactory pageManagerFactory) {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         super(ioManager, bufferCache, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, fileNameMapper, pageManagerFactory);
+                tokenCmpFactories, fileNameMapper, pageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     @Override
@@ -46,6 +47,7 @@
         FileReference invListsFile = ioManager.resolveAbsolutePath(invListsFilePath);
         IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
         return new PartitionedOnDiskInvertedIndex(bufferCache, invListBuilder, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory);
+                tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile, freePageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
index 11c0901..ce6daa8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
 import org.apache.hyracks.storage.am.lsm.invertedindex.impls.AbstractOnDiskInvertedListCursor;
 import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -49,11 +50,12 @@
     private int[] elementIndexes = new int[10];
 
     public FixedSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields,
-            IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
-        super(bufferCache, fileId, invListFields, ctx, stats);
+            IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        super(bufferCache, fileId, invListFields, ctx, stats, nullTypeTraits, nullIntrospector);
 
         this.bufferEndElementIx = 0;
-        this.bufferEndElementTuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields);
+        this.bufferEndElementTuple = InvertedIndexUtils.createInvertedListTupleReference(invListFields, nullTypeTraits);
 
         int tmpSize = 0;
         for (int i = 0; i < invListFields.length; i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
index 3ce26c5..c667487 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
@@ -22,6 +22,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.AbstractInvertedListBuilder;
@@ -35,8 +36,8 @@
 
     // The tokenTypeTraits is necessary because the underlying TypeAwareTupleWriter requires all the type traits of the tuple
     // even if the first a few fields in the tuple are never accessed by the writer
-    public VariableSizeElementInvertedListBuilder(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields)
-            throws HyracksDataException {
+    public VariableSizeElementInvertedListBuilder(ITypeTraits[] tokenTypeTraits, ITypeTraits[] invListFields,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         super(invListFields);
 
         this.allFields = new ITypeTraits[invListFields.length + tokenTypeTraits.length];
@@ -46,7 +47,7 @@
         for (int i = 0; i < invListFields.length; i++) {
             allFields[i + tokenTypeTraits.length] = invListFields[i];
         }
-        this.writer = new TypeAwareTupleWriter(allFields, null, null);
+        this.writer = new TypeAwareTupleWriter(allFields, nullTypeTraits, nullIntrospector);
 
         InvertedIndexUtils.verifyHasVarSizeTypeTrait(invListFields);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
index a5515b3..8d863b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
@@ -49,16 +50,18 @@
     private ITreeIndexTupleWriter tupleWriter;
 
     public VariableSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId,
-            ITypeTraits[] invListFields, IIndexCursorStats stats) throws HyracksDataException {
-        super(bufferCache, fileId, invListFields, stats);
+            ITypeTraits[] invListFields, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        super(bufferCache, fileId, invListFields, stats, nullTypeTraits, nullIntrospector);
         this.isInit = true;
-        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields);
-        this.tupleWriter = new TypeAwareTupleWriter(invListFields, null, null);
+        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields, nullTypeTraits);
+        this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
     }
 
     public VariableSizeElementOnDiskInvertedListCursor(IBufferCache bufferCache, int fileId,
-            ITypeTraits[] invListFields, IHyracksTaskContext ctx, IIndexCursorStats stats) throws HyracksDataException {
-        super(bufferCache, fileId, invListFields, ctx, stats);
+            ITypeTraits[] invListFields, IHyracksTaskContext ctx, IIndexCursorStats stats, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
+        super(bufferCache, fileId, invListFields, ctx, stats, nullTypeTraits, nullIntrospector);
         isInit = true;
     }
 
@@ -69,8 +72,8 @@
         // Note that the cursors can be re-used in the upper-layer callers so we need to reset the state variables when open()
         currentElementIxForScan = 0;
         isInit = true;
-        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields);
-        this.tupleWriter = new TypeAwareTupleWriter(invListFields, null, null);
+        this.tupleReference = new VariableSizeInvertedListTupleReference(invListFields, nullTypeTraits);
+        this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
index 75d0cfe..f71f347 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
@@ -54,12 +55,12 @@
     private IInvertedListTupleReference tupleReference;
     private ITreeIndexTupleWriter tupleWriter;
 
-    public VariableSizeInvertedListSearchResultFrameTupleAccessor(int frameSize, ITypeTraits[] fields)
-            throws HyracksDataException {
+    public VariableSizeInvertedListSearchResultFrameTupleAccessor(int frameSize, ITypeTraits[] fields,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         super(frameSize, fields);
 
-        this.tupleWriter = new TypeAwareTupleWriter(fields, null, null);
-        this.tupleReference = new VariableSizeInvertedListTupleReference(fields);
+        this.tupleWriter = new TypeAwareTupleWriter(fields, nullTypeTraits, nullIntrospector);
+        this.tupleReference = new VariableSizeInvertedListTupleReference(fields, nullTypeTraits);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
index 1e69076..997c93a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
@@ -28,17 +28,18 @@
 
 public class VariableSizeInvertedListTupleReference extends AbstractInvertedListTupleReference {
 
-    private ITreeIndexTupleReference tupleReference;
+    private final ITreeIndexTupleReference tupleReference;
 
     @Override
     protected void verifyTypeTrait() throws HyracksDataException {
         InvertedIndexUtils.verifyHasVarSizeTypeTrait(typeTraits);
     }
 
-    public VariableSizeInvertedListTupleReference(ITypeTraits[] typeTraits) throws HyracksDataException {
+    public VariableSizeInvertedListTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits)
+            throws HyracksDataException {
         super(typeTraits);
 
-        this.tupleReference = new TypeAwareTupleReference(typeTraits, null);
+        this.tupleReference = new TypeAwareTupleReference(typeTraits, nullTypeTraits);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 2fb620c..aa6f7c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -103,8 +103,8 @@
         if (bufferManager == null) {
             throw HyracksDataException.create(ErrorCode.CANNOT_CONTINUE_TEXT_SEARCH_BUFFER_MANAGER_IS_NULL);
         }
-        this.finalSearchResult =
-                new InvertedIndexFinalSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
+        this.finalSearchResult = new InvertedIndexFinalSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+                invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
         this.invListMerger = new InvertedListMerger(ctx, invIndex, bufferManager);
         this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
         this.invListCursorFactory = new InvertedListCursorFactory(invIndex, ctx);
@@ -118,9 +118,10 @@
         this.queryTokenAppender = new FrameTupleAppenderAccessor(QUERY_TOKEN_REC_DESC);
         this.queryTokenAppender.reset(queryTokenFrame, true);
         this.isSingleInvertedList = false;
-        this.searchResultTuple = InvertedIndexUtils.createInvertedListTupleReference(invIndex.getInvListTypeTraits());
+        this.searchResultTuple = InvertedIndexUtils.createInvertedListTupleReference(invIndex.getInvListTypeTraits(),
+                invIndex.getNullTypeTraits());
         this.searchResultFta = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(),
-                invIndex.getInvListTypeTraits());
+                invIndex.getInvListTypeTraits(), invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
     }
 
     protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
index e5f4622..f07983f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 
 /**
  * This is an in-memory based storage for final results of inverted-index searches.
@@ -34,8 +35,9 @@
 public class InvertedIndexFinalSearchResult extends InvertedIndexSearchResult {
 
     public InvertedIndexFinalSearchResult(ITypeTraits[] invListFields, IHyracksTaskContext ctx,
-            ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
-        super(invListFields, ctx, bufferManager);
+            ISimpleFrameBufferManager bufferManager, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
+        super(invListFields, ctx, bufferManager, nullTypeTraits, nullIntrospector);
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
index 6958fd5..35d0dd8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
@@ -37,6 +37,7 @@
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
 import org.apache.hyracks.dataflow.std.buffermanager.BufferManagerBackedVSizeFrame;
 import org.apache.hyracks.dataflow.std.buffermanager.ISimpleFrameBufferManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListSearchResultFrameTupleAppender;
@@ -84,14 +85,16 @@
     protected byte[] tempBytes;
 
     public InvertedIndexSearchResult(ITypeTraits[] invListFields, IHyracksTaskContext ctx,
-            ISimpleFrameBufferManager bufferManager) throws HyracksDataException {
+            ISimpleFrameBufferManager bufferManager, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         this.invListFields = invListFields;
-        this.tupleWriter = new TypeAwareTupleWriter(invListFields, null, null);
+        this.tupleWriter = new TypeAwareTupleWriter(invListFields, nullTypeTraits, nullIntrospector);
         initTypeTraits(invListFields);
         this.ctx = ctx;
         appender = new InvertedListSearchResultFrameTupleAppender(ctx.getInitialFrameSize());
-        accessor = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits);
-        tuple = InvertedIndexUtils.createInvertedListTupleReference(typeTraits);
+        accessor = InvertedIndexUtils.createInvertedListFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits,
+                nullTypeTraits, nullIntrospector);
+        tuple = InvertedIndexUtils.createInvertedListTupleReference(typeTraits, nullTypeTraits);
         this.bufferManager = bufferManager;
         this.isInReadMode = false;
         this.isWriteFinished = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
index deea12a..8bf16fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
@@ -92,8 +92,10 @@
     public InvertedListMerger(IHyracksTaskContext ctx, IInvertedIndex invIndex, ISimpleFrameBufferManager bufferManager)
             throws HyracksDataException {
         this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
-        this.prevSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
-        this.newSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager);
+        this.prevSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+                invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
+        this.newSearchResult = new InvertedIndexSearchResult(invIndex.getInvListTypeTraits(), ctx, bufferManager,
+                invIndex.getNullTypeTraits(), invIndex.getNullIntrospector());
     }
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index bee6be6..aefb461 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManager;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -84,41 +85,45 @@
             IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
-            throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef,
+                nullTypeTraits, nullIntrospector);
     }
 
     public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
             IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
-            throws HyracksDataException {
+            IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         return new PartitionedInMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits,
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                fullTextConfigEvaluatorFactory, btreeFileRef);
+                fullTextConfigEvaluatorFactory, btreeFileRef, nullTypeTraits, nullIntrospector);
     }
 
     public static OnDiskInvertedIndex createOnDiskInvertedIndex(IIOManager ioManager, IBufferCache bufferCache,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile,
-            IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
         FileReference btreeFile = getBTreeFile(ioManager, invListsFile);
         return new OnDiskInvertedIndex(bufferCache, builder, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory);
+                tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     public static PartitionedOnDiskInvertedIndex createPartitionedOnDiskInvertedIndex(IIOManager ioManager,
             IBufferCache bufferCache, ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile,
-            IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
         FileReference btreeFile = getBTreeFile(ioManager, invListsFile);
         return new PartitionedOnDiskInvertedIndex(bufferCache, builder, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory);
+                tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile, pageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 
     public static FileReference getBTreeFile(IIOManager ioManager, FileReference invListsFile)
@@ -128,9 +133,10 @@
 
     public static BTreeFactory createDeletedKeysBTreeFactory(IIOManager ioManager, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, IBufferCache diskBufferCache,
-            IPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+            IPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector)
+            throws HyracksDataException {
         BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
-                new BTreeTypeAwareTupleWriterFactory(invListTypeTraits, false, null, null);
+                new BTreeTypeAwareTupleWriterFactory(invListTypeTraits, false, nullTypeTraits, nullIntrospector);
         ITreeIndexFrameFactory leafFrameFactory =
                 BTreeUtils.getLeafFrameFactory(tupleWriterFactory, BTreeLeafFrameType.REGULAR_NSM);
         ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
@@ -148,11 +154,11 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
-            boolean durable, IMetadataPageManagerFactory pageManagerFactory, ITracer tracer)
-            throws HyracksDataException {
+            boolean durable, IMetadataPageManagerFactory pageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
-                invListCmpFactories, diskBufferCache, pageManagerFactory);
+                invListCmpFactories, diskBufferCache, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
         for (int i = 0; i < invListCmpFactories.length; i++) {
@@ -165,17 +171,17 @@
                 new LSMInvertedIndexFileManager(ioManager, onDiskDirFileRef, deletedKeysBTreeFactory);
 
         IInvertedListBuilderFactory invListBuilderFactory =
-                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits);
-        OnDiskInvertedIndexFactory invIndexFactory =
-                new OnDiskInvertedIndexFactory(ioManager, diskBufferCache, invListBuilderFactory, invListTypeTraits,
-                        invListCmpFactories, tokenTypeTraits, tokenCmpFactories, fileManager, pageManagerFactory);
+                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits, nullTypeTraits, nullIntrospector);
+        OnDiskInvertedIndexFactory invIndexFactory = new OnDiskInvertedIndexFactory(ioManager, diskBufferCache,
+                invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                fileManager, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         ComponentFilterHelper filterHelper = null;
         LSMComponentFilterFrameFactory filterFrameFactory = null;
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -188,7 +194,7 @@
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
     }
 
     public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -201,10 +207,11 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
-            boolean durable, IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
+            boolean durable, IPageManagerFactory pageManagerFactory, ITracer tracer, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
-                invListCmpFactories, diskBufferCache, pageManagerFactory);
+                invListCmpFactories, diskBufferCache, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
         for (int i = 0; i < invListCmpFactories.length; i++) {
@@ -217,17 +224,17 @@
                 new LSMInvertedIndexFileManager(ioManager, onDiskDirFileRef, deletedKeysBTreeFactory);
 
         IInvertedListBuilderFactory invListBuilderFactory =
-                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits);
+                new InvertedListBuilderFactory(tokenTypeTraits, invListTypeTraits, nullTypeTraits, nullIntrospector);
         PartitionedOnDiskInvertedIndexFactory invIndexFactory = new PartitionedOnDiskInvertedIndexFactory(ioManager,
                 diskBufferCache, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, fileManager, pageManagerFactory);
+                tokenCmpFactories, fileManager, pageManagerFactory, nullTypeTraits, nullIntrospector);
 
         ComponentFilterHelper filterHelper = null;
         LSMComponentFilterFrameFactory filterFrameFactory = null;
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -240,7 +247,7 @@
                 invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                 fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
-                invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+                invertedIndexFieldsForNonBulkLoadOps, durable, tracer, nullTypeTraits, nullIntrospector);
     }
 
     public static boolean checkTypeTraitsAllFixed(ITypeTraits[] typeTraits) {
@@ -266,21 +273,22 @@
         }
     }
 
-    public static IInvertedListTupleReference createInvertedListTupleReference(ITypeTraits[] typeTraits)
-            throws HyracksDataException {
+    public static IInvertedListTupleReference createInvertedListTupleReference(ITypeTraits[] typeTraits,
+            ITypeTraits nullTypeTraits) throws HyracksDataException {
         if (checkTypeTraitsAllFixed(typeTraits)) {
             return new FixedSizeInvertedListTupleReference(typeTraits);
         } else {
-            return new VariableSizeInvertedListTupleReference(typeTraits);
+            return new VariableSizeInvertedListTupleReference(typeTraits, nullTypeTraits);
         }
     }
 
-    public static IFrameTupleAccessor createInvertedListFrameTupleAccessor(int frameSize, ITypeTraits[] typeTraits)
-            throws HyracksDataException {
+    public static IFrameTupleAccessor createInvertedListFrameTupleAccessor(int frameSize, ITypeTraits[] typeTraits,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
         if (checkTypeTraitsAllFixed(typeTraits)) {
             return new FixedSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits);
         } else {
-            return new VariableSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits);
+            return new VariableSizeInvertedListSearchResultFrameTupleAccessor(frameSize, typeTraits, nullTypeTraits,
+                    nullIntrospector);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
index 9404d91..e6d50d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -61,12 +62,13 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
     }
 
     private ExternalRTreeLocalResource(IPersistedResourceRegistry registry, JsonNode json,
@@ -90,7 +92,7 @@
                 opTrackerProvider.getOperationTracker(ncServiceCtx, this),
                 ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
                 linearizeCmpFactory, buddyBTreeFields, durable, isPointMBR, metadataPageManagerFactory,
-                ncServiceCtx.getTracer());
+                ncServiceCtx.getTracer(), nullTypeTraits, nullIntrospector);
 
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
index 1f4971d..ca8caec 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -49,12 +50,13 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, null,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
     }
 
     @Override
@@ -63,7 +65,8 @@
                 filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
                 pageWriteCallbackFactory, metadataPageManagerFactory, ioSchedulerProvider, mergePolicyFactory,
                 mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate);
+                linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR, bloomFilterFalsePositiveRate,
+                nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
index c7a59f4..dd21f7c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -73,10 +74,12 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -117,7 +120,7 @@
                 opTrackerProvider.getOperationTracker(ncServiceCtx, this),
                 ioSchedulerProvider.getIoScheduler(ncServiceCtx), ioOpCallbackFactory, pageWriteCallbackFactory,
                 linearizeCmpFactory, rtreeFields, buddyBTreeFields, filterTypeTraits, filterCmpFactories, filterFields,
-                durable, isPointMBR, metadataPageManagerFactory);
+                durable, isPointMBR, metadataPageManagerFactory, nullTypeTraits, nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
index 31f78d8..873a023 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -60,10 +61,12 @@
             Map<String, String> mergePolicyProperties, boolean durable, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, int[] buddyBTreeFields,
-            boolean isPointMBR, double bloomFilterFalsePositiveRate) {
+            boolean isPointMBR, double bloomFilterFalsePositiveRate, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -81,6 +84,6 @@
                 pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
                 mergePolicyFactory, mergePolicyProperties, durable, btreeCmpFactories, valueProviderFactories,
                 rtreePolicyType, linearizeCmpFactory, rtreeFields, buddyBTreeFields, isPointMBR,
-                bloomFilterFalsePositiveRate);
+                bloomFilterFalsePositiveRate, nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
index a0d0220..33b9da6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.api.io.IJsonSerializable;
 import org.apache.hyracks.api.io.IPersistedResourceRegistry;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -70,10 +71,12 @@
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable) {
+            ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR, boolean durable,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         super(path, storageManager, typeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.btreeCmpFactories = btreeCmpFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -108,7 +111,8 @@
                 mergePolicyFactory.createMergePolicy(mergePolicyProperties, serviceCtx),
                 opTrackerProvider.getOperationTracker(serviceCtx, this), ioSchedulerProvider.getIoScheduler(serviceCtx),
                 ioOpCallbackFactory, pageWriteCallbackFactory, linearizeCmpFactory, rtreeFields, filterTypeTraits,
-                filterCmpFactories, filterFields, durable, isPointMBR, metadataPageManagerFactory);
+                filterCmpFactories, filterFields, durable, isPointMBR, metadataPageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
index ed4c5b5..e9b4c90 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
@@ -57,10 +58,12 @@
             Map<String, String> mergePolicyProperties, boolean durable,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] rtreeFields, boolean isPointMBR,
-            IBinaryComparatorFactory[] btreeComparatorFactories) {
+            IBinaryComparatorFactory[] btreeComparatorFactories, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         super(storageManager, rteeTypeTraits, rtreeCmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
                 opTrackerFactory, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
-                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, null, null);
+                vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable, nullTypeTraits,
+                nullIntrospector);
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
         this.linearizeCmpFactory = linearizeCmpFactory;
@@ -75,6 +78,7 @@
                 cmpFactories, filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider,
                 ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider,
                 ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, btreeComparatorFactories,
-                valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR, durable);
+                valueProviderFactories, rtreePolicyType, linearizeCmpFactory, rtreeFields, isPointMBR, durable,
+                nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
index 3b7c150..4f3b49b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
@@ -21,11 +21,13 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 
 public class LSMRTreeCopyTupleWriter extends LSMRTreeTupleWriter {
-    public LSMRTreeCopyTupleWriter(ITypeTraits[] typeTraits) {
+    public LSMRTreeCopyTupleWriter(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
         // Third parameter is never used locally, just give false.
-        super(typeTraits, false);
+        super(typeTraits, false, nullTypeTraits, nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
index 6585ffe..41fa508 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
@@ -20,18 +20,20 @@
 package org.apache.hyracks.storage.am.lsm.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 
 public class LSMRTreeCopyTupleWriterFactory extends RTreeTypeAwareTupleWriterFactory {
     private static final long serialVersionUID = 1L;
 
-    public LSMRTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits) {
-        super(typeTraits);
+    public LSMRTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
     }
 
     @Override
     public RTreeTypeAwareTupleWriter createTupleWriter() {
-        return new LSMRTreeCopyTupleWriter(typeTraits);
+        return new LSMRTreeCopyTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
index 86a6a39..3e25e8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
@@ -26,8 +26,8 @@
 
 public class LSMRTreeTupleReference extends RTreeTypeAwareTupleReference implements ILSMTreeTupleReference {
 
-    public LSMRTreeTupleReference(ITypeTraits[] typeTraits) {
-        super(typeTraits);
+    public LSMRTreeTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
+        super(typeTraits, nullTypeTraits);
     }
 
     @Override
@@ -45,4 +45,10 @@
     public int getTupleStart() {
         return tupleStartOff;
     }
+
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        // 1 for antimatter
+        return fieldIdx + 1;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
index ef9eb47..7c05daa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
@@ -34,8 +34,8 @@
     private final boolean antimatterAware;
 
     public LSMRTreeTupleReferenceForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
-            boolean antimatterAware) {
-        super(typeTraits);
+            boolean antimatterAware, ITypeTraits nullTypeTraits) {
+        super(typeTraits, nullTypeTraits);
         this.inputKeyFieldCount = keyFieldCount;
         this.inputTotalFieldCount = keyFieldCount + valueFieldCount;
         this.storedKeyFieldCount = keyFieldCount / 2;
@@ -55,8 +55,15 @@
         int cumul = 0;
         //step1. decode field slots for stored key
         for (int i = 0; i < storedKeyFieldCount; i++) {
+            if (!typeTraits[i].isFixedLength()) {
+                cumul += encDec.decode();
+            } else if (nullTypeTraits != null && isNull(buf, tupleStartOff, i)) {
+                cumul += nullTypeTraits.getFixedLength();
+            } else {
+                cumul += typeTraits[i].getFixedLength();
+            }
             //key or value fields
-            cumul += typeTraits[i].getFixedLength();
+            //            cumul += typeTraits[i].getFixedLength();
             decodedFieldSlots[field++] = cumul;
         }
         //step2. decode field slots for non-stored (duplicated point) key
@@ -68,14 +75,22 @@
         encDec.reset(buf, tupleStartOff + nullFlagsBytes);
         for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++) {
             if (!typeTraits[i].isFixedLength()) {
-                //value fields
                 cumul += encDec.decode();
-                decodedFieldSlots[field++] = cumul;
+            } else if (nullTypeTraits != null && isNull(buf, tupleStartOff, i)) {
+                cumul += nullTypeTraits.getFixedLength();
             } else {
-                //key or value fields
                 cumul += typeTraits[i].getFixedLength();
-                decodedFieldSlots[field++] = cumul;
             }
+            decodedFieldSlots[field++] = cumul;
+            //            if (!typeTraits[i].isFixedLength()) {
+            //                //value fields
+            //                cumul += encDec.decode();
+            //                decodedFieldSlots[field++] = cumul;
+            //            } else {
+            //                //key or value fields
+            //                cumul += typeTraits[i].getFixedLength();
+            //                decodedFieldSlots[field++] = cumul;
+            //            }
         }
 
         dataStartOff = encDec.getPos();
@@ -145,4 +160,10 @@
         // Check antimatter bit.
         return BitOperationUtils.getBit(buf, tupleStartOff, ANTIMATTER_BIT_OFFSET);
     }
+
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        // 1 for antimatter
+        return antimatterAware ? fieldIdx + 1 : fieldIdx;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
index 5650cbb..6417839 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
@@ -30,14 +31,15 @@
 public class LSMRTreeTupleWriter extends RTreeTypeAwareTupleWriter implements ILSMTreeTupleWriter {
     private boolean isAntimatter;
 
-    public LSMRTreeTupleWriter(ITypeTraits[] typeTraits, boolean isAntimatter) {
-        super(typeTraits);
+    public LSMRTreeTupleWriter(ITypeTraits[] typeTraits, boolean isAntimatter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.isAntimatter = isAntimatter;
     }
 
     @Override
     public LSMRTreeTupleReference createTupleReference() {
-        return new LSMRTreeTupleReference(typeTraits);
+        return new LSMRTreeTupleReference(typeTraits, nullTypeTraits);
     }
 
     @Override
@@ -72,4 +74,9 @@
         this.isAntimatter = isAntimatter;
     }
 
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        // 1 for antimatter
+        return fieldIdx + 1;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
index 4ad5e3a..040c805 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 
 public class LSMRTreeTupleWriterFactory extends RTreeTypeAwareTupleWriterFactory {
@@ -27,14 +28,15 @@
     private static final long serialVersionUID = 1L;
     private final boolean isAntimatter;
 
-    public LSMRTreeTupleWriterFactory(ITypeTraits[] typeTraits, boolean isAntimatter) {
-        super(typeTraits);
+    public LSMRTreeTupleWriterFactory(ITypeTraits[] typeTraits, boolean isAntimatter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.isAntimatter = isAntimatter;
     }
 
     @Override
     public LSMRTreeTupleWriter createTupleWriter() {
-        return new LSMRTreeTupleWriter(typeTraits, isAntimatter);
+        return new LSMRTreeTupleWriter(typeTraits, isAntimatter, nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
index 6a87af2..625883f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
 import org.apache.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 
@@ -32,8 +33,8 @@
     private final boolean isAntimatter;
 
     public LSMRTreeTupleWriterFactoryForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
-            boolean antimatterAware, boolean isDelete) {
-        super(typeTraits);
+            boolean antimatterAware, boolean isDelete, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.keyFieldCount = keyFieldCount;
         this.valueFieldCount = valueFieldCount;
         this.antimatterAware = antimatterAware;
@@ -43,7 +44,7 @@
     @Override
     public RTreeTypeAwareTupleWriter createTupleWriter() {
         return new LSMRTreeTupleWriterForPointMBR(typeTraits, keyFieldCount, valueFieldCount, antimatterAware,
-                isAntimatter);
+                isAntimatter, nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
index ae2d642..7542524 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
@@ -23,6 +23,7 @@
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.util.BitOperationUtils;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleWriter;
@@ -54,8 +55,9 @@
     private boolean isAntimatter;
 
     public LSMRTreeTupleWriterForPointMBR(ITypeTraits[] typeTraits, int keyFieldCount, int valueFieldCount,
-            boolean antimatterAware, boolean isAntimatter) {
-        super(typeTraits);
+            boolean antimatterAware, boolean isAntimatter, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
         this.inputKeyFieldCount = keyFieldCount;
         this.valueFieldCount = valueFieldCount;
         this.inputTotalFieldCount = keyFieldCount + valueFieldCount;
@@ -81,14 +83,15 @@
 
     @Override
     public LSMRTreeTupleReferenceForPointMBR createTupleReference() {
-        return new LSMRTreeTupleReferenceForPointMBR(typeTraits, inputKeyFieldCount, valueFieldCount, antimatterAware);
+        return new LSMRTreeTupleReferenceForPointMBR(typeTraits, inputKeyFieldCount, valueFieldCount, antimatterAware,
+                nullTypeTraits);
     }
 
     @Override
     public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
         int runner = targetOff;
         int nullFlagsBytes = getNullFlagsBytes(tuple);
-        // write null indicator bits
+        // reset null indicator bits
         for (int i = 0; i < nullFlagsBytes; i++) {
             targetBuf[runner++] = (byte) 0;
         }
@@ -102,8 +105,14 @@
 
         // write key fields
         for (int i = 0; i < storedKeyFieldCount; i++) {
-            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
-            runner += tuple.getFieldLength(i);
+            byte[] fieldData = tuple.getFieldData(i);
+            int fieldOffset = tuple.getFieldStart(i);
+            int fieldLength = tuple.getFieldLength(i);
+            if (nullIntrospector != null && nullIntrospector.isNull(fieldData, fieldOffset, fieldLength)) {
+                setNullFlag(targetBuf, targetOff, i);
+            }
+            System.arraycopy(fieldData, fieldOffset, targetBuf, runner, fieldLength);
+            runner += fieldLength;
         }
         // write value fields
         for (int i = inputKeyFieldCount; i < inputTotalFieldCount; i++) {
@@ -138,7 +147,7 @@
 
     @Override
     protected int getNullFlagsBytes(ITupleReference tuple) {
-        return BitOperationUtils.getFlagBytes(storedTotalFieldCount + (antimatterAware ? 1 : 0));
+        return BitOperationUtils.getFlagBytes(storedTotalFieldCount + valueFieldCount + (antimatterAware ? 1 : 0));
     }
 
     @Override
@@ -167,4 +176,9 @@
     public void setAntimatter(boolean isAntimatter) {
         this.isAntimatter = isAntimatter;
     }
+
+    @Override
+    protected int getAdjustedFieldIdx(int fieldIdx) {
+        return antimatterAware ? fieldIdx + 1 : fieldIdx;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index cf44b2f..a436c43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.tuples.BTreeTypeAwareTupleWriterFactory;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
@@ -85,7 +86,8 @@
             ILSMPageWriteCallbackFactory pageWriteCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
             int[] rtreeFields, int[] buddyBTreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
-            IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+            IMetadataPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         int valueFieldCount = buddyBTreeFields.length;
         int keyFieldCount = typeTraits.length - valueFieldCount;
         ITypeTraits[] btreeTypeTraits = new ITypeTraits[valueFieldCount];
@@ -93,16 +95,16 @@
             btreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
         }
         RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
-                new RTreeTypeAwareTupleWriterFactory(typeTraits);
+                new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory = null;
         if (isPointMBR) {
-            rtreeLeafFrameTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, false, false);
+            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, false, false, nullTypeTraits, nullIntrospector);
         } else {
             rtreeLeafFrameTupleWriterFactory = rtreeInteriorFrameTupleWriterFactory;
         }
         BTreeTypeAwareTupleWriterFactory btreeTupleWriterFactory =
-                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, null, null);
+                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, nullTypeTraits, nullIntrospector);
         RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
                 rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
         RTreeFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeLeafFrameTupleWriterFactory,
@@ -128,7 +130,7 @@
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -155,28 +157,33 @@
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             ILinearizeComparatorFactory linearizerCmpFactory, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, boolean durable, boolean isPointMBR,
-            IMetadataPageManagerFactory freePageManagerFactory) throws HyracksDataException {
+            IMetadataPageManagerFactory freePageManagerFactory, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) throws HyracksDataException {
         RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
-                new LSMRTreeTupleWriterFactory(typeTraits, false);
+                new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory;
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameCopyTupleWriterFactory;
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameBulkLoadWriterFactory;
         if (isPointMBR) {
             int keyFieldCount = rtreeCmpFactories.length;
             int valueFieldCount = btreeComparatorFactories.length - keyFieldCount;
-            rtreeLeafFrameTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
-            rtreeLeafFrameCopyTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
-            rtreeLeafFrameBulkLoadWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, true, false);
+            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameCopyTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameBulkLoadWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, true, false, nullTypeTraits, nullIntrospector);
         } else {
-            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
-            rtreeLeafFrameCopyTupleWriterFactory = new LSMRTreeCopyTupleWriterFactory(typeTraits);
-            rtreeLeafFrameBulkLoadWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
+            rtreeLeafFrameTupleWriterFactory =
+                    new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameCopyTupleWriterFactory =
+                    new LSMRTreeCopyTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
+            rtreeLeafFrameBulkLoadWriterFactory =
+                    new LSMRTreeTupleWriterFactory(typeTraits, false, nullTypeTraits, nullIntrospector);
         }
 
-        LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
+        LSMRTreeTupleWriterFactory btreeTupleWriterFactory =
+                new LSMRTreeTupleWriterFactory(typeTraits, true, nullTypeTraits, nullIntrospector);
 
         RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
                 rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
@@ -219,7 +226,7 @@
         LSMComponentFilterManager filterManager = null;
         if (filterCmpFactories != null) {
             TypeAwareTupleWriterFactory filterTupleWriterFactory =
-                    new TypeAwareTupleWriterFactory(filterTypeTraits, null, null);
+                    new TypeAwareTupleWriterFactory(filterTypeTraits, nullTypeTraits, nullIntrospector);
             filterHelper = new ComponentFilterHelper(filterTupleWriterFactory, filterCmpFactories);
             filterFrameFactory = new LSMComponentFilterFrameFactory(filterTupleWriterFactory);
             filterManager = new LSMComponentFilterManager(filterFrameFactory);
@@ -247,8 +254,8 @@
             ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory, int[] buddyBTreeFields, boolean durable,
-            boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer)
-            throws HyracksDataException {
+            boolean isPointMBR, IMetadataPageManagerFactory freePageManagerFactory, ITracer tracer,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) throws HyracksDataException {
 
         int keyFieldCount = rtreeCmpFactories.length;
         int valueFieldCount = typeTraits.length - keyFieldCount;
@@ -257,16 +264,16 @@
             btreeTypeTraits[i] = typeTraits[buddyBTreeFields[i]];
         }
         RTreeTypeAwareTupleWriterFactory rtreeInteriorFrameTupleWriterFactory =
-                new RTreeTypeAwareTupleWriterFactory(typeTraits);
+                new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
         RTreeTypeAwareTupleWriterFactory rtreeLeafFrameTupleWriterFactory = null;
         if (isPointMBR) {
-            rtreeLeafFrameTupleWriterFactory =
-                    new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount, valueFieldCount, false, false);
+            rtreeLeafFrameTupleWriterFactory = new LSMRTreeTupleWriterFactoryForPointMBR(typeTraits, keyFieldCount,
+                    valueFieldCount, false, false, nullTypeTraits, nullIntrospector);
         } else {
             rtreeLeafFrameTupleWriterFactory = rtreeInteriorFrameTupleWriterFactory;
         }
         BTreeTypeAwareTupleWriterFactory btreeTupleWriterFactory =
-                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, null, null);
+                new BTreeTypeAwareTupleWriterFactory(btreeTypeTraits, false, nullTypeTraits, nullIntrospector);
         RTreeFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
                 rtreeInteriorFrameTupleWriterFactory, valueProviderFactories, rtreePolicyType, isPointMBR);
         RTreeFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeLeafFrameTupleWriterFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
index f9eb844..9195982 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -42,10 +43,13 @@
     private final IPageManagerFactory pageManagerFactory;
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
     private final RTreePolicyType rtreePolicyType;
+    private final ITypeTraits nullTypeTraits;
+    private final INullIntrospector nullIntrospector;
 
     public RTreeResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType) {
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         this.path = path;
         this.storageManager = storageManager;
         this.typeTraits = typeTraits;
@@ -53,6 +57,8 @@
         this.pageManagerFactory = pageManagerFactory;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
@@ -60,7 +66,8 @@
         IIOManager ioManager = ctx.getIoManager();
         FileReference resourceRef = ioManager.resolve(path);
         return RTreeUtils.createRTree(storageManager.getBufferCache(ctx), typeTraits, valueProviderFactories,
-                comparatorFactories, rtreePolicyType, resourceRef, false, pageManagerFactory);
+                comparatorFactories, rtreePolicyType, resourceRef, false, pageManagerFactory, nullTypeTraits,
+                nullIntrospector);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
index 1a3bb0b..9ef59a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
@@ -21,6 +21,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
@@ -37,22 +38,27 @@
     private final IPageManagerFactory pageManagerFactory;
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
     private final RTreePolicyType rtreePolicyType;
+    private final ITypeTraits nullTypeTraits;
+    private final INullIntrospector nullIntrospector;
 
     public RTreeResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, IPageManagerFactory pageManagerFactory,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType) {
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
         this.storageManager = storageManager;
         this.typeTraits = typeTraits;
         this.comparatorFactories = comparatorFactories;
         this.pageManagerFactory = pageManagerFactory;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
+        this.nullTypeTraits = nullTypeTraits;
+        this.nullIntrospector = nullIntrospector;
     }
 
     @Override
     public IResource createResource(FileReference fileRef) {
         return new RTreeResource(fileRef.getRelativePath(), storageManager, typeTraits, comparatorFactories,
-                pageManagerFactory, valueProviderFactories, rtreePolicyType);
+                pageManagerFactory, valueProviderFactories, rtreePolicyType, nullTypeTraits, nullIntrospector);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
index 8f7f3f0..8f1b660 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
@@ -26,8 +26,8 @@
 
 public class RTreeTypeAwareTupleReference extends TypeAwareTupleReference implements ITreeIndexTupleReference {
 
-    public RTreeTypeAwareTupleReference(ITypeTraits[] typeTraits) {
-        super(typeTraits, null);
+    public RTreeTypeAwareTupleReference(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits) {
+        super(typeTraits, nullTypeTraits);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
index 42a9662..02bf4a2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
@@ -22,14 +22,16 @@
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
 
 public class RTreeTypeAwareTupleWriter extends TypeAwareTupleWriter {
 
-    public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits) {
-        super(typeTraits, null, null);
+    public RTreeTypeAwareTupleWriter(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
     }
 
     public int writeTupleFields(ITreeIndexTupleReference[] refs, int startField, ByteBuffer targetBuf, int targetOff) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
index 0069cc3..56241e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
@@ -20,18 +20,20 @@
 package org.apache.hyracks.storage.am.rtree.tuples;
 
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 
 public class RTreeTypeAwareTupleWriterFactory extends TypeAwareTupleWriterFactory {
 
     private static final long serialVersionUID = 1L;
 
-    public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits) {
-        super(typeTraits, null, null);
+    public RTreeTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits, ITypeTraits nullTypeTraits,
+            INullIntrospector nullIntrospector) {
+        super(typeTraits, nullTypeTraits, nullIntrospector);
     }
 
     @Override
     public RTreeTypeAwareTupleWriter createTupleWriter() {
-        return new RTreeTypeAwareTupleWriter(typeTraits);
+        return new RTreeTypeAwareTupleWriter(typeTraits, nullTypeTraits, nullIntrospector);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
index 204248f..7df5802 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
@@ -25,6 +25,7 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.data.std.api.IPointableFactory;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.INullIntrospector;
 import org.apache.hyracks.storage.am.common.api.IPageManagerFactory;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -45,9 +46,10 @@
     public static RTree createRTree(IBufferCache bufferCache, ITypeTraits[] typeTraits,
             IPrimitiveValueProviderFactory[] valueProviderFactories, IBinaryComparatorFactory[] cmpFactories,
             RTreePolicyType rtreePolicyType, FileReference file, boolean isPointMBR,
-            IPageManagerFactory pageManagerFactory) {
+            IPageManagerFactory pageManagerFactory, ITypeTraits nullTypeTraits, INullIntrospector nullIntrospector) {
 
-        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
+        RTreeTypeAwareTupleWriterFactory tupleWriterFactory =
+                new RTreeTypeAwareTupleWriterFactory(typeTraits, nullTypeTraits, nullIntrospector);
         ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
                 valueProviderFactories, rtreePolicyType, isPointMBR);
         ITreeIndexFrameFactory leafFrameFactory =
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
index 0c626df..1b17ef9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
@@ -65,7 +65,7 @@
                 .resolveAbsolutePath(harness.getInvListsFileRef().getFile().getAbsolutePath() + "_btree");
         index = new OnDiskInvertedIndex(harness.getDiskBufferCache(), invListBuilder, invListTypeTraits,
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, harness.getInvListsFileRef(), btreeFile,
-                harness.getMetadataPageManagerFactory());
+                harness.getMetadataPageManagerFactory(), null, null);
 
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 3e8db34..e745f5d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -159,7 +159,8 @@
                 invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCaches().get(0),
                         new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()), null,
+                        null);
                 break;
             }
             case PARTITIONED_INMEMORY: {
@@ -167,47 +168,53 @@
                         harness.getVirtualBufferCaches().get(0),
                         new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+                        fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()), null,
+                        null);
                 break;
             }
             case ONDISK: {
                 invIndex = InvertedIndexUtils.createOnDiskInvertedIndex(ioManager, harness.getDiskBufferCache(),
                         invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                        harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory());
+                        harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory(), null, null);
                 break;
             }
             case PARTITIONED_ONDISK: {
-                invIndex = InvertedIndexUtils.createPartitionedOnDiskInvertedIndex(ioManager,
-                        harness.getDiskBufferCache(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory());
+                invIndex =
+                        InvertedIndexUtils.createPartitionedOnDiskInvertedIndex(ioManager, harness.getDiskBufferCache(),
+                                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                                harness.getInvListsFileRef(), harness.getMetadataPageManagerFactory(), null, null);
                 break;
             }
             case LSM: {
-                invIndex = InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
-                        invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                        fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
-                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                        harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
-                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
-                        harness.getMetadataPageManagerFactory(),
-                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
-                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
+                invIndex =
+                        InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+                                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                                tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
+                                harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+                                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                                invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+                                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+                                harness.getMetadataPageManagerFactory(),
+                                new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                        ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
+                                null, null);
                 break;
             }
             case PARTITIONED_LSM: {
-                invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager,
-                        harness.getVirtualBufferCaches(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
-                        harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
-                        harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
-                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
-                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
-                        harness.getMetadataPageManagerFactory(),
-                        new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
-                                ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
+                invIndex = InvertedIndexUtils
+                        .createPartitionedLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
+                                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                                tokenizerFactory, fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(),
+                                harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+                                harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+                                harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
+                                invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+                                filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
+                                harness.getMetadataPageManagerFactory(),
+                                new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+                                        ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()),
+                                null, null);
                 break;
             }
             default: {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index 3ca25ab..9e6fb83 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -50,7 +50,7 @@
                 harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields, btreeFields,
                 filterTypeTraits, filterCmpFactories, filterFields, true, false,
-                harness.getMetadataPageManagerFactory());
+                harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Before
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 75f656d..baf54f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -50,7 +50,7 @@
                 harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), rtreeFields,
                 filterTypeTraits, filterCmpFactories, filterFields, true, false,
-                harness.getMetadataPageManagerFactory());
+                harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Before
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index bbf6263..980f21f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -66,7 +66,7 @@
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, btreeFields, null,
-                null, null, true, false, harness.getMetadataPageManagerFactory());
+                null, null, true, false, harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index 7d6e2e6..f40add9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -67,7 +67,7 @@
                 harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackFactory(),
                 harness.getPageWriteCallbackFactory(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), null, null, null, null, true,
-                false, harness.getMetadataPageManagerFactory());
+                false, harness.getMetadataPageManagerFactory(), null, null);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index 2140558..02742a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -109,7 +109,7 @@
                 typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
-                null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory);
+                null, btreeFields, null, null, null, true, isPointMBR, metadataPageManagerFactory, null, null);
         LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index f9d9046..ff953f1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -90,7 +90,7 @@
                 virtualBufferCaches, file, diskBufferCache, typeTraits, rtreeCmpFactories, btreeCmpFactories,
                 valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
                 pageWriteCallbackFactory, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length),
-                null, null, null, null, true, false, metadataPageManagerFactory);
+                null, null, null, null, true, false, metadataPageManagerFactory, null, null);
         LSMRTreeWithAntiMatterTuplesTestContext testCtx =
                 new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes, lsmTree);
         return testCtx;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
index 999fd65..cb68d77 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
@@ -54,7 +54,7 @@
             RTreePolicyType rtreePolicyType, int[] rtreeFields, int[] btreeFields, ITypeTraits[] filterTypeTraits,
             IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) throws HyracksDataException {
         return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits, valueProviderFactories, rtreeCmpFactories,
-                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory());
+                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory(), null, null);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index bef7ac7..19672dd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -76,7 +76,7 @@
     public static final IPrimitiveValueProviderFactory[] VALUE_PROVIDER_FACTORY =
             RTreeUtils.createPrimitiveValueProviderFactories(CMP_FACTORIES.length, IntegerPointable.FACTORY);
     public static final RTreeTypeAwareTupleWriterFactory TUPLE_WRITER_FACTORY =
-            new RTreeTypeAwareTupleWriterFactory(TYPE_TRAITS);
+            new RTreeTypeAwareTupleWriterFactory(TYPE_TRAITS, null, null);
     public static final ITreeIndexMetadataFrameFactory META_FRAME_FACTORY = new LIFOMetaDataFrameFactory();
     public static final ITreeIndexFrameFactory INTERIOR_FRAME_FACTORY = new RTreeNSMInteriorFrameFactory(
             TUPLE_WRITER_FACTORY, VALUE_PROVIDER_FACTORY, RTreePolicyType.RTREE, false);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
index faa63c5..02c267d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
@@ -61,7 +61,7 @@
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, int[] btreeFields) throws HyracksDataException {
         return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits, valueProviderFactories, rtreeCmpFactories,
-                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory());
+                rtreePolicyType, harness.getFileReference(), false, harness.getMetadataManagerFactory(), null, null);
 
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
index bf70dc1..06ee100 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
@@ -60,7 +60,7 @@
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
         RTree rtree = RTreeUtils.createRTree(bufferCache, typeTraits, valueProviderFactories, cmpFactories,
-                rtreePolicyType, file, false, pageManagerFactory);
+                rtreePolicyType, file, false, pageManagerFactory, null, null);
         RTreeTestContext testCtx = new RTreeTestContext(fieldSerdes, rtree);
         return testCtx;
     }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/3PdLt27FiRtUqUr6A : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Sun, 01 Aug 2021 00:08:58 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Murtadha Hubail <mh...@apache.org>:

Murtadha Hubail has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 3: Code-Review+2


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Mon, 02 Aug 2021 19:54:55 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Hello Anon. E. Moose #1000171, Jenkins, 

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

    https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604

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

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................

[ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees

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

Details:
When writing an index entry into RTree, set the corresponding null flags
for keys that are considered null. When reading an index entry,
examine the null flags to read the index entry properly.
Also, pass the null type traits and null introspector to inverted indexes.

Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
---
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
63 files changed, 498 insertions(+), 254 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/04/12604/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-MessageType: newpatchset

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

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


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sun, 01 Aug 2021 16:38:57 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/ekwUk422LEYc4ae56 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sun, 01 Aug 2021 15:39:02 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/FyPajhXZK3WE9t776 : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sun, 01 Aug 2021 18:03:58 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Sun, 01 Aug 2021 01:13:53 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has uploaded a new patch set (#2). ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................

[ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees

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

Details:
When writing an index entry into RTree, set the corresponding null flags
for keys that are considered null. When reading an index entry,
examine the null flags to read the index entry properly.
Also, pass the null type traits and null introspector to inverted indexes.

Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
---
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/AbstractOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/InvertedListBuilderFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/fixedsize/FixedSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementInvertedListBuilder.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeElementOnDiskInvertedListCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListSearchResultFrameTupleAccessor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/variablesize/VariableSizeInvertedListTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexFinalSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchResult.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/ExternalRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterLocalResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReferenceForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactoryForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterForPointMBR.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResource.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeResourceFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleReference.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriter.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/tuples/RTreeTypeAwareTupleWriterFactory.java
M hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/util/RTreeUtils.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeExamplesTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
M hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/utils/RTreeTestContext.java
63 files changed, 468 insertions(+), 242 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/04/12604/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-CC: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-MessageType: newpatchset

Change in asterixdb[master]: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604 )

Change subject: [ASTERIXDB-2791][IDX] Set null flags when writing an index entry into RTrees
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/dvBfejaq5R2b6h467 : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12604
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I88f5075019f317bccd937d071abadbd4f4b3c846
Gerrit-Change-Number: 12604
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sun, 01 Aug 2021 02:40:25 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment